carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ravipes...@apache.org
Subject [1/2] incubator-carbondata git commit: Size based blocklet
Date Tue, 14 Mar 2017 11:35:44 GMT
Repository: incubator-carbondata
Updated Branches:
  refs/heads/master c16917e75 -> 088240ef4


Size based blocklet


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/ebf13dc7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/ebf13dc7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/ebf13dc7

Branch: refs/heads/master
Commit: ebf13dc7d52c1ac07f6653399ad1e1e17cd96877
Parents: c16917e
Author: kumarvishal <kumarvishal.1802@gmail.com>
Authored: Sun Mar 12 23:58:36 2017 +0800
Committer: ravipesala <ravi.pesala@gmail.com>
Committed: Tue Mar 14 17:04:40 2017 +0530

----------------------------------------------------------------------
 .../constants/CarbonV3DataFormatConstants.java  | 17 ++--
 .../impl/btree/BlockletBTreeLeafNode.java       |  9 ++-
 .../core/util/CarbonMetadataUtil.java           | 81 +++++++++++++++++++-
 .../carbondata/core/util/CarbonProperties.java  | 41 +++++-----
 .../apache/carbondata/core/util/NodeHolder.java | 13 ++++
 .../store/CarbonFactDataHandlerColumnar.java    | 18 +++--
 .../store/writer/AbstractFactDataWriter.java    |  4 +-
 .../store/writer/CarbonFactDataWriter.java      |  5 +-
 .../writer/v1/CarbonFactDataWriterImplV1.java   | 13 ++--
 .../writer/v3/CarbonFactDataWriterImplV3.java   | 70 ++++++++++++++---
 .../store/writer/v3/DataWriterHolder.java       | 20 +----
 11 files changed, 210 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
index 060b55c..0ce73f0 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
@@ -22,24 +22,19 @@ package org.apache.carbondata.core.constants;
 public interface CarbonV3DataFormatConstants {
 
   /**
-   * number of page per blocklet column
+   * each blocklet group size in mb
    */
-  String NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN = "carbon.number.of.page.in.blocklet.column";
+  String BLOCKLET_SIZE_IN_MB = "carbon.blockletgroup.size.in.mb";
 
   /**
-   * number of page per blocklet column default value
+   * blocklet group size default value
    */
-  String NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_DEFAULT_VALUE = "10";
+  String BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE = "64";
 
   /**
-   * number of page per blocklet column max value
+   * blocklet group size min value
    */
-  short NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_MAX = 20;
-
-  /**
-   * number of page per blocklet column min value
-   */
-  short NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_MIN = 1;
+  short BLOCKLET_SIZE_IN_MB_MIN = 16;
 
   /**
    * number of column to be read in one IO in query

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
index 9a123d1..2e52770 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
@@ -80,6 +80,8 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
             builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex),
             builderInfos.getFooterList().get(0).getBlockInfo().getTableBlockInfo().getFilePath());
     this.nodeNumber = nodeNumber;
+    this.numberOfPages =
+        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex).getNumberOfPages();
   }
 
   /**
@@ -101,8 +103,8 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * @param blockIndex block index to be read
    * @return dimension data chunk
    */
-  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader,
-      int blockIndex) throws IOException {
+  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndex)
+      throws IOException {
     return dimensionChunksReader.readRawDimensionChunk(fileReader, blockIndex);
   }
 
@@ -133,8 +135,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
   /**
    * @return the number of pages in blocklet
    */
-  @Override
-  public int numberOfPages() {
+  @Override public int numberOfPages() {
     return numberOfPages;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
index 9347977..abffea4 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
@@ -126,8 +126,7 @@ public class CarbonMetadataUtil {
   public static FileFooter3 convertFileFooterVersion3(List<BlockletInfo3> infoList,
       List<BlockletIndex> blockletIndexs, int[] cardinalities, int numberOfColumns,
       SegmentProperties segmentProperties) throws IOException {
-    FileFooter3 footer =
-        getFileFooter3(infoList, blockletIndexs, cardinalities, numberOfColumns);
+    FileFooter3 footer = getFileFooter3(infoList, blockletIndexs, cardinalities, numberOfColumns);
     for (BlockletInfo3 info : infoList) {
       footer.addToBlocklet_info_list3(info);
     }
@@ -902,4 +901,82 @@ public class CarbonMetadataUtil {
     fileHeader.setVersion(version.number());
     return fileHeader;
   }
+
+  /**
+   * Below method will be used to get the data chunk2 serialize object list
+   *
+   * @param nodeHolder        node holder
+   * @param columnSchenma     table columns
+   * @param segmentProperties segment properties
+   * @param isDimensionColumn to get the list of dimension column or measure column
+   * @return list of data chunk2
+   * @throws IOException
+   */
+  public static List<byte[]> getDataChunk2(NodeHolder nodeHolder, List<ColumnSchema>
columnSchenma,
+      SegmentProperties segmentProperties, boolean isDimensionColumn) throws IOException
{
+    List<byte[]> dataChunkBuffer = new ArrayList<>();
+    if (isDimensionColumn) {
+      for (int i = 0; i < nodeHolder.getKeyArray().length; i++) {
+        DataChunk2 dataChunk = new DataChunk2();
+        dataChunk.min_max = new BlockletMinMaxIndex();
+        dataChunk.setChunk_meta(getChunkCompressionMeta());
+        dataChunk.setNumberOfRowsInpage(nodeHolder.getEntryCount());
+        List<Encoding> encodings = new ArrayList<Encoding>();
+        dataChunk.setData_page_length(nodeHolder.getKeyLengths()[i]);
+        if (containsEncoding(i, Encoding.DICTIONARY, columnSchenma, segmentProperties)) {
+          encodings.add(Encoding.DICTIONARY);
+        }
+        if (containsEncoding(i, Encoding.DIRECT_DICTIONARY, columnSchenma, segmentProperties))
{
+          encodings.add(Encoding.DIRECT_DICTIONARY);
+        }
+        dataChunk.setRowMajor(nodeHolder.getColGrpBlocks()[i]);
+        if (nodeHolder.getAggBlocks()[i]) {
+          dataChunk.setRle_page_length(nodeHolder.getDataIndexMapLength()[i]);
+          encodings.add(Encoding.RLE);
+        }
+        dataChunk.setSort_state(
+            nodeHolder.getIsSortedKeyBlock()[i] ? SortState.SORT_EXPLICIT : SortState.SORT_NATIVE);
+        if (!nodeHolder.getIsSortedKeyBlock()[i]) {
+          dataChunk.setRowid_page_length(nodeHolder.getKeyBlockIndexLength()[i]);
+          encodings.add(Encoding.INVERTED_INDEX);
+        }
+        dataChunk.min_max.addToMax_values(ByteBuffer.wrap(nodeHolder.getColumnMaxData()[i]));
+        dataChunk.min_max.addToMin_values(ByteBuffer.wrap(nodeHolder.getColumnMinData()[i]));
+        dataChunk.setEncoders(encodings);
+        dataChunkBuffer.add(CarbonUtil.getByteArray(dataChunk));
+      }
+    } else {
+      for (int i = 0; i < nodeHolder.getDataArray().length; i++) {
+        DataChunk2 dataChunk = new DataChunk2();
+        dataChunk.min_max = new BlockletMinMaxIndex();
+        dataChunk.setChunk_meta(getChunkCompressionMeta());
+        dataChunk.setNumberOfRowsInpage(nodeHolder.getEntryCount());
+        dataChunk.setData_page_length(nodeHolder.getDataArray()[i].length);
+        List<Encoding> encodings = new ArrayList<Encoding>();
+        // TODO : Right now the encodings are happening at runtime. change as
+        // per this encoders.
+        dataChunk.setEncoders(encodings);
+        dataChunk.setRowMajor(false);
+        // TODO : Right now the encodings are happening at runtime. change as
+        // per this encoders.
+        encodings.add(Encoding.DELTA);
+        dataChunk.setEncoders(encodings);
+        // TODO writing dummy presence meta need to set actual presence
+        // meta
+        PresenceMeta presenceMeta = new PresenceMeta();
+        presenceMeta.setPresent_bit_streamIsSet(true);
+        presenceMeta.setPresent_bit_stream(CompressorFactory.getInstance().getCompressor()
+            .compressByte(nodeHolder.getMeasureNullValueIndex()[i].toByteArray()));
+        dataChunk.setPresence(presenceMeta);
+        List<ByteBuffer> encoderMetaList = new ArrayList<ByteBuffer>();
+        encoderMetaList.add(ByteBuffer.wrap(serializeEncodeMetaUsingByteBuffer(
+            createValueEncoderMeta(nodeHolder.getCompressionModel(), i))));
+        dataChunk.setEncoder_meta(encoderMetaList);
+        dataChunk.min_max.addToMax_values(ByteBuffer.wrap(nodeHolder.getMeasureColumnMaxData()[i]));
+        dataChunk.min_max.addToMin_values(ByteBuffer.wrap(nodeHolder.getMeasureColumnMinData()[i]));
+        dataChunkBuffer.add(CarbonUtil.getByteArray(dataChunk));
+      }
+    }
+    return dataChunkBuffer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 39c36ea..8e0e5af 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -86,7 +86,7 @@ public final class CarbonProperties {
     validateCarbonDataFileVersion();
     validateExecutorStartUpTime();
     validatePrefetchBufferSize();
-    validateNumberOfPagesPerBlocklet();
+    validateBlockletGroupSizeInMB();
     validateNumberOfColumnPerIORead();
     validateNumberOfRowsPerBlockletColumnPage();
   }
@@ -114,31 +114,29 @@ public final class CarbonProperties {
   /**
    * This method validates the number of pages per blocklet column
    */
-  private void validateNumberOfPagesPerBlocklet() {
+  private void validateBlockletGroupSizeInMB() {
     String numberOfPagePerBlockletColumnString = carbonProperties
-        .getProperty(CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN,
-            CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_DEFAULT_VALUE);
+        .getProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+            CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
     try {
       short numberOfPagePerBlockletColumn = Short.parseShort(numberOfPagePerBlockletColumnString);
-      if (numberOfPagePerBlockletColumn
-          < CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_MIN
-          || numberOfPagePerBlockletColumn
-          > CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_MAX) {
-        LOGGER.info(
-            "The Number Of pages per blocklet column value \"" + numberOfPagePerBlockletColumnString
-                + "\" is invalid. Using the default value \""
-                + CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_DEFAULT_VALUE);
-        carbonProperties.setProperty(CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN,
-            CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_DEFAULT_VALUE);
+      if (numberOfPagePerBlockletColumn < CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_MIN)
{
+        LOGGER.info("Blocklet Size Configured value \"" + numberOfPagePerBlockletColumnString
+            + "\" is invalid. Using the default value \""
+            + CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
+        carbonProperties.setProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+            CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
       }
     } catch (NumberFormatException e) {
-      LOGGER.info(
-          "The Number Of pages per blocklet column value \"" + numberOfPagePerBlockletColumnString
-              + "\" is invalid. Using the default value \""
-              + CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_DEFAULT_VALUE);
-      carbonProperties.setProperty(CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN,
-          CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_DEFAULT_VALUE);
+      LOGGER.info("Blocklet Size Configured value \"" + numberOfPagePerBlockletColumnString
+          + "\" is invalid. Using the default value \""
+          + CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
+      carbonProperties.setProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+          CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
     }
+    LOGGER.info("Blocklet Size Configured value is \"" + carbonProperties
+        .getProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+            CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE));
   }
 
   /**
@@ -393,6 +391,9 @@ public final class CarbonProperties {
             CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION);
       }
     }
+    LOGGER.info("Carbon Current data file version: " + carbonProperties
+        .setProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION,
+            CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/core/src/main/java/org/apache/carbondata/core/util/NodeHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/NodeHolder.java b/core/src/main/java/org/apache/carbondata/core/util/NodeHolder.java
index d46b806..f80a08d 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NodeHolder.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NodeHolder.java
@@ -151,6 +151,11 @@ public class NodeHolder {
   private int totalMeasureArrayLength;
 
   /**
+   * data size this node holder is holding
+   */
+  private int holderSize;
+
+  /**
    * @return the keyArray
    */
   public byte[][] getKeyArray() {
@@ -427,4 +432,12 @@ public class NodeHolder {
   public void setMeasureColumnMinData(byte[][] measureColumnMinData) {
     this.measureColumnMinData = measureColumnMinData;
   }
+
+  public int getHolderSize() {
+    return holderSize;
+  }
+
+  public void setHolderSize(int holderSize) {
+    this.holderSize = holderSize;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index ed461f9..abebf59 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -602,8 +602,8 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler
{
             .clone();
     NodeHolder nodeHolder =
         getNodeHolderObject(writableMeasureDataArray, byteArrayValues, dataRows.size(), startKey,
-            endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey);
-    nodeHolder.setMeasureNullValueIndex(nullValueIndexBitSet);
+            endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey,
+            nullValueIndexBitSet);
     LOGGER.info("Number Of records processed: " + dataRows.size());
     return nodeHolder;
   }
@@ -748,8 +748,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler
{
     NodeHolder nodeHolder =
         getNodeHolderObjectWithOutKettle(writableMeasureDataArray, byteArrayValues, dataRows.size(),
             startKey, endKey, compressionModel, noDictionaryValueHolder, noDictStartKey,
-            noDictEndKey);
-    nodeHolder.setMeasureNullValueIndex(nullValueIndexBitSet);
+            noDictEndKey, nullValueIndexBitSet);
     LOGGER.info("Number Of records processed: " + dataRows.size());
     return nodeHolder;
   }
@@ -758,7 +757,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler
{
   private NodeHolder getNodeHolderObject(byte[][] dataHolderLocal, byte[][] byteArrayValues,
       int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal,
       WriterCompressModel compressionModel, byte[][] noDictionaryData, byte[] noDictionaryStartKey,
-      byte[] noDictionaryEndKey) throws CarbonDataWriterException {
+      byte[] noDictionaryEndKey, BitSet[] nullValueIndexBitSet) throws CarbonDataWriterException
{
     byte[][][] noDictionaryColumnsData = null;
     List<ArrayList<byte[]>> colsAndValues = new ArrayList<ArrayList<byte[]>>();
     int complexColCount = getComplexColsCount();
@@ -878,13 +877,15 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler
{
     }
     return this.dataWriter
         .buildDataNodeHolder(blockStorage, dataHolderLocal, entryCountLocal, startkeyLocal,
-            endKeyLocal, compressionModel, noDictionaryStartKey, noDictionaryEndKey);
+            endKeyLocal, compressionModel, noDictionaryStartKey, noDictionaryEndKey,
+            nullValueIndexBitSet);
   }
 
   private NodeHolder getNodeHolderObjectWithOutKettle(byte[][] dataHolderLocal,
       byte[][] byteArrayValues, int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal,
       WriterCompressModel compressionModel, byte[][][] noDictionaryData,
-      byte[][] noDictionaryStartKey, byte[][] noDictionaryEndKey) throws CarbonDataWriterException
{
+      byte[][] noDictionaryStartKey, byte[][] noDictionaryEndKey, BitSet[] nullValueIndexBitSet)
+      throws CarbonDataWriterException {
     byte[][][] noDictionaryColumnsData = null;
     List<ArrayList<byte[]>> colsAndValues = new ArrayList<ArrayList<byte[]>>();
     int complexColCount = getComplexColsCount();
@@ -1012,7 +1013,8 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler
{
     }
     return this.dataWriter
         .buildDataNodeHolder(blockStorage, dataHolderLocal, entryCountLocal, startkeyLocal,
-            endKeyLocal, compressionModel, composedNonDictStartKey, composedNonDictEndKey);
+            endKeyLocal, compressionModel, composedNonDictStartKey, composedNonDictEndKey,
+            nullValueIndexBitSet);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index 289c156..ce5aa03 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -113,7 +113,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   /**
    * data file size;
    */
-  private long fileSizeInBytes;
+  protected int fileSizeInBytes;
   /**
    * file count will be used to give sequence number to the data file
    */
@@ -163,7 +163,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     CarbonProperties propInstance = CarbonProperties.getInstance();
     // if blocksize=2048, then 2048*1024*1024 will beyond the range of Int
     this.fileSizeInBytes =
-        (long) dataWriterVo.getTableBlocksize() * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR
+        dataWriterVo.getTableBlocksize() * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR
             * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR;
     this.spaceReservedForBlockMetaSize = Integer.parseInt(propInstance
         .getProperty(CarbonCommonConstants.CARBON_BLOCK_META_RESERVED_SPACE,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
index 94d2727..227f92b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
@@ -17,6 +17,8 @@
 
 package org.apache.carbondata.processing.store.writer;
 
+import java.util.BitSet;
+
 import org.apache.carbondata.core.datastore.columnar.IndexStorage;
 import org.apache.carbondata.core.datastore.compression.WriterCompressModel;
 import org.apache.carbondata.core.util.NodeHolder;
@@ -40,7 +42,8 @@ public interface CarbonFactDataWriter<T> {
 
   NodeHolder buildDataNodeHolder(IndexStorage<T>[] keyStorageArray, byte[][] dataArray,
       int entryCount, byte[] startKey, byte[] endKey, WriterCompressModel compressionModel,
-      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey) throws CarbonDataWriterException;
+      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey, BitSet[] nullValueIndexBitSet)
+      throws CarbonDataWriterException;
 
   /**
    * If node holder flag is enabled the object will be added to list

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/processing/src/main/java/org/apache/carbondata/processing/store/writer/v1/CarbonFactDataWriterImplV1.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v1/CarbonFactDataWriterImplV1.java
b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v1/CarbonFactDataWriterImplV1.java
index 3218a51..a3dd242 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v1/CarbonFactDataWriterImplV1.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v1/CarbonFactDataWriterImplV1.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.processing.store.writer.v1;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.util.BitSet;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -38,8 +39,8 @@ import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterE
 
 public class CarbonFactDataWriterImplV1 extends AbstractFactDataWriter<int[]> {
 
-  private static final LogService LOGGER = LogServiceFactory
-      .getLogService(CarbonFactDataWriterImplV1.class.getName());
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonFactDataWriterImplV1.class.getName());
 
   public CarbonFactDataWriterImplV1(CarbonDataWriterVo dataWriterVo) {
     super(dataWriterVo);
@@ -48,7 +49,8 @@ public class CarbonFactDataWriterImplV1 extends AbstractFactDataWriter<int[]>
{
   @Override
   public NodeHolder buildDataNodeHolder(IndexStorage<int[]>[] keyStorageArray, byte[][]
dataArray,
       int entryCount, byte[] startKey, byte[] endKey, WriterCompressModel compressionModel,
-      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey) throws CarbonDataWriterException
{
+      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey, BitSet[] nullValueIndexBitSet)
+      throws CarbonDataWriterException {
     // if there are no NO-Dictionary column present in the table then
     // set the empty byte array
     if (null == noDictionaryEndKey) {
@@ -149,6 +151,7 @@ public class CarbonFactDataWriterImplV1 extends AbstractFactDataWriter<int[]>
{
     NodeHolder holder = new NodeHolder();
     holder.setDataArray(dataArray);
     holder.setKeyArray(keyBlockData);
+    holder.setMeasureNullValueIndex(nullValueIndexBitSet);
     // end key format will be <length of dictionary key><length of no
     // dictionary key><DictionaryKey><No Dictionary key>
     byte[] updatedNoDictionaryEndKey = updateNoDictionaryStartAndEndKey(noDictionaryEndKey);
@@ -362,8 +365,8 @@ public class CarbonFactDataWriterImplV1 extends AbstractFactDataWriter<int[]>
{
   /**
    * This method will write metadata at the end of file file format in thrift format
    */
-  protected void writeBlockletInfoToFile(FileChannel channel,
-      String filePath) throws CarbonDataWriterException {
+  protected void writeBlockletInfoToFile(FileChannel channel, String filePath)
+      throws CarbonDataWriterException {
     try {
       long currentPosition = channel.size();
       CarbonFooterWriter writer = new CarbonFooterWriter(filePath);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
index bfb3683..df75366 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
@@ -20,8 +20,11 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.List;
 
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datastore.columnar.IndexStorage;
@@ -50,21 +53,27 @@ import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterE
  */
 public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter<short[]> {
 
-  /**
-   * number of pages in one column
-   */
-  private int numberOfChunksInBlocklet;
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonFactDataWriterImplV3.class.getName());
 
   /**
    * persist the page data to be written in the file
    */
   private DataWriterHolder dataWriterHolder;
 
+  private int blockletSize;
+
   public CarbonFactDataWriterImplV3(CarbonDataWriterVo dataWriterVo) {
     super(dataWriterVo);
-    this.numberOfChunksInBlocklet = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN,
-            CarbonV3DataFormatConstants.NUMBER_OF_PAGE_IN_BLOCKLET_COLUMN_DEFAULT_VALUE));
+    blockletSize = Integer.parseInt(CarbonProperties.getInstance()
+        .getProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+            CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE))
+        * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR
+        * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR;
+    if (blockletSize > fileSizeInBytes) {
+      blockletSize = fileSizeInBytes;
+      LOGGER.info("Blocklet size configure for table is: " + blockletSize);
+    }
     dataWriterHolder = new DataWriterHolder();
   }
 
@@ -75,8 +84,8 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter<short[]>
    */
   @Override public NodeHolder buildDataNodeHolder(IndexStorage<short[]>[] keyStorageArray,
       byte[][] dataArray, int entryCount, byte[] startKey, byte[] endKey,
-      WriterCompressModel compressionModel, byte[] noDictionaryStartKey, byte[] noDictionaryEndKey)
-      throws CarbonDataWriterException {
+      WriterCompressModel compressionModel, byte[] noDictionaryStartKey, byte[] noDictionaryEndKey,
+      BitSet[] nullValueIndexBitSet) throws CarbonDataWriterException {
     // if there are no NO-Dictionary column present in the table then
     // set the empty byte array
     if (null == noDictionaryEndKey) {
@@ -175,6 +184,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter<short[]>
     NodeHolder holder = new NodeHolder();
     holder.setDataArray(dataArray);
     holder.setKeyArray(keyBlockData);
+    holder.setMeasureNullValueIndex(nullValueIndexBitSet);
     // end key format will be <length of dictionary key><length of no
     // dictionary key><DictionaryKey><No Dictionary key>
     byte[] updatedNoDictionaryEndKey = updateNoDictionaryStartAndEndKey(noDictionaryEndKey);
@@ -218,9 +228,47 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter<short[]>
     holder.setColumnMinData(dimensionMinValue);
     holder.setAggBlocks(dataWriterVo.getAggBlocks());
     holder.setColGrpBlocks(colGrpBlock);
+    List<byte[]> dimensionDataChunk2 = null;
+    List<byte[]> measureDataChunk2 = null;
+    try {
+      dimensionDataChunk2 = CarbonMetadataUtil
+          .getDataChunk2(holder, thriftColumnSchemaList, dataWriterVo.getSegmentProperties(),
true);
+      measureDataChunk2 = CarbonMetadataUtil
+          .getDataChunk2(holder, thriftColumnSchemaList, dataWriterVo.getSegmentProperties(),
+              false);
+
+    } catch (IOException e) {
+      throw new CarbonDataWriterException(e.getMessage());
+    }
+    holder.setHolderSize(calculateSize(holder, dimensionDataChunk2, measureDataChunk2));
     return holder;
   }
 
+  private int calculateSize(NodeHolder holder, List<byte[]> dimensionDataChunk2,
+      List<byte[]> measureDataChunk2) {
+    int size = 0;
+    // add row id index length
+    for (int i = 0; i < holder.getKeyBlockIndexLength().length; i++) {
+      if (!holder.getIsSortedKeyBlock()[i]) {
+        size += holder.getKeyBlockIndexLength()[i];
+      }
+    }
+    // add rle index length
+    for (int i = 0; i < holder.getDataIndexMapLength().length; i++) {
+      if (holder.getAggBlocks()[i]) {
+        size += holder.getDataIndexMapLength()[i];
+      }
+    }
+    for (int i = 0; i < dimensionDataChunk2.size(); i++) {
+      size += dimensionDataChunk2.get(i).length;
+    }
+    for (int i = 0; i < measureDataChunk2.size(); i++) {
+      size += measureDataChunk2.get(i).length;
+    }
+    size += holder.getTotalDimensionArrayLength() + holder.getTotalMeasureArrayLength();
+    return size;
+  }
+
   /**
    * Below method will be used to convert short array to byte array
    *
@@ -266,7 +314,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter<short[]>
   @Override public void writeBlockletData(NodeHolder holder) throws CarbonDataWriterException
{
     // check the number of pages present in data holder, if pages is exceeding threshold
     // it will write the pages to file
-    if (dataWriterHolder.getNumberOfPagesAdded() == numberOfChunksInBlocklet) {
+    if (dataWriterHolder.getSize() + holder.getHolderSize() >= blockletSize) {
       writeDataToFile(fileChannel);
     }
     dataWriterHolder.addNodeHolder(holder);
@@ -419,7 +467,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter<short[]>
         .getBlockletIndex(nodeHolderList, dataWriterVo.getSegmentProperties().getMeasures()));
     BlockletInfo3 blockletInfo3 =
         new BlockletInfo3(numberOfRows, currentDataChunksOffset, currentDataChunksLength,
-            dimensionOffset, measureOffset, dataWriterHolder.getNumberOfPagesAdded());
+            dimensionOffset, measureOffset, dataWriterHolder.getNodeHolder().size());
     blockletMetadata.add(blockletInfo3);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebf13dc7/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/DataWriterHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/DataWriterHolder.java
b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/DataWriterHolder.java
index 0827bd0..4368b2b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/DataWriterHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/DataWriterHolder.java
@@ -36,26 +36,12 @@ public class DataWriterHolder {
 
   public void addNodeHolder(NodeHolder holder) {
     this.nodeHolder.add(holder);
-
-    int size = 0;
-    // add row id index length
-    for (int i = 0; i < holder.getKeyBlockIndexLength().length; i++) {
-      if (!holder.getIsSortedKeyBlock()[i]) {
-        size += holder.getKeyBlockIndexLength()[i];
-      }
-    }
-    // add rle index length
-    for (int i = 0; i < holder.getDataIndexMapLength().length; i++) {
-      if (holder.getAggBlocks()[i]) {
-        size += holder.getDataIndexMapLength()[i];
-      }
-    }
-    currentSize +=
-        holder.getTotalDimensionArrayLength() + holder.getTotalMeasureArrayLength() + size;
+    currentSize += holder.getHolderSize();
   }
 
   public long getSize() {
-    return currentSize;
+    // increasing it by 15 percent for data chunk 3 of each column each page
+    return currentSize + ((currentSize * 15) / 100);
   }
 
   public int getNumberOfPagesAdded() {


Mime
View raw message