You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2016/12/01 09:53:54 UTC

[1/5] incubator-carbondata git commit: Improve first time query performance

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 9ad98f432 -> 7213ac057


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/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 16bd771..279bb63 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
@@ -26,6 +26,7 @@ import java.io.FileFilter;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -39,7 +40,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletBTreeIndex;
 import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
@@ -52,22 +52,21 @@ import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSche
 import org.apache.carbondata.core.carbon.path.CarbonStorePath;
 import org.apache.carbondata.core.carbon.path.CarbonTablePath;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastorage.store.columnar.IndexStorage;
+import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression.SnappyByteCompression;
 import org.apache.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
+import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.metadata.BlockletInfoColumnar;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonMergerUtil;
 import org.apache.carbondata.core.util.CarbonMetadataUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.writer.CarbonFooterWriter;
 import org.apache.carbondata.core.writer.CarbonIndexFileWriter;
 import org.apache.carbondata.format.BlockIndex;
-import org.apache.carbondata.format.FileFooter;
 import org.apache.carbondata.format.IndexHeader;
 import org.apache.carbondata.processing.mdkeygen.file.FileData;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
-import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
 import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException;
 
 import org.apache.commons.lang3.ArrayUtils;
@@ -87,10 +86,6 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    */
   private static final int HDFS_CHECKSUM_LENGTH = 512;
   /**
-   * measure count
-   */
-  protected int measureCount;
-  /**
    * file channel
    */
   protected FileChannel fileChannel;
@@ -98,16 +93,8 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    * this will be used for holding blocklet metadata
    */
   protected List<BlockletInfoColumnar> blockletInfoList;
-  /**
-   * keyBlockSize
-   */
-  protected int[] keyBlockSize;
   protected boolean[] isNoDictionary;
   /**
-   * mdkeySize
-   */
-  protected int mdkeySize;
-  /**
    * file name
    */
   protected String fileName;
@@ -115,15 +102,14 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    * Local cardinality for the segment
    */
   protected int[] localCardinality;
-  protected String databaseName;
   /**
    * thrift column schema
    */
   protected List<org.apache.carbondata.format.ColumnSchema> thriftColumnSchemaList;
-  /**
-   * tabel name
-   */
-  private String tableName;
+  protected NumberCompressor numberCompressor;
+  protected CarbonDataWriterVo dataWriterVo;
+  protected List<List<Long>> dataChunksOffsets;
+  protected List<List<Short>> dataChunksLength;
   /**
    * data file size;
    */
@@ -133,32 +119,15 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    */
   private int fileCount;
   /**
-   * File manager
-   */
-  private IFileManagerComposite fileManager;
-  /**
-   * Store Location
-   */
-  private String storeLocation;
-  /**
    * executorService
    */
   private ExecutorService executorService;
-
   /**
    * executorService
    */
   private List<Future<Void>> executorServiceSubmitList;
-  /**
-   * data file attributes which will used for file construction
-   */
-  private CarbonDataFileAttributes carbonDataFileAttributes;
   private CarbonTablePath carbonTablePath;
   /**
-   * data directory location in carbon store path
-   */
-  private String carbonDataDirectoryPath;
-  /**
    * data block size for one carbon data file
    */
   private long dataBlockSize;
@@ -171,68 +140,55 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    */
   private int spaceReservedForBlockMetaSize;
   private FileOutputStream fileOutputStream;
-
-  private SegmentProperties segmentProperties;
-
   private List<BlockIndexInfo> blockIndexInfoList;
 
-  public AbstractFactDataWriter(String storeLocation, int measureCount, int mdKeyLength,
-      String databaseName, String tableName, IFileManagerComposite fileManager, int[] keyBlockSize,
-      CarbonDataFileAttributes carbonDataFileAttributes, List<ColumnSchema> columnSchema,
-      String carbonDataDirectoryPath, int[] colCardinality, SegmentProperties segmentProperties,
-      int blocksize) {
-
-    // measure count
-    this.measureCount = measureCount;
-    // table name
-    this.tableName = tableName;
-    this.databaseName = databaseName;
-
-    this.databaseName = databaseName;
-
-    this.storeLocation = storeLocation;
-    this.segmentProperties = segmentProperties;
+  public AbstractFactDataWriter(CarbonDataWriterVo dataWriterVo) {
+    this.dataWriterVo = dataWriterVo;
     this.blockletInfoList =
         new ArrayList<BlockletInfoColumnar>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
     blockIndexInfoList = new ArrayList<>();
     // get max file size;
     CarbonProperties propInstance = CarbonProperties.getInstance();
     // if blocksize=2048, then 2048*1024*1024 will beyond the range of Int
-    this.fileSizeInBytes = (long) blocksize * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR
-        * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR;
+    this.fileSizeInBytes =
+        (long) 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,
             CarbonCommonConstants.CARBON_BLOCK_META_RESERVED_SPACE_DEFAULT));
     this.dataBlockSize = fileSizeInBytes - (fileSizeInBytes * spaceReservedForBlockMetaSize) / 100;
     LOGGER.info("Total file size: " + fileSizeInBytes + " and dataBlock Size: " + dataBlockSize);
-    this.fileManager = fileManager;
-    this.carbonDataDirectoryPath = carbonDataDirectoryPath;
-    this.keyBlockSize = keyBlockSize;
-    this.mdkeySize = mdKeyLength;
+
     this.executorService = Executors.newFixedThreadPool(1);
     executorServiceSubmitList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     // in case of compaction we will pass the cardinality.
-    this.localCardinality = colCardinality;
-    this.carbonDataFileAttributes = carbonDataFileAttributes;
-    CarbonTable carbonTable = CarbonMetadata.getInstance()
-        .getCarbonTable(databaseName + CarbonCommonConstants.UNDERSCORE + tableName);
-    carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(storeLocation, carbonTable.getCarbonTableIdentifier());
+    this.localCardinality = dataWriterVo.getColCardinality();
+    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(
+        dataWriterVo.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + dataWriterVo
+            .getTableName());
+    carbonTablePath = CarbonStorePath.getCarbonTablePath(dataWriterVo.getStoreLocation(),
+        carbonTable.getCarbonTableIdentifier());
     //TODO: We should delete the levelmetadata file after reading here.
     // so only data loading flow will need to read from cardinality file.
     if (null == this.localCardinality) {
-      this.localCardinality =
-          CarbonMergerUtil.getCardinalityFromLevelMetadata(storeLocation, tableName);
+      this.localCardinality = CarbonMergerUtil
+          .getCardinalityFromLevelMetadata(dataWriterVo.getStoreLocation(),
+              dataWriterVo.getTableName());
       List<Integer> cardinalityList = new ArrayList<Integer>();
-      thriftColumnSchemaList =
-          getColumnSchemaListAndCardinality(cardinalityList, localCardinality, columnSchema);
+      thriftColumnSchemaList = getColumnSchemaListAndCardinality(cardinalityList, localCardinality,
+          dataWriterVo.getWrapperColumnSchemaList());
       localCardinality =
           ArrayUtils.toPrimitive(cardinalityList.toArray(new Integer[cardinalityList.size()]));
     } else { // for compaction case
       List<Integer> cardinalityList = new ArrayList<Integer>();
-      thriftColumnSchemaList =
-          getColumnSchemaListAndCardinality(cardinalityList, localCardinality, columnSchema);
+      thriftColumnSchemaList = getColumnSchemaListAndCardinality(cardinalityList, localCardinality,
+          dataWriterVo.getWrapperColumnSchemaList());
     }
+    this.numberCompressor = new NumberCompressor(Integer.parseInt(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
+            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL)));
+    this.dataChunksOffsets = new ArrayList<>();
+    this.dataChunksLength = new ArrayList<>();
   }
 
   /**
@@ -258,8 +214,8 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     String readableFileSize = ByteUtil.convertByteToReadable(fileSize);
     String readableMaxSize = ByteUtil.convertByteToReadable(maxSize);
     LOGGER.info("The configured block size is " + readableBlockSize +
-            ", the actual carbon file size is " + readableFileSize +
-            ", choose the max value " + readableMaxSize + " as the block size on HDFS");
+        ", the actual carbon file size is " + readableFileSize +
+        ", choose the max value " + readableMaxSize + " as the block size on HDFS");
     return maxSize;
   }
 
@@ -291,6 +247,8 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
       this.currentFileSize = 0;
       blockletInfoList =
           new ArrayList<BlockletInfoColumnar>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      this.dataChunksOffsets = new ArrayList<>();
+      this.dataChunksLength = new ArrayList<>();
       CarbonUtil.closeStreams(this.fileOutputStream, this.fileChannel);
       // rename carbon data file from in progress status to actual
       renameCarbonDataFile();
@@ -312,12 +270,12 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     // increment the file sequence counter
     initFileCount();
     String carbonDataFileName = carbonTablePath
-        .getCarbonDataFileName(fileCount, carbonDataFileAttributes.getTaskId(),
-            carbonDataFileAttributes.getFactTimeStamp());
+        .getCarbonDataFileName(fileCount, dataWriterVo.getCarbonDataFileAttributes().getTaskId(),
+            dataWriterVo.getCarbonDataFileAttributes().getFactTimeStamp());
     String actualFileNameVal = carbonDataFileName + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-    FileData fileData = new FileData(actualFileNameVal, this.storeLocation);
-    fileManager.add(fileData);
-    this.fileName = storeLocation + File.separator + carbonDataFileName
+    FileData fileData = new FileData(actualFileNameVal, dataWriterVo.getStoreLocation());
+    dataWriterVo.getFileManager().add(fileData);
+    this.fileName = dataWriterVo.getStoreLocation() + File.separator + carbonDataFileName
         + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
     this.fileCount++;
     try {
@@ -332,11 +290,10 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
 
   private int initFileCount() {
     int fileInitialCount = 0;
-    File[] dataFiles = new File(storeLocation).listFiles(new FileFilter() {
-
+    File[] dataFiles = new File(dataWriterVo.getStoreLocation()).listFiles(new FileFilter() {
       @Override public boolean accept(File pathVal) {
-        if (!pathVal.isDirectory() && pathVal.getName().startsWith(tableName) && pathVal.getName()
-            .contains(CarbonCommonConstants.FACT_FILE_EXT)) {
+        if (!pathVal.isDirectory() && pathVal.getName().startsWith(dataWriterVo.getTableName())
+            && pathVal.getName().contains(CarbonCommonConstants.FACT_FILE_EXT)) {
           return true;
         }
         return false;
@@ -359,20 +316,8 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   /**
    * This method will write metadata at the end of file file format in thrift format
    */
-  protected void writeBlockletInfoToFile(List<BlockletInfoColumnar> infoList, FileChannel channel,
-      String filePath) throws CarbonDataWriterException {
-    try {
-      long currentPosition = channel.size();
-      CarbonFooterWriter writer = new CarbonFooterWriter(filePath);
-      FileFooter convertFileMeta = CarbonMetadataUtil
-          .convertFileFooter(infoList, localCardinality.length, localCardinality,
-              thriftColumnSchemaList, segmentProperties);
-      fillBlockIndexInfoDetails(infoList, convertFileMeta.getNum_rows(), filePath, currentPosition);
-      writer.writeFooter(convertFileMeta, currentPosition);
-    } catch (IOException e) {
-      throw new CarbonDataWriterException("Problem while writing the carbon file: ", e);
-    }
-  }
+  protected abstract void writeBlockletInfoToFile(List<BlockletInfoColumnar> infoList,
+      FileChannel channel, String filePath) throws CarbonDataWriterException;
 
   /**
    * Below method will be used to fill the vlock info details
@@ -382,7 +327,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    * @param filePath        file path
    * @param currentPosition current offset
    */
-  private void fillBlockIndexInfoDetails(List<BlockletInfoColumnar> infoList, long numberOfRows,
+  protected void fillBlockIndexInfoDetails(List<BlockletInfoColumnar> infoList, long numberOfRows,
       String filePath, long currentPosition) {
 
     // as min-max will change for each blocklet and second blocklet min-max can be lesser than
@@ -448,56 +393,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    *
    * @return BlockletInfo - blocklet metadata
    */
-  protected BlockletInfoColumnar getBlockletInfo(NodeHolder nodeHolder, long offset) {
-    // create the info object for leaf entry
-    BlockletInfoColumnar infoObj = new BlockletInfoColumnar();
-    // add total entry count
-    infoObj.setNumberOfKeys(nodeHolder.getEntryCount());
-
-    // add the key array length
-    infoObj.setKeyLengths(nodeHolder.getKeyLengths());
-    //add column min max data
-    infoObj.setColumnMaxData(nodeHolder.getColumnMaxData());
-    infoObj.setColumnMinData(nodeHolder.getColumnMinData());
-    infoObj.setMeasureNullValueIndex(nodeHolder.getMeasureNullValueIndex());
-    long[] keyOffSets = new long[nodeHolder.getKeyLengths().length];
-
-    for (int i = 0; i < keyOffSets.length; i++) {
-      keyOffSets[i] = offset;
-      offset += nodeHolder.getKeyLengths()[i];
-    }
-    // add key offset
-    infoObj.setKeyOffSets(keyOffSets);
-
-    // add measure length
-    infoObj.setMeasureLength(nodeHolder.getMeasureLenght());
-
-    long[] msrOffset = new long[this.measureCount];
-
-    for (int i = 0; i < this.measureCount; i++) {
-      msrOffset[i] = offset;
-      // now increment the offset by adding measure length to get the next
-      // measure offset;
-      offset += nodeHolder.getMeasureLenght()[i];
-    }
-    // add measure offset
-    infoObj.setMeasureOffset(msrOffset);
-    infoObj.setIsSortedKeyColumn(nodeHolder.getIsSortedKeyBlock());
-    infoObj.setKeyBlockIndexLength(nodeHolder.getKeyBlockIndexLength());
-    long[] keyBlockIndexOffsets = new long[nodeHolder.getKeyBlockIndexLength().length];
-    for (int i = 0; i < keyBlockIndexOffsets.length; i++) {
-      keyBlockIndexOffsets[i] = offset;
-      offset += nodeHolder.getKeyBlockIndexLength()[i];
-    }
-    infoObj.setKeyBlockIndexOffSets(keyBlockIndexOffsets);
-    // set startkey
-    infoObj.setStartKey(nodeHolder.getStartKey());
-    // set end key
-    infoObj.setEndKey(nodeHolder.getEndKey());
-    infoObj.setCompressionModel(nodeHolder.getCompressionModel());
-    // return leaf metadata
-    return infoObj;
-  }
+  protected abstract BlockletInfoColumnar getBlockletInfo(NodeHolder nodeHolder, long offset);
 
   /**
    * Method will be used to close the open file channel
@@ -528,9 +424,9 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
         CarbonMetadataUtil.getIndexHeader(localCardinality, thriftColumnSchemaList);
     // get the block index info thrift
     List<BlockIndex> blockIndexThrift = CarbonMetadataUtil.getBlockIndexInfo(blockIndexInfoList);
-    String fileName = storeLocation + File.separator + carbonTablePath
-        .getCarbonIndexFileName(carbonDataFileAttributes.getTaskId(),
-            carbonDataFileAttributes.getFactTimeStamp());
+    String fileName = dataWriterVo.getStoreLocation() + File.separator + carbonTablePath
+        .getCarbonIndexFileName(dataWriterVo.getCarbonDataFileAttributes().getTaskId(),
+            dataWriterVo.getCarbonDataFileAttributes().getFactTimeStamp());
     CarbonIndexFileWriter writer = new CarbonIndexFileWriter();
     // open file
     writer.openThriftWriter(fileName);
@@ -591,11 +487,11 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   private void copyCarbonDataFileToCarbonStorePath(String localFileName)
       throws CarbonDataWriterException {
     long copyStartTime = System.currentTimeMillis();
-    LOGGER.info("Copying " + localFileName + " --> " + carbonDataDirectoryPath);
+    LOGGER.info("Copying " + localFileName + " --> " + dataWriterVo.getCarbonDataDirectoryPath());
     try {
       CarbonFile localCarbonFile =
           FileFactory.getCarbonFile(localFileName, FileFactory.getFileType(localFileName));
-      String carbonFilePath = carbonDataDirectoryPath + localFileName
+      String carbonFilePath = dataWriterVo.getCarbonDataDirectoryPath() + localFileName
           .substring(localFileName.lastIndexOf(File.separator));
       copyLocalFileToCarbonStore(carbonFilePath, localFileName,
           CarbonCommonConstants.BYTEBUFFER_SIZE,
@@ -654,18 +550,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    * @throws CarbonDataWriterException
    * @throws CarbonDataWriterException throws new CarbonDataWriterException if any problem
    */
-  protected void writeDataToFile(NodeHolder nodeHolder) throws CarbonDataWriterException {
-    // write data to file and get its offset
-    long offset = writeDataToFile(nodeHolder, fileChannel);
-    // get the blocklet info for currently added blocklet
-    BlockletInfoColumnar blockletInfo = getBlockletInfo(nodeHolder, offset);
-    // add blocklet info to list
-    blockletInfoList.add(blockletInfo);
-    // calculate the current size of the file
-  }
-
-  protected abstract long writeDataToFile(NodeHolder nodeHolder, FileChannel channel)
-      throws CarbonDataWriterException;
+  public abstract void writeBlockletData(NodeHolder nodeHolder) throws CarbonDataWriterException;
 
   @Override public int getLeafMetadataSize() {
     return blockletInfoList.size();
@@ -675,6 +560,99 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     return this.fileName;
   }
 
+  protected byte[][] fillAndCompressedKeyBlockData(IndexStorage<int[]>[] keyStorageArray,
+      int entryCount) {
+    byte[][] keyBlockData = new byte[keyStorageArray.length][];
+    int destPos = 0;
+    int keyBlockSizePosition = -1;
+    for (int i = 0; i < keyStorageArray.length; i++) {
+      destPos = 0;
+      //handling for high card dims
+      if (!dataWriterVo.getIsComplexType()[i] && !dataWriterVo.getIsDictionaryColumn()[i]) {
+        int totalLength = 0;
+        // calc size of the total bytes in all the colmns.
+        for (int k = 0; k < keyStorageArray[i].getKeyBlock().length; k++) {
+          byte[] colValue = keyStorageArray[i].getKeyBlock()[k];
+          totalLength += colValue.length;
+        }
+        keyBlockData[i] = new byte[totalLength];
+
+        for (int j = 0; j < keyStorageArray[i].getKeyBlock().length; j++) {
+          int length = keyStorageArray[i].getKeyBlock()[j].length;
+          System
+              .arraycopy(keyStorageArray[i].getKeyBlock()[j], 0, keyBlockData[i], destPos, length);
+          destPos += length;
+        }
+      } else {
+        keyBlockSizePosition++;
+        if (dataWriterVo.getAggBlocks()[i]) {
+          keyBlockData[i] = new byte[keyStorageArray[i].getTotalSize()];
+          for (int j = 0; j < keyStorageArray[i].getKeyBlock().length; j++) {
+            System.arraycopy(keyStorageArray[i].getKeyBlock()[j], 0, keyBlockData[i], destPos,
+                keyStorageArray[i].getKeyBlock()[j].length);
+            destPos += keyStorageArray[i].getKeyBlock()[j].length;
+          }
+        } else {
+          if (dataWriterVo.getIsComplexType()[i]) {
+            keyBlockData[i] = new byte[keyStorageArray[i].getKeyBlock().length * dataWriterVo
+                .getKeyBlockSize()[keyBlockSizePosition]];
+          } else {
+            keyBlockData[i] =
+                new byte[entryCount * dataWriterVo.getKeyBlockSize()[keyBlockSizePosition]];
+          }
+          for (int j = 0; j < keyStorageArray[i].getKeyBlock().length; j++) {
+            System.arraycopy(keyStorageArray[i].getKeyBlock()[j], 0, keyBlockData[i], destPos,
+                dataWriterVo.getKeyBlockSize()[keyBlockSizePosition]);
+            destPos += dataWriterVo.getKeyBlockSize()[keyBlockSizePosition];
+          }
+        }
+      }
+      keyBlockData[i] = SnappyByteCompression.INSTANCE.compress(keyBlockData[i]);
+    }
+    return keyBlockData;
+  }
+
+  /**
+   * Below method will be used to update the min or max value
+   * by removing the length from it
+   *
+   * @return min max value without length
+   */
+  protected byte[] updateMinMaxForNoDictionary(byte[] valueWithLength) {
+    ByteBuffer buffer = ByteBuffer.wrap(valueWithLength);
+    byte[] actualValue = new byte[buffer.getShort()];
+    buffer.get(actualValue);
+    return actualValue;
+  }
+
+  /**
+   * Below method will be used to update the no dictionary start and end key
+   *
+   * @param key key to be updated
+   * @return return no dictionary key
+   */
+  protected byte[] updateNoDictionaryStartAndEndKey(byte[] key) {
+    if (key.length == 0) {
+      return key;
+    }
+    // add key to byte buffer remove the length part of the data
+    ByteBuffer buffer = ByteBuffer.wrap(key, 2, key.length - 2);
+    // create a output buffer without length
+    ByteBuffer output = ByteBuffer.allocate(key.length - 2);
+    short numberOfByteToStorLength = 2;
+    // as length part is removed, so each no dictionary value index
+    // needs to be reshuffled by 2 bytes
+    for (int i = 0; i < dataWriterVo.getNoDictionaryCount(); i++) {
+      output.putShort((short) (buffer.getShort() - numberOfByteToStorLength));
+    }
+    // copy the data part
+    while (buffer.hasRemaining()) {
+      output.put(buffer.get());
+    }
+    output.rewind();
+    return output.array();
+  }
+
   /**
    * This method will copy the carbon data file from local store location to
    * carbon store location

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java
new file mode 100644
index 0000000..6e0287d
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.processing.store.writer;
+
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
+import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
+
+/**
+ * Value object for writing the data
+ */
+public class CarbonDataWriterVo {
+
+  private String storeLocation;
+
+  private int measureCount;
+
+  private int mdKeyLength;
+
+  private String tableName;
+
+  private IFileManagerComposite fileManager;
+
+  private int[] keyBlockSize;
+
+  private boolean[] aggBlocks;
+
+  private boolean[] isComplexType;
+
+  private int NoDictionaryCount;
+
+  private CarbonDataFileAttributes carbonDataFileAttributes;
+
+  private String databaseName;
+
+  private List<ColumnSchema> wrapperColumnSchemaList;
+
+  private int numberOfNoDictionaryColumn;
+
+  private boolean[] isDictionaryColumn;
+
+  private String carbonDataDirectoryPath;
+
+  private int[] colCardinality;
+
+  private SegmentProperties segmentProperties;
+
+  private int tableBlocksize;
+
+  /**
+   * @return the storeLocation
+   */
+  public String getStoreLocation() {
+    return storeLocation;
+  }
+
+  /**
+   * @param storeLocation the storeLocation to set
+   */
+  public void setStoreLocation(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  /**
+   * @return the measureCount
+   */
+  public int getMeasureCount() {
+    return measureCount;
+  }
+
+  /**
+   * @param measureCount the measureCount to set
+   */
+  public void setMeasureCount(int measureCount) {
+    this.measureCount = measureCount;
+  }
+
+  /**
+   * @return the mdKeyLength
+   */
+  public int getMdKeyLength() {
+    return mdKeyLength;
+  }
+
+  /**
+   * @param mdKeyLength the mdKeyLength to set
+   */
+  public void setMdKeyLength(int mdKeyLength) {
+    this.mdKeyLength = mdKeyLength;
+  }
+
+  /**
+   * @return the tableName
+   */
+  public String getTableName() {
+    return tableName;
+  }
+
+  /**
+   * @param tableName the tableName to set
+   */
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  /**
+   * @return the fileManager
+   */
+  public IFileManagerComposite getFileManager() {
+    return fileManager;
+  }
+
+  /**
+   * @param fileManager the fileManager to set
+   */
+  public void setFileManager(IFileManagerComposite fileManager) {
+    this.fileManager = fileManager;
+  }
+
+  /**
+   * @return the keyBlockSize
+   */
+  public int[] getKeyBlockSize() {
+    return keyBlockSize;
+  }
+
+  /**
+   * @param keyBlockSize the keyBlockSize to set
+   */
+  public void setKeyBlockSize(int[] keyBlockSize) {
+    this.keyBlockSize = keyBlockSize;
+  }
+
+  /**
+   * @return the aggBlocks
+   */
+  public boolean[] getAggBlocks() {
+    return aggBlocks;
+  }
+
+  /**
+   * @param aggBlocks the aggBlocks to set
+   */
+  public void setAggBlocks(boolean[] aggBlocks) {
+    this.aggBlocks = aggBlocks;
+  }
+
+  /**
+   * @return the isComplexType
+   */
+  public boolean[] getIsComplexType() {
+    return isComplexType;
+  }
+
+  /**
+   * @param isComplexType the isComplexType to set
+   */
+  public void setIsComplexType(boolean[] isComplexType) {
+    this.isComplexType = isComplexType;
+  }
+
+  /**
+   * @return the noDictionaryCount
+   */
+  public int getNoDictionaryCount() {
+    return NoDictionaryCount;
+  }
+
+  /**
+   * @param noDictionaryCount the noDictionaryCount to set
+   */
+  public void setNoDictionaryCount(int noDictionaryCount) {
+    NoDictionaryCount = noDictionaryCount;
+  }
+
+  /**
+   * @return the carbonDataFileAttributes
+   */
+  public CarbonDataFileAttributes getCarbonDataFileAttributes() {
+    return carbonDataFileAttributes;
+  }
+
+  /**
+   * @param carbonDataFileAttributes the carbonDataFileAttributes to set
+   */
+  public void setCarbonDataFileAttributes(CarbonDataFileAttributes carbonDataFileAttributes) {
+    this.carbonDataFileAttributes = carbonDataFileAttributes;
+  }
+
+  /**
+   * @return the databaseName
+   */
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  /**
+   * @param databaseName the databaseName to set
+   */
+  public void setDatabaseName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+
+  /**
+   * @return the wrapperColumnSchemaList
+   */
+  public List<ColumnSchema> getWrapperColumnSchemaList() {
+    return wrapperColumnSchemaList;
+  }
+
+  /**
+   * @param wrapperColumnSchemaList the wrapperColumnSchemaList to set
+   */
+  public void setWrapperColumnSchemaList(List<ColumnSchema> wrapperColumnSchemaList) {
+    this.wrapperColumnSchemaList = wrapperColumnSchemaList;
+  }
+
+  /**
+   * @return the numberOfNoDictionaryColumn
+   */
+  public int getNumberOfNoDictionaryColumn() {
+    return numberOfNoDictionaryColumn;
+  }
+
+  /**
+   * @param numberOfNoDictionaryColumn the numberOfNoDictionaryColumn to set
+   */
+  public void setNumberOfNoDictionaryColumn(int numberOfNoDictionaryColumn) {
+    this.numberOfNoDictionaryColumn = numberOfNoDictionaryColumn;
+  }
+
+  /**
+   * @return the isDictionaryColumn
+   */
+  public boolean[] getIsDictionaryColumn() {
+    return isDictionaryColumn;
+  }
+
+  /**
+   * @param isDictionaryColumn the isDictionaryColumn to set
+   */
+  public void setIsDictionaryColumn(boolean[] isDictionaryColumn) {
+    this.isDictionaryColumn = isDictionaryColumn;
+  }
+
+  /**
+   * @return the carbonDataDirectoryPath
+   */
+  public String getCarbonDataDirectoryPath() {
+    return carbonDataDirectoryPath;
+  }
+
+  /**
+   * @param carbonDataDirectoryPath the carbonDataDirectoryPath to set
+   */
+  public void setCarbonDataDirectoryPath(String carbonDataDirectoryPath) {
+    this.carbonDataDirectoryPath = carbonDataDirectoryPath;
+  }
+
+  /**
+   * @return the colCardinality
+   */
+  public int[] getColCardinality() {
+    return colCardinality;
+  }
+
+  /**
+   * @param colCardinality the colCardinality to set
+   */
+  public void setColCardinality(int[] colCardinality) {
+    this.colCardinality = colCardinality;
+  }
+
+  /**
+   * @return the segmentProperties
+   */
+  public SegmentProperties getSegmentProperties() {
+    return segmentProperties;
+  }
+
+  /**
+   * @param segmentProperties the segmentProperties to set
+   */
+  public void setSegmentProperties(SegmentProperties segmentProperties) {
+    this.segmentProperties = segmentProperties;
+  }
+
+  /**
+   * @return the tableBlocksize
+   */
+  public int getTableBlocksize() {
+    return tableBlocksize;
+  }
+
+  /**
+   * @param tableBlocksize the tableBlocksize to set
+   */
+  public void setTableBlocksize(int tableBlocksize) {
+    this.tableBlocksize = tableBlocksize;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImpl2.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImpl2.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImpl2.java
new file mode 100644
index 0000000..d399280
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImpl2.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.processing.store.writer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+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.metadata.BlockletInfoColumnar;
+import org.apache.carbondata.core.util.CarbonMetadataUtil;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.writer.CarbonFooterWriter;
+import org.apache.carbondata.format.DataChunk2;
+import org.apache.carbondata.format.FileFooter;
+import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException;
+
+/**
+ * Below method will be used to write the data in version 2 format
+ */
+public class CarbonFactDataWriterImpl2 extends CarbonFactDataWriterImplForIntIndexAndAggBlock {
+
+  /**
+   * logger
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonFactDataWriterImpl2.class.getName());
+
+  /**
+   * Constructor create instance of this class
+   *
+   * @param dataWriterVo
+   */
+  public CarbonFactDataWriterImpl2(CarbonDataWriterVo dataWriterVo) {
+    super(dataWriterVo);
+  }
+
+  /**
+   * Below method will be used to write the data to carbon data file
+   *
+   * @param holder
+   * @throws CarbonDataWriterException any problem in writing operation
+   */
+  @Override public void writeBlockletData(NodeHolder holder) throws CarbonDataWriterException {
+    // size to calculate the size of the blocklet
+    int size = 0;
+    // get the blocklet info object
+    BlockletInfoColumnar blockletInfo = getBlockletInfo(holder, 0);
+
+    List<DataChunk2> datachunks = null;
+    try {
+      // get all the data chunks
+      datachunks = CarbonMetadataUtil
+          .getDatachunk2(blockletInfo, thriftColumnSchemaList, dataWriterVo.getSegmentProperties());
+    } catch (IOException e) {
+      throw new CarbonDataWriterException("Problem while getting the data chunks", e);
+    }
+    // data chunk byte array
+    byte[][] dataChunkByteArray = new byte[datachunks.size()][];
+    for (int i = 0; i < dataChunkByteArray.length; i++) {
+      dataChunkByteArray[i] = CarbonUtil.getByteArray(datachunks.get(i));
+      // add the data chunk size
+      size += dataChunkByteArray[i].length;
+    }
+    // add row id index length
+    for (int i = 0; i < holder.getKeyBlockIndexLength().length; i++) {
+      size += holder.getKeyBlockIndexLength()[i];
+    }
+    // add rle index length
+    for (int i = 0; i < holder.getDataIndexMapLength().length; i++) {
+      size += holder.getDataIndexMapLength()[i];
+    }
+    // add dimension column data page and measure column data page size
+    long blockletDataSize =
+        holder.getTotalDimensionArrayLength() + holder.getTotalMeasureArrayLength() + size;
+    // if size of the file already reached threshold size then create a new file and get the file
+    // channel object
+    updateBlockletFileChannel(blockletDataSize);
+    // writer the version header in the file if current file size is zero
+    // this is done so carbondata file can be read separately
+    try {
+      if (fileChannel.size() == 0) {
+        short version = Short.parseShort(CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION));
+        byte[] header = (CarbonCommonConstants.CARBON_DATA_VERSION_HEADER + version).getBytes();
+        ByteBuffer buffer = ByteBuffer.allocate(header.length);
+        buffer.put(header);
+        buffer.rewind();
+        fileChannel.write(buffer);
+      }
+    } catch (IOException e) {
+      throw new CarbonDataWriterException("Problem while getting the file channel size", e);
+    }
+    // write data to file and get its offset
+    writeDataToFile(holder, dataChunkByteArray, fileChannel);
+    // add blocklet info to list
+    blockletInfoList.add(blockletInfo);
+    LOGGER.info("A new blocklet is added, its data size is: " + blockletDataSize + " Byte");
+  }
+
+  /**
+   * Below method will be used to write the data to file
+   * Data Format
+   * <DColumn1DataChunk><DColumnDataPage><DColumnRle>
+   * <DColumn2DataChunk><DColumn2DataPage><DColumn2RowIds><DColumn2Rle>
+   * <DColumn3DataChunk><DColumn3DataPage><column3RowIds>
+   * <MColumn1DataChunk><MColumn1DataPage>
+   * <MColumn2DataChunk><MColumn2DataPage>
+   * <MColumn2DataChunk><MColumn2DataPage>
+   *
+   * @param nodeHolder
+   * @param dataChunksBytes
+   * @param channel
+   * @throws CarbonDataWriterException
+   */
+  private void writeDataToFile(NodeHolder nodeHolder, byte[][] dataChunksBytes, FileChannel channel)
+      throws CarbonDataWriterException {
+    long offset = 0;
+    try {
+      offset = channel.size();
+    } catch (IOException e) {
+      throw new CarbonDataWriterException("Problem while getting the file channel size");
+    }
+    List<Long> currentDataChunksOffset = new ArrayList<>();
+    List<Short> currentDataChunksLength = new ArrayList<>();
+    dataChunksLength.add(currentDataChunksLength);
+    dataChunksOffsets.add(currentDataChunksOffset);
+    int bufferSize = 0;
+    int rowIdIndex = 0;
+    int rleIndex = 0;
+    for (int i = 0; i < nodeHolder.getIsSortedKeyBlock().length; i++) {
+      currentDataChunksOffset.add(offset);
+      currentDataChunksLength.add((short) dataChunksBytes[i].length);
+      bufferSize += dataChunksBytes[i].length + nodeHolder.getKeyLengths()[i] + (!nodeHolder
+          .getIsSortedKeyBlock()[i] ? nodeHolder.getKeyBlockIndexLength()[rowIdIndex] : 0) + (
+          dataWriterVo.getAggBlocks()[i] ?
+              nodeHolder.getCompressedDataIndex()[rleIndex].length :
+              0);
+      offset += dataChunksBytes[i].length;
+      offset += nodeHolder.getKeyLengths()[i];
+      if (!nodeHolder.getIsSortedKeyBlock()[i]) {
+        offset += nodeHolder.getKeyBlockIndexLength()[rowIdIndex];
+        rowIdIndex++;
+      }
+      if (dataWriterVo.getAggBlocks()[i]) {
+        offset += nodeHolder.getDataIndexMapLength()[rleIndex];
+        rleIndex++;
+      }
+    }
+    ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
+    rleIndex = 0;
+    rowIdIndex = 0;
+    for (int i = 0; i < nodeHolder.getIsSortedKeyBlock().length; i++) {
+      buffer.put(dataChunksBytes[i]);
+      buffer.put(nodeHolder.getKeyArray()[i]);
+      if (!nodeHolder.getIsSortedKeyBlock()[i]) {
+        buffer.putInt(nodeHolder.getCompressedIndex()[rowIdIndex].length);
+        buffer.put(nodeHolder.getCompressedIndex()[rowIdIndex]);
+        if (nodeHolder.getCompressedIndexMap()[rowIdIndex].length > 0) {
+          buffer.put(nodeHolder.getCompressedIndexMap()[rowIdIndex]);
+        }
+        rowIdIndex++;
+      }
+      if (dataWriterVo.getAggBlocks()[i]) {
+        buffer.put(nodeHolder.getCompressedDataIndex()[rleIndex]);
+        rleIndex++;
+      }
+    }
+    try {
+      buffer.flip();
+      channel.write(buffer);
+    } catch (IOException e) {
+      throw new CarbonDataWriterException(
+          "Problem while writing the dimension data in carbon data file", e);
+    }
+
+    int dataChunkIndex = nodeHolder.getKeyArray().length;
+    int totalLength = 0;
+    for (int i = 0; i < nodeHolder.getDataArray().length; i++) {
+      currentDataChunksOffset.add(offset);
+      currentDataChunksLength.add((short) dataChunksBytes[dataChunkIndex].length);
+      offset += dataChunksBytes[dataChunkIndex].length;
+      offset += nodeHolder.getDataArray()[i].length;
+      totalLength += dataChunksBytes[dataChunkIndex].length;
+      totalLength += nodeHolder.getDataArray()[i].length;
+      dataChunkIndex++;
+    }
+    buffer = ByteBuffer.allocate(totalLength);
+    dataChunkIndex = nodeHolder.getKeyArray().length;
+    for (int i = 0; i < nodeHolder.getDataArray().length; i++) {
+      buffer.put(dataChunksBytes[dataChunkIndex++]);
+      buffer.put(nodeHolder.getDataArray()[i]);
+    }
+    try {
+      buffer.flip();
+      channel.write(buffer);
+    } catch (IOException e) {
+      throw new CarbonDataWriterException(
+          "Problem while writing the measure data in carbon data file", e);
+    }
+  }
+
+  /**
+   * This method will be used to get the blocklet metadata
+   *
+   * @return BlockletInfo - blocklet metadata
+   */
+  protected BlockletInfoColumnar getBlockletInfo(NodeHolder nodeHolder, long offset) {
+    // create the info object for leaf entry
+    BlockletInfoColumnar info = new BlockletInfoColumnar();
+    //add aggBlocks array
+    info.setAggKeyBlock(nodeHolder.getAggBlocks());
+    // add total entry count
+    info.setNumberOfKeys(nodeHolder.getEntryCount());
+
+    // add the key array length
+    info.setKeyLengths(nodeHolder.getKeyLengths());
+    // adding null measure index bit set
+    info.setMeasureNullValueIndex(nodeHolder.getMeasureNullValueIndex());
+    //add column min max length
+    info.setColumnMaxData(nodeHolder.getColumnMaxData());
+    info.setColumnMinData(nodeHolder.getColumnMinData());
+
+    // add measure length
+    info.setMeasureLength(nodeHolder.getMeasureLenght());
+
+    info.setIsSortedKeyColumn(nodeHolder.getIsSortedKeyBlock());
+    info.setKeyBlockIndexLength(nodeHolder.getKeyBlockIndexLength());
+    info.setDataIndexMapLength(nodeHolder.getDataIndexMapLength());
+    // set startkey
+    info.setStartKey(nodeHolder.getStartKey());
+    // set end key
+    info.setEndKey(nodeHolder.getEndKey());
+    info.setCompressionModel(nodeHolder.getCompressionModel());
+    // return leaf metadata
+
+    //colGroup Blocks
+    info.setColGrpBlocks(nodeHolder.getColGrpBlocks());
+
+    return info;
+  }
+
+  /**
+   * This method will write metadata at the end of file file format in thrift format
+   */
+  protected void writeBlockletInfoToFile(List<BlockletInfoColumnar> infoList, FileChannel channel,
+      String filePath) throws CarbonDataWriterException {
+    try {
+      // get the current file position
+      long currentPosition = channel.size();
+      CarbonFooterWriter writer = new CarbonFooterWriter(filePath);
+      // get thrift file footer instance
+      FileFooter convertFileMeta = CarbonMetadataUtil
+          .convertFilterFooter2(infoList, localCardinality, thriftColumnSchemaList,
+              dataChunksOffsets, dataChunksLength);
+      // fill the carbon index details
+      fillBlockIndexInfoDetails(infoList, convertFileMeta.getNum_rows(), filePath, currentPosition);
+      // write the footer
+      writer.writeFooter(convertFileMeta, currentPosition);
+    } catch (IOException e) {
+      throw new CarbonDataWriterException("Problem while writing the carbon file: ", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
index 259482e..8c2608b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
@@ -26,48 +26,23 @@ import java.util.List;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.columnar.IndexStorage;
-import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression.SnappyByteCompression;
 import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
-import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.metadata.BlockletInfoColumnar;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
-import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
+import org.apache.carbondata.core.util.CarbonMetadataUtil;
+import org.apache.carbondata.core.writer.CarbonFooterWriter;
+import org.apache.carbondata.format.FileFooter;
 import org.apache.carbondata.processing.store.colgroup.ColGroupBlockStorage;
 import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException;
 
 public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFactDataWriter<int[]> {
 
-  protected boolean[] aggBlocks;
-  private NumberCompressor numberCompressor;
-  private boolean[] isComplexType;
-  private int numberOfNoDictionaryColumn;
-  private boolean[] isDictionaryColumn;
-  private static final LogService LOGGER = LogServiceFactory.getLogService(
-      CarbonFactDataWriterImplForIntIndexAndAggBlock.class.getName());
+  private static final LogService LOGGER = LogServiceFactory
+      .getLogService(CarbonFactDataWriterImplForIntIndexAndAggBlock.class.getName());
 
-  public CarbonFactDataWriterImplForIntIndexAndAggBlock(String storeLocation, int measureCount,
-      int mdKeyLength, String tableName, IFileManagerComposite fileManager, int[] keyBlockSize,
-      boolean[] aggBlocks, boolean[] isComplexType, int NoDictionaryCount,
-      CarbonDataFileAttributes carbonDataFileAttributes, String databaseName,
-      List<ColumnSchema> wrapperColumnSchemaList, int numberOfNoDictionaryColumn,
-      boolean[] isDictionaryColumn, String carbonDataDirectoryPath, int[] colCardinality,
-      SegmentProperties segmentProperties, int tableBlocksize) {
-    super(storeLocation, measureCount, mdKeyLength, databaseName, tableName, fileManager,
-        keyBlockSize, carbonDataFileAttributes, wrapperColumnSchemaList, carbonDataDirectoryPath,
-        colCardinality, segmentProperties, tableBlocksize);
-    this.isComplexType = isComplexType;
-    this.databaseName = databaseName;
-    this.numberOfNoDictionaryColumn = numberOfNoDictionaryColumn;
-    this.isDictionaryColumn = isDictionaryColumn;
-    this.aggBlocks = aggBlocks;
-    this.numberCompressor = new NumberCompressor(Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL)));
+  public CarbonFactDataWriterImplForIntIndexAndAggBlock(CarbonDataWriterVo dataWriterVo) {
+    super(dataWriterVo);
   }
 
   @Override
@@ -110,7 +85,7 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
 
       }
       totalKeySize += keyLengths[i];
-      if (isComplexType[i] || isDictionaryColumn[i]) {
+      if (dataWriterVo.getIsComplexType()[i] || dataWriterVo.getIsDictionaryColumn()[i]) {
         allMinValue[i] = keyStorageArray[i].getMin();
         allMaxValue[i] = keyStorageArray[i].getMax();
       } else {
@@ -142,16 +117,16 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
       }
     }
     int compressDataBlockSize = 0;
-    for (int i = 0; i < aggBlocks.length; i++) {
-      if (aggBlocks[i]) {
+    for (int i = 0; i < dataWriterVo.getAggBlocks().length; i++) {
+      if (dataWriterVo.getAggBlocks()[i]) {
         compressDataBlockSize++;
       }
     }
     byte[][] compressedDataIndex = new byte[compressDataBlockSize][];
     int[] dataIndexMapLength = new int[compressDataBlockSize];
     idx = 0;
-    for (int i = 0; i < aggBlocks.length; i++) {
-      if (aggBlocks[i]) {
+    for (int i = 0; i < dataWriterVo.getAggBlocks().length; i++) {
+      if (dataWriterVo.getAggBlocks()[i]) {
         try {
           compressedDataIndex[idx] =
               numberCompressor.compress(keyStorageArray[i].getDataIndexMap());
@@ -163,13 +138,7 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
       }
     }
 
-    byte[] writableKeyArray = new byte[totalKeySize];
-    int startPosition = 0;
-    for (int i = 0; i < keyLengths.length; i++) {
-      System.arraycopy(keyBlockData[i], 0, writableKeyArray, startPosition, keyBlockData[i].length);
-      startPosition += keyLengths[i];
-    }
-    int[] msrLength = new int[this.measureCount];
+    int[] msrLength = new int[dataWriterVo.getMeasureCount()];
     // calculate the total size required for all the measure and get the
     // each measure size
     for (int i = 0; i < dataArray.length; i++) {
@@ -177,30 +146,9 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
       totalMsrArrySize += currentMsrLenght;
       msrLength[i] = currentMsrLenght;
     }
-    byte[] writableDataArray = new byte[totalMsrArrySize];
-
-    // start position will be used for adding the measure in
-    // writableDataArray after adding measure increment the start position
-    // by added measure length which will be used for next measure start
-    // position
-    startPosition = 0;
-    for (int i = 0; i < dataArray.length; i++) {
-      System.arraycopy(dataArray[i], 0, writableDataArray, startPosition, dataArray[i].length);
-      startPosition += msrLength[i];
-    }
-    // current file size;
-    int indexBlockSize = 0;
-    for (int i = 0; i < keyBlockIdxLengths.length; i++) {
-      indexBlockSize += keyBlockIdxLengths[i] + CarbonCommonConstants.INT_SIZE_IN_BYTE;
-    }
-
-    for (int i = 0; i < dataIndexMapLength.length; i++) {
-      indexBlockSize += dataIndexMapLength[i];
-    }
-
     NodeHolder holder = new NodeHolder();
-    holder.setDataArray(writableDataArray);
-    holder.setKeyArray(writableKeyArray);
+    holder.setDataArray(dataArray);
+    holder.setKeyArray(keyBlockData);
     // end key format will be <length of dictionary key><length of no
     // dictionary key><DictionaryKey><No Dictionary key>
     byte[] updatedNoDictionaryEndKey = updateNoDictionaryStartAndEndKey(noDictionaryEndKey);
@@ -235,10 +183,12 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
     holder.setDataIndexMapLength(dataIndexMapLength);
     holder.setCompressedDataIndex(compressedDataIndex);
     holder.setCompressionModel(compressionModel);
+    holder.setTotalDimensionArrayLength(totalKeySize);
+    holder.setTotalMeasureArrayLength(totalMsrArrySize);
     //setting column min max value
     holder.setColumnMaxData(allMaxValue);
     holder.setColumnMinData(allMinValue);
-    holder.setAggBlocks(aggBlocks);
+    holder.setAggBlocks(dataWriterVo.getAggBlocks());
     holder.setColGrpBlocks(colGrpBlock);
     return holder;
   }
@@ -252,113 +202,28 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
     for (int i = 0; i < holder.getDataIndexMapLength().length; i++) {
       indexBlockSize += holder.getDataIndexMapLength()[i];
     }
+
     long blockletDataSize =
-        holder.getKeyArray().length + holder.getDataArray().length + indexBlockSize;
+        holder.getTotalDimensionArrayLength() + holder.getTotalMeasureArrayLength()
+            + indexBlockSize;
     updateBlockletFileChannel(blockletDataSize);
-    writeDataToFile(holder);
+    // write data to file and get its offset
+    long offset = writeDataToFile(holder, fileChannel);
+    // get the blocklet info for currently added blocklet
+    BlockletInfoColumnar blockletInfo = getBlockletInfo(holder, offset);
+    // add blocklet info to list
+    blockletInfoList.add(blockletInfo);
     LOGGER.info("A new blocklet is added, its data size is: " + blockletDataSize + " Byte");
   }
 
   /**
-   * Below method will be used to update the min or max value
-   * by removing the length from it
-   *
-   * @return min max value without length
-   */
-  private byte[] updateMinMaxForNoDictionary(byte[] valueWithLength) {
-    ByteBuffer buffer = ByteBuffer.wrap(valueWithLength);
-    byte[] actualValue = new byte[buffer.getShort()];
-    buffer.get(actualValue);
-    return actualValue;
-  }
-
-  /**
-   * Below method will be used to update the no dictionary start and end key
-   *
-   * @param key key to be updated
-   * @return return no dictionary key
-   */
-  private byte[] updateNoDictionaryStartAndEndKey(byte[] key) {
-    if (key.length == 0) {
-      return key;
-    }
-    // add key to byte buffer remove the length part of the data
-    ByteBuffer buffer = ByteBuffer.wrap(key, 2, key.length - 2);
-    // create a output buffer without length
-    ByteBuffer output = ByteBuffer.allocate(key.length - 2);
-    short numberOfByteToStorLength = 2;
-    // as length part is removed, so each no dictionary value index
-    // needs to be reshuffled by 2 bytes
-    for (int i = 0; i < numberOfNoDictionaryColumn; i++) {
-      output.putShort((short) (buffer.getShort() - numberOfByteToStorLength));
-    }
-    // copy the data part
-    while (buffer.hasRemaining()) {
-      output.put(buffer.get());
-    }
-    output.rewind();
-    return output.array();
-  }
-
-  protected byte[][] fillAndCompressedKeyBlockData(IndexStorage<int[]>[] keyStorageArray,
-      int entryCount) {
-    byte[][] keyBlockData = new byte[keyStorageArray.length][];
-    int destPos = 0;
-    int keyBlockSizePosition = -1;
-    for (int i = 0; i < keyStorageArray.length; i++) {
-      destPos = 0;
-      //handling for high card dims
-      if (!isComplexType[i] && !this.isDictionaryColumn[i]) {
-        int totalLength = 0;
-        // calc size of the total bytes in all the colmns.
-        for (int k = 0; k < keyStorageArray[i].getKeyBlock().length; k++) {
-          byte[] colValue = keyStorageArray[i].getKeyBlock()[k];
-          totalLength += colValue.length;
-        }
-        keyBlockData[i] = new byte[totalLength];
-
-        for (int j = 0; j < keyStorageArray[i].getKeyBlock().length; j++) {
-          int length = keyStorageArray[i].getKeyBlock()[j].length;
-          System
-              .arraycopy(keyStorageArray[i].getKeyBlock()[j], 0, keyBlockData[i], destPos, length);
-          destPos += length;
-        }
-      } else {
-        keyBlockSizePosition++;
-        if (aggBlocks[i]) {
-          keyBlockData[i] = new byte[keyStorageArray[i].getTotalSize()];
-          for (int j = 0; j < keyStorageArray[i].getKeyBlock().length; j++) {
-            System.arraycopy(keyStorageArray[i].getKeyBlock()[j], 0, keyBlockData[i], destPos,
-                keyStorageArray[i].getKeyBlock()[j].length);
-            destPos += keyStorageArray[i].getKeyBlock()[j].length;
-          }
-        } else {
-          if (isComplexType[i]) {
-            keyBlockData[i] = new byte[keyStorageArray[i].getKeyBlock().length
-                * keyBlockSize[keyBlockSizePosition]];
-          } else {
-            keyBlockData[i] = new byte[entryCount * keyBlockSize[keyBlockSizePosition]];
-          }
-          for (int j = 0; j < keyStorageArray[i].getKeyBlock().length; j++) {
-            System.arraycopy(keyStorageArray[i].getKeyBlock()[j], 0, keyBlockData[i], destPos,
-                keyBlockSize[keyBlockSizePosition]);
-            destPos += keyBlockSize[keyBlockSizePosition];
-          }
-        }
-      }
-      keyBlockData[i] = SnappyByteCompression.INSTANCE.compress(keyBlockData[i]);
-    }
-    return keyBlockData;
-  }
-
-  /**
    * This method is responsible for writing blocklet to the data file
    *
    * @return file offset offset is the current position of the file
    * @throws CarbonDataWriterException if will throw CarbonDataWriterException when any thing
    *                                   goes wrong while while writing the leaf file
    */
-  protected long writeDataToFile(NodeHolder nodeHolder, FileChannel channel)
+  private long writeDataToFile(NodeHolder nodeHolder, FileChannel channel)
       throws CarbonDataWriterException {
     // create byte buffer
     byte[][] compressedIndex = nodeHolder.getCompressedIndex();
@@ -375,15 +240,20 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
       indexBlockSize += nodeHolder.getDataIndexMapLength()[i];
     }
     ByteBuffer byteBuffer = ByteBuffer.allocate(
-        nodeHolder.getKeyArray().length + nodeHolder.getDataArray().length + indexBlockSize);
+        nodeHolder.getTotalDimensionArrayLength() + nodeHolder.getTotalMeasureArrayLength()
+            + indexBlockSize);
     long offset = 0;
     try {
       // get the current offset
       offset = channel.size();
       // add key array to byte buffer
-      byteBuffer.put(nodeHolder.getKeyArray());
+      for (int i = 0; i < nodeHolder.getKeyArray().length; i++) {
+        byteBuffer.put(nodeHolder.getKeyArray()[i]);
+      }
+      for (int i = 0; i < nodeHolder.getDataArray().length; i++) {
+        byteBuffer.put(nodeHolder.getDataArray()[i]);
+      }
       // add measure data array to byte buffer
-      byteBuffer.put(nodeHolder.getDataArray());
 
       ByteBuffer buffer1 = null;
       for (int i = 0; i < compressedIndex.length; i++) {
@@ -448,9 +318,9 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
     // add measure length
     info.setMeasureLength(nodeHolder.getMeasureLenght());
 
-    long[] msrOffset = new long[this.measureCount];
+    long[] msrOffset = new long[dataWriterVo.getMeasureCount()];
 
-    for (int i = 0; i < this.measureCount; i++) {
+    for (int i = 0; i < msrOffset.length; i++) {
       // increment the current offset by 4 bytes because 4 bytes will be
       // used for measure byte length
       //            offset += CarbonCommonConstants.INT_SIZE_IN_BYTE;
@@ -489,4 +359,21 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
     return info;
   }
 
+  /**
+   * This method will write metadata at the end of file file format in thrift format
+   */
+  protected void writeBlockletInfoToFile(List<BlockletInfoColumnar> infoList, FileChannel channel,
+      String filePath) throws CarbonDataWriterException {
+    try {
+      long currentPosition = channel.size();
+      CarbonFooterWriter writer = new CarbonFooterWriter(filePath);
+      FileFooter convertFileMeta = CarbonMetadataUtil
+          .convertFileFooter(infoList, localCardinality.length, localCardinality,
+              thriftColumnSchemaList, dataWriterVo.getSegmentProperties());
+      fillBlockIndexInfoDetails(infoList, convertFileMeta.getNum_rows(), filePath, currentPosition);
+      writer.writeFooter(convertFileMeta, currentPosition);
+    } catch (IOException e) {
+      throw new CarbonDataWriterException("Problem while writing the carbon file: ", e);
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/processing/src/main/java/org/apache/carbondata/processing/store/writer/NodeHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/NodeHolder.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/NodeHolder.java
index aa758e6..a7d14f0 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/NodeHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/NodeHolder.java
@@ -27,12 +27,12 @@ public class NodeHolder {
   /**
    * keyArray
    */
-  private byte[] keyArray;
+  private byte[][] keyArray;
 
   /**
    * dataArray
    */
-  private byte[] dataArray;
+  private byte[][] dataArray;
 
   /**
    * measureLenght
@@ -139,30 +139,40 @@ public class NodeHolder {
   private BitSet[] measureNullValueIndex;
 
   /**
+   * total length of dimension values
+   */
+  private int totalDimensionArrayLength;
+
+  /**
+   * total length of all measure values
+   */
+  private int totalMeasureArrayLength;
+
+  /**
    * @return the keyArray
    */
-  public byte[] getKeyArray() {
+  public byte[][] getKeyArray() {
     return keyArray;
   }
 
   /**
    * @param keyArray the keyArray to set
    */
-  public void setKeyArray(byte[] keyArray) {
+  public void setKeyArray(byte[][] keyArray) {
     this.keyArray = keyArray;
   }
 
   /**
    * @return the dataArray
    */
-  public byte[] getDataArray() {
+  public byte[][] getDataArray() {
     return dataArray;
   }
 
   /**
    * @param dataArray the dataArray to set
    */
-  public void setDataArray(byte[] dataArray) {
+  public void setDataArray(byte[][] dataArray) {
     this.dataArray = dataArray;
   }
 
@@ -453,4 +463,20 @@ public class NodeHolder {
   public void setMeasureNullValueIndex(BitSet[] measureNullValueIndex) {
     this.measureNullValueIndex = measureNullValueIndex;
   }
+
+  public int getTotalDimensionArrayLength() {
+    return totalDimensionArrayLength;
+  }
+
+  public void setTotalDimensionArrayLength(int totalDimensionArrayLength) {
+    this.totalDimensionArrayLength = totalDimensionArrayLength;
+  }
+
+  public int getTotalMeasureArrayLength() {
+    return totalMeasureArrayLength;
+  }
+
+  public void setTotalMeasureArrayLength(int totalMeasureArrayLength) {
+    this.totalMeasureArrayLength = totalMeasureArrayLength;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
index 62b442f..84192b8 100644
--- a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
+++ b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
@@ -35,9 +35,12 @@ import org.apache.carbondata.core.carbon.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.datastore.exception.IndexBuilderException;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.test.util.StoreCreator;
 
 import junit.framework.TestCase;
+
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -45,10 +48,22 @@ public class BlockIndexStoreTest extends TestCase {
 
   private BlockIndexStore indexStore;
 
+  private String property;
   @BeforeClass public void setUp() {
+	property = CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION);
+	
+	CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, "1");  
     StoreCreator.createCarbonStore();
     indexStore = BlockIndexStore.getInstance();
   }
+  
+  @AfterClass public void tearDown() {
+	    if(null!=property) {
+		CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, property);
+	    }else {
+	    	CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION+"");
+	    }
+	  }
 
   @Test public void testloadAndGetTaskIdToSegmentsMapForSingleSegment() throws IOException {
     String canonicalPath =
@@ -56,7 +71,7 @@ public class BlockIndexStoreTest extends TestCase {
     File file = getPartFile();
     TableBlockInfo info =
         new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
+            file.length(),(short)1);
     CarbonTableIdentifier carbonTableIdentifier =
             new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
     AbsoluteTableIdentifier absoluteTableIdentifier =
@@ -78,20 +93,20 @@ public class BlockIndexStoreTest extends TestCase {
     File file = getPartFile();
     TableBlockInfo info =
         new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
     TableBlockInfo info1 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
 
     TableBlockInfo info2 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
     TableBlockInfo info3 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
     TableBlockInfo info4 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
 
     CarbonTableIdentifier carbonTableIdentifier =
             new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
@@ -133,31 +148,31 @@ public class BlockIndexStoreTest extends TestCase {
     File file = getPartFile();
     TableBlockInfo info =
         new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
     TableBlockInfo info1 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
 
     TableBlockInfo info2 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
     TableBlockInfo info3 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
     TableBlockInfo info4 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
 
     TableBlockInfo info5 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
-            file.length());
+            file.length(),(short)1);
     TableBlockInfo info6 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
 
     TableBlockInfo info7 =
         new TableBlockInfo(file.getAbsolutePath(), 0, "3", new String[] { "loclhost" },
-            file.length());
+            file.length(), (short)1);
 
     CarbonTableIdentifier carbonTableIdentifier =
             new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");


[4/5] incubator-carbondata git commit: Improve first time query performance

Posted by ra...@apache.org.
Improve first time query performance

Rebased


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

Branch: refs/heads/master
Commit: d54dc647c69496ecaa7e0c8a9cc3d8e9028ab73f
Parents: 9ad98f4
Author: kumarvishal <ku...@gmail.com>
Authored: Thu Oct 27 22:54:49 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Dec 1 15:18:35 2016 +0530

----------------------------------------------------------------------
 .../core/carbon/datastore/BlockIndexStore.java  |   4 +-
 .../core/carbon/datastore/DataRefNode.java      |  18 +-
 .../datastore/block/SegmentProperties.java      |   7 +
 .../carbon/datastore/block/TableBlockInfo.java  |  20 +-
 .../chunk/reader/CarbonDataReaderFactory.java   |  93 +++++
 .../reader/DimensionColumnChunkReader.java      |   2 +-
 .../chunk/reader/MeasureColumnChunkReader.java  |   2 +-
 .../reader/dimension/AbstractChunkReader.java   |  12 +-
 ...CompressedDimensionChunkFileBasedReader.java | 135 -------
 ...mpressedDimensionChunkFileBasedReaderV1.java | 146 +++++++
 ...mpressedDimensionChunkFileBasedReaderV2.java | 308 +++++++++++++++
 .../measure/AbstractMeasureChunkReader.java     |  33 +-
 .../CompressedMeasureChunkFileBasedReader.java  |  92 -----
 ...CompressedMeasureChunkFileBasedReaderV1.java | 106 +++++
 ...CompressedMeasureChunkFileBasedReaderV2.java | 234 +++++++++++
 .../impl/btree/AbstractBTreeLeafNode.java       |   4 +-
 .../datastore/impl/btree/BTreeNonLeafNode.java  |   4 +-
 .../impl/btree/BlockletBTreeLeafNode.java       |  36 +-
 .../carbon/metadata/blocklet/BlockletInfo.java  |  40 ++
 .../metadata/blocklet/DataFileFooter.java       |   6 +-
 .../core/constants/CarbonCommonConstants.java   |  18 +-
 .../util/AbstractDataFileFooterConverter.java   | 390 ++++++++++++++++++
 .../core/util/CarbonMetadataUtil.java           | 158 +++++++-
 .../carbondata/core/util/CarbonProperties.java  | 121 +++---
 .../apache/carbondata/core/util/CarbonUtil.java | 209 ++++++++--
 .../core/util/DataFileFooterConverter.java      | 391 +------------------
 .../core/util/DataFileFooterConverter2.java     | 135 +++++++
 .../util/DataFileFooterConverterFactory.java    |  64 +++
 .../core/writer/CarbonFooterWriter.java         |   4 -
 .../executor/impl/AbstractQueryExecutor.java    |  55 ++-
 .../executor/impl/QueryExecutorProperties.java  |   4 +-
 .../scan/executor/infos/BlockExecutionInfo.java |  12 +-
 .../scan/executor/util/QueryUtil.java           | 102 +++--
 .../scan/scanner/AbstractBlockletScanner.java   |   7 +-
 .../scan/scanner/impl/FilterScanner.java        |  48 ++-
 .../carbon/datastore/block/BlockInfoTest.java   |  12 +-
 .../datastore/block/TableBlockInfoTest.java     |  32 +-
 .../datastore/block/TableTaskInfoTest.java      |   8 +-
 ...ressedDimensionChunkFileBasedReaderTest.java |  23 +-
 ...mpressedMeasureChunkFileBasedReaderTest.java |  30 +-
 .../core/util/CarbonMetadataUtilTest.java       |   3 +-
 .../carbondata/core/util/CarbonUtilTest.java    |  18 +-
 .../core/util/DataFileFooterConverterTest.java  |  11 +-
 format/src/main/thrift/carbondata.thrift        |  30 +-
 .../carbondata/hadoop/CarbonInputFormat.java    |  66 ++--
 .../carbondata/hadoop/CarbonInputSplit.java     | 107 +++--
 .../internal/index/impl/InMemoryBTreeIndex.java |   5 +-
 .../spark/merger/CarbonCompactionUtil.java      |   4 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   4 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   2 +-
 .../spark/src/test/resources/OLDFORMATTABLE.csv |  34 ++
 .../src/test/resources/OLDFORMATTABLEHIVE.csv   |  33 ++
 .../TestQueryWithOldCarbonDataFile.scala        |  70 ++++
 .../store/CarbonDataWriterFactory.java          |  70 ++++
 .../store/CarbonFactDataHandlerColumnar.java    |  54 ++-
 .../store/writer/AbstractFactDataWriter.java    | 316 +++++++--------
 .../store/writer/CarbonDataWriterVo.java        | 321 +++++++++++++++
 .../store/writer/CarbonFactDataWriterImpl2.java | 285 ++++++++++++++
 ...actDataWriterImplForIntIndexAndAggBlock.java | 223 +++--------
 .../processing/store/writer/NodeHolder.java     |  38 +-
 .../carbon/datastore/BlockIndexStoreTest.java   |  43 +-
 61 files changed, 3504 insertions(+), 1358 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
index 34c2983..d7ba318 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
@@ -265,9 +265,7 @@ public class BlockIndexStore {
     AbstractIndex tableBlock;
     DataFileFooter footer;
     // getting the data file meta data of the block
-    footer = CarbonUtil.readMetadatFile(blockInfo.getTableBlockInfo().getFilePath(),
-        blockInfo.getTableBlockInfo().getBlockOffset(),
-        blockInfo.getTableBlockInfo().getBlockLength());
+    footer = CarbonUtil.readMetadatFile(blockInfo.getTableBlockInfo());
     tableBlock = new BlockIndex();
     footer.setBlockInfo(blockInfo);
     // building the block

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/DataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/DataRefNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/DataRefNode.java
index e81a9a6..0ddd8c5 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/DataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/DataRefNode.java
@@ -71,10 +71,15 @@ public interface DataRefNode {
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes indexes of the blocks need to be read
+   * @param blockIndexes range indexes of the blocks need to be read
+   *                     value can be {{0,10},{11,12},{13,13}}
+   *                     here 0 to 10 and 11 to 12 column blocks will be read in one
+   *                     IO operation 13th column block will be read separately
+   *                     This will be helpful to reduce IO by reading bigger chunk of
+   *                     data in On IO
    * @return dimension data chunks
    */
-  DimensionColumnDataChunk[] getDimensionChunks(FileHolder fileReader, int[] blockIndexes);
+  DimensionColumnDataChunk[] getDimensionChunks(FileHolder fileReader, int[][] blockIndexes);
 
   /**
    * Below method will be used to get the dimension chunk
@@ -89,10 +94,15 @@ public interface DataRefNode {
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes block indexes to be read from file
+   * @param blockIndexes range indexes of the blocks need to be read
+   *                     value can be {{0,10},{11,12},{13,13}}
+   *                     here 0 to 10 and 11 to 12 column blocks will be read in one
+   *                     IO operation 13th column block will be read separately
+   *                     This will be helpful to reduce IO by reading bigger chunk of
+   *                     data in On IO
    * @return measure column data chunk
    */
-  MeasureColumnDataChunk[] getMeasureChunks(FileHolder fileReader, int[] blockIndexes);
+  MeasureColumnDataChunk[] getMeasureChunks(FileHolder fileReader, int[][] blockIndexes);
 
   /**
    * Below method will be used to read the measure chunk

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
index 816ca3a..05ad4e6 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
@@ -745,4 +745,11 @@ public class SegmentProperties {
     return blockTodimensionOrdinalMapping.get(blockIndex);
   }
 
+  /**
+   * @return It returns block index to dimension ordinal mapping
+   */
+  public Map<Integer, Set<Integer>> getBlockTodimensionOrdinalMapping() {
+    return blockTodimensionOrdinalMapping;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
index 4bf0047..0d60567 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
@@ -56,18 +56,21 @@ public class TableBlockInfo implements Distributable, Serializable {
   private String segmentId;
 
   private String[] locations;
+
+  private short version;
   /**
    * The class holds the blockletsinfo
    */
   private BlockletInfos blockletInfos = new BlockletInfos();
 
   public TableBlockInfo(String filePath, long blockOffset, String segmentId, String[] locations,
-      long blockLength) {
+      long blockLength, short version) {
     this.filePath = FileFactory.getUpdatedFilePath(filePath);
     this.blockOffset = blockOffset;
     this.segmentId = segmentId;
     this.locations = locations;
     this.blockLength = blockLength;
+    this.version = version;
   }
 
   /**
@@ -81,13 +84,14 @@ public class TableBlockInfo implements Distributable, Serializable {
    * @param blockletInfos
    */
   public TableBlockInfo(String filePath, long blockOffset, String segmentId, String[] locations,
-      long blockLength, BlockletInfos blockletInfos) {
+      long blockLength, BlockletInfos blockletInfos, short version) {
     this.filePath = FileFactory.getUpdatedFilePath(filePath);
     this.blockOffset = blockOffset;
     this.segmentId = segmentId;
     this.locations = locations;
     this.blockLength = blockLength;
     this.blockletInfos = blockletInfos;
+    this.version = version;
   }
 
   /**
@@ -104,6 +108,10 @@ public class TableBlockInfo implements Distributable, Serializable {
     return blockOffset;
   }
 
+  public void setBlockOffset(long blockOffset) {
+    this.blockOffset = blockOffset;
+  }
+
   /**
    * @return the segmentId
    */
@@ -250,4 +258,12 @@ public class TableBlockInfo implements Distributable, Serializable {
   public void setBlockletInfos(BlockletInfos blockletInfos) {
     this.blockletInfos = blockletInfos;
   }
+
+  public short getVersion() {
+    return version;
+  }
+
+  public void setVersion(short version) {
+    this.version = version;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/CarbonDataReaderFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/CarbonDataReaderFactory.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/CarbonDataReaderFactory.java
new file mode 100644
index 0000000..08a1869
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/CarbonDataReaderFactory.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.datastore.chunk.reader;
+
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.v1.CompressedDimensionChunkFileBasedReaderV1;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.v2.CompressedDimensionChunkFileBasedReaderV2;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.v1.CompressedMeasureChunkFileBasedReaderV1;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.v2.CompressedMeasureChunkFileBasedReaderV2;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
+
+/**
+ * Factory class to get the data reader instance based on version
+ */
+public class CarbonDataReaderFactory {
+
+  /**
+   * static instance
+   */
+  private static final CarbonDataReaderFactory CARBON_DATA_READER_FACTORY =
+      new CarbonDataReaderFactory();
+
+  /**
+   * private constructor
+   */
+  private CarbonDataReaderFactory() {
+
+  }
+
+  /**
+   * To get the instance of the reader factor
+   *
+   * @return reader factory
+   */
+  public static CarbonDataReaderFactory getInstance() {
+    return CARBON_DATA_READER_FACTORY;
+  }
+
+  /**
+   * Below method will be used to get the dimension column chunk reader based on version number
+   *
+   * @param version             reader version
+   * @param blockletInfo        blocklet info
+   * @param eachColumnValueSize size of each dimension column
+   * @param filePath            carbon data file path
+   * @return dimension column data reader based on version number
+   */
+  public DimensionColumnChunkReader getDimensionColumnChunkReader(short version,
+      BlockletInfo blockletInfo, int[] eachColumnValueSize, String filePath) {
+    switch (version) {
+      case 2:
+        return new CompressedDimensionChunkFileBasedReaderV2(blockletInfo, eachColumnValueSize,
+            filePath);
+      default:
+        return new CompressedDimensionChunkFileBasedReaderV1(blockletInfo, eachColumnValueSize,
+            filePath);
+    }
+  }
+
+  /**
+   * Below method will be used to get the measure column chunk reader based version number
+   *
+   * @param version      reader version
+   * @param blockletInfo blocklet info
+   * @param filePath     carbon data file path
+   * @return measure column data reader based on version number
+   */
+  public MeasureColumnChunkReader getMeasureColumnChunkReader(short version,
+      BlockletInfo blockletInfo, String filePath) {
+    switch (version) {
+      case 2:
+        return new CompressedMeasureChunkFileBasedReaderV2(blockletInfo, filePath);
+      default:
+        return new CompressedMeasureChunkFileBasedReaderV1(blockletInfo, filePath);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java
index b958245..0153211 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java
@@ -35,7 +35,7 @@ public interface DimensionColumnChunkReader {
    * @param blockIndexes blocks to be read
    * @return dimension column chunks
    */
-  DimensionColumnDataChunk[] readDimensionChunks(FileHolder fileReader, int... blockIndexes);
+  DimensionColumnDataChunk[] readDimensionChunks(FileHolder fileReader, int[][] blockIndexes);
 
   /**
    * Below method will be used to read the chunk based on block index

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java
index 8a7c8ef..523a14e 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java
@@ -33,7 +33,7 @@ public interface MeasureColumnChunkReader {
    * @param blockIndexes blocks to be read
    * @return measure data chunks
    */
-  MeasureColumnDataChunk[] readMeasureChunks(FileHolder fileReader, int... blockIndexes);
+  MeasureColumnDataChunk[] readMeasureChunks(FileHolder fileReader, int[][] blockIndexes);
 
   /**
    * Method to read the blocks data based on block index

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java
index 59dcd38..ced33fe 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.core.carbon.datastore.chunk.reader.DimensionColumnChunkReader;
-import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.compression.Compressor;
 import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression;
@@ -43,12 +42,6 @@ public abstract class AbstractChunkReader implements DimensionColumnChunkReader
       SnappyCompression.SnappyByteCompression.INSTANCE;
 
   /**
-   * data chunk list which holds the information
-   * about the data block metadata
-   */
-  protected List<DataChunk> dimensionColumnChunk;
-
-  /**
    * size of the each column value
    * for no dictionary column it will be -1
    */
@@ -79,9 +72,8 @@ public abstract class AbstractChunkReader implements DimensionColumnChunkReader
    * @param eachColumnValueSize  size of the each column value
    * @param filePath             file from which data will be read
    */
-  public AbstractChunkReader(List<DataChunk> dimensionColumnChunk, int[] eachColumnValueSize,
-      String filePath) {
-    this.dimensionColumnChunk = dimensionColumnChunk;
+  public AbstractChunkReader(final int[] eachColumnValueSize,
+      final String filePath) {
     this.eachColumnValueSize = eachColumnValueSize;
     this.filePath = filePath;
     int numberOfElement = 0;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java
deleted file mode 100644
index 209217b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension;
-
-import java.util.List;
-
-import org.apache.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
-import org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.carbon.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
-import org.apache.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.apache.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
-import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
-import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.apache.carbondata.core.datastorage.store.FileHolder;
-import org.apache.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-/**
- * Compressed dimension chunk reader class
- */
-public class CompressedDimensionChunkFileBasedReader extends AbstractChunkReader {
-
-  /**
-   * Constructor to get minimum parameter to create instance of this class
-   *
-   * @param dimensionColumnChunk dimension chunk metadata
-   * @param eachColumnValueSize  size of the each column value
-   * @param filePath             file from which data will be read
-   */
-  public CompressedDimensionChunkFileBasedReader(List<DataChunk> dimensionColumnChunk,
-      int[] eachColumnValueSize, String filePath) {
-    super(dimensionColumnChunk, eachColumnValueSize, filePath);
-  }
-
-  /**
-   * Below method will be used to read the chunk based on block indexes
-   *
-   * @param fileReader   file reader to read the blocks from file
-   * @param blockIndexes blocks to be read
-   * @return dimension column chunks
-   */
-  @Override public DimensionColumnDataChunk[] readDimensionChunks(FileHolder fileReader,
-      int... blockIndexes) {
-    // read the column chunk based on block index and add
-    DimensionColumnDataChunk[] dataChunks =
-        new DimensionColumnDataChunk[dimensionColumnChunk.size()];
-    for (int i = 0; i < blockIndexes.length; i++) {
-      dataChunks[blockIndexes[i]] = readDimensionChunk(fileReader, blockIndexes[i]);
-    }
-    return dataChunks;
-  }
-
-  /**
-   * Below method will be used to read the chunk based on block index
-   *
-   * @param fileReader file reader to read the blocks from file
-   * @param blockIndex block to be read
-   * @return dimension column chunk
-   */
-  @Override public DimensionColumnDataChunk readDimensionChunk(FileHolder fileReader,
-      int blockIndex) {
-    byte[] dataPage = null;
-    int[] invertedIndexes = null;
-    int[] invertedIndexesReverse = null;
-    int[] rlePage = null;
-
-    // first read the data and uncompressed it
-    dataPage = COMPRESSOR.unCompress(fileReader
-        .readByteArray(filePath, dimensionColumnChunk.get(blockIndex).getDataPageOffset(),
-            dimensionColumnChunk.get(blockIndex).getDataPageLength()));
-    // if row id block is present then read the row id chunk and uncompress it
-    if (CarbonUtil.hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(),
-        Encoding.INVERTED_INDEX)) {
-      invertedIndexes = CarbonUtil
-          .getUnCompressColumnIndex(dimensionColumnChunk.get(blockIndex).getRowIdPageLength(),
-              fileReader.readByteArray(filePath,
-                  dimensionColumnChunk.get(blockIndex).getRowIdPageOffset(),
-                  dimensionColumnChunk.get(blockIndex).getRowIdPageLength()), numberComressor);
-      // get the reverse index
-      invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
-    }
-    // if rle is applied then read the rle block chunk and then uncompress
-    //then actual data based on rle block
-    if (CarbonUtil
-        .hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(), Encoding.RLE)) {
-      // read and uncompress the rle block
-      rlePage = numberComressor.unCompress(fileReader
-          .readByteArray(filePath, dimensionColumnChunk.get(blockIndex).getRlePageOffset(),
-              dimensionColumnChunk.get(blockIndex).getRlePageLength()));
-      // uncompress the data with rle indexes
-      dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[blockIndex]);
-      rlePage = null;
-    }
-    // fill chunk attributes
-    DimensionChunkAttributes chunkAttributes = new DimensionChunkAttributes();
-    chunkAttributes.setEachRowSize(eachColumnValueSize[blockIndex]);
-    chunkAttributes.setInvertedIndexes(invertedIndexes);
-    chunkAttributes.setInvertedIndexesReverse(invertedIndexesReverse);
-    DimensionColumnDataChunk columnDataChunk = null;
-
-    if (dimensionColumnChunk.get(blockIndex).isRowMajor()) {
-      // to store fixed length column chunk values
-      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, chunkAttributes);
-    }
-    // if no dictionary column then first create a no dictionary column chunk
-    // and set to data chunk instance
-    else if (!CarbonUtil
-        .hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(), Encoding.DICTIONARY)) {
-      columnDataChunk =
-          new VariableLengthDimensionDataChunk(getNoDictionaryDataChunk(dataPage), chunkAttributes);
-      chunkAttributes.setNoDictionary(true);
-    } else {
-      // to store fixed length column chunk values
-      columnDataChunk = new FixedLengthDimensionDataChunk(dataPage, chunkAttributes);
-    }
-    return columnDataChunk;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
new file mode 100644
index 0000000..3ead985
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.v1;
+
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
+import org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.AbstractChunkReader;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+/**
+ * Compressed dimension chunk reader class
+ */
+public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkReader {
+
+  /**
+   * data chunk list which holds the information
+   * about the data block metadata
+   */
+  private final List<DataChunk> dimensionColumnChunk;
+
+  /**
+   * Constructor to get minimum parameter to create instance of this class
+   *
+   * @param blockletInfo        blocklet info
+   * @param eachColumnValueSize size of the each column value
+   * @param filePath            file from which data will be read
+   */
+  public CompressedDimensionChunkFileBasedReaderV1(final BlockletInfo blockletInfo,
+      final int[] eachColumnValueSize, final String filePath) {
+    super(eachColumnValueSize, filePath);
+    this.dimensionColumnChunk = blockletInfo.getDimensionColumnChunk();
+  }
+
+  /**
+   * Below method will be used to read the chunk based on block indexes
+   *
+   * @param fileReader   file reader to read the blocks from file
+   * @param blockIndexes blocks to be read
+   * @return dimension column chunks
+   */
+  @Override public DimensionColumnDataChunk[] readDimensionChunks(FileHolder fileReader,
+      int[][] blockIndexes) {
+    // read the column chunk based on block index and add
+    DimensionColumnDataChunk[] dataChunks =
+        new DimensionColumnDataChunk[dimensionColumnChunk.size()];
+    for (int i = 0; i < blockIndexes.length; i++) {
+      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+        dataChunks[j] = readDimensionChunk(fileReader, j);
+      }
+    }
+    return dataChunks;
+  }
+
+  /**
+   * Below method will be used to read the chunk based on block index
+   *
+   * @param fileReader file reader to read the blocks from file
+   * @param blockIndex block to be read
+   * @return dimension column chunk
+   */
+  @Override public DimensionColumnDataChunk readDimensionChunk(FileHolder fileReader,
+      int blockIndex) {
+    byte[] dataPage = null;
+    int[] invertedIndexes = null;
+    int[] invertedIndexesReverse = null;
+    int[] rlePage = null;
+
+    // first read the data and uncompressed it
+    dataPage = COMPRESSOR.unCompress(fileReader
+        .readByteArray(filePath, dimensionColumnChunk.get(blockIndex).getDataPageOffset(),
+            dimensionColumnChunk.get(blockIndex).getDataPageLength()));
+    // if row id block is present then read the row id chunk and uncompress it
+    if (CarbonUtil.hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(),
+        Encoding.INVERTED_INDEX)) {
+      invertedIndexes = CarbonUtil
+          .getUnCompressColumnIndex(dimensionColumnChunk.get(blockIndex).getRowIdPageLength(),
+              fileReader.readByteArray(filePath,
+                  dimensionColumnChunk.get(blockIndex).getRowIdPageOffset(),
+                  dimensionColumnChunk.get(blockIndex).getRowIdPageLength()), numberComressor);
+      // get the reverse index
+      invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
+    }
+    // if rle is applied then read the rle block chunk and then uncompress
+    //then actual data based on rle block
+    if (CarbonUtil
+        .hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(), Encoding.RLE)) {
+      // read and uncompress the rle block
+      rlePage = numberComressor.unCompress(fileReader
+          .readByteArray(filePath, dimensionColumnChunk.get(blockIndex).getRlePageOffset(),
+              dimensionColumnChunk.get(blockIndex).getRlePageLength()));
+      // uncompress the data with rle indexes
+      dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[blockIndex]);
+      rlePage = null;
+    }
+    // fill chunk attributes
+    DimensionChunkAttributes chunkAttributes = new DimensionChunkAttributes();
+    chunkAttributes.setEachRowSize(eachColumnValueSize[blockIndex]);
+    chunkAttributes.setInvertedIndexes(invertedIndexes);
+    chunkAttributes.setInvertedIndexesReverse(invertedIndexesReverse);
+    DimensionColumnDataChunk columnDataChunk = null;
+
+    if (dimensionColumnChunk.get(blockIndex).isRowMajor()) {
+      // to store fixed length column chunk values
+      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, chunkAttributes);
+    }
+    // if no dictionary column then first create a no dictionary column chunk
+    // and set to data chunk instance
+    else if (!CarbonUtil
+        .hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(), Encoding.DICTIONARY)) {
+      columnDataChunk =
+          new VariableLengthDimensionDataChunk(getNoDictionaryDataChunk(dataPage), chunkAttributes);
+      chunkAttributes.setNoDictionary(true);
+    } else {
+      // to store fixed length column chunk values
+      columnDataChunk = new FixedLengthDimensionDataChunk(dataPage, chunkAttributes);
+    }
+    return columnDataChunk;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
new file mode 100644
index 0000000..af83514
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.v2;
+
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
+import org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.AbstractChunkReader;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.format.DataChunk2;
+import org.apache.carbondata.format.Encoding;
+
+/**
+ * Compressed dimension chunk reader class for version 2
+ */
+public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkReader {
+
+  /**
+   * dimension chunks offset
+   */
+  private List<Long> dimensionChunksOffset;
+
+  /**
+   * dimension chunks length
+   */
+  private List<Short> dimensionChunksLength;
+
+  /**
+   * Constructor to get minimum parameter to create instance of this class
+   *
+   * @param blockletInfo
+   * @param eachColumnValueSize
+   * @param filePath
+   */
+  public CompressedDimensionChunkFileBasedReaderV2(final BlockletInfo blockletInfo,
+      final int[] eachColumnValueSize, final String filePath) {
+    super(eachColumnValueSize, filePath);
+    this.dimensionChunksOffset = blockletInfo.getDimensionChunkOffsets();
+    this.dimensionChunksLength = blockletInfo.getDimensionChunksLength();
+
+  }
+
+  /**
+   * Below method will be used to read the chunk based on block indexes
+   * Reading logic of below method is:
+   * Except last column all the column chunk can be read in group
+   * if not last column then read data of all the column present in block index
+   * together then process it.
+   * For last column read is separately and process
+   *
+   * @param fileReader   file reader to read the blocks from file
+   * @param blockIndexes blocks range to be read
+   * @return dimension column chunks
+   */
+  @Override public DimensionColumnDataChunk[] readDimensionChunks(final FileHolder fileReader,
+      final int[][] blockIndexes) {
+    // read the column chunk based on block index and add
+    DimensionColumnDataChunk[] dataChunks =
+        new DimensionColumnDataChunk[dimensionChunksOffset.size()];
+    // if blocklet index is empty then return empry data chunk
+    if (blockIndexes.length == 0) {
+      return dataChunks;
+    }
+    DimensionColumnDataChunk[] groupChunk = null;
+    int index = 0;
+    // iterate till block indexes -1 as block index will be in sorted order, so to avoid
+    // the last column reading in group
+    for (int i = 0; i < blockIndexes.length - 1; i++) {
+      index = 0;
+      groupChunk = readDimensionChunksInGroup(fileReader, blockIndexes[i][0], blockIndexes[i][1]);
+      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+        dataChunks[j] = groupChunk[index++];
+      }
+    }
+    // check last index is present in block index, if it is present then read separately
+    if (blockIndexes[blockIndexes.length - 1][0] == dimensionChunksOffset.size() - 1) {
+      dataChunks[blockIndexes[blockIndexes.length - 1][0]] =
+          readDimensionChunk(fileReader, blockIndexes[blockIndexes.length - 1][0]);
+    }
+    // otherwise read the data in group
+    else {
+      groupChunk = readDimensionChunksInGroup(fileReader, blockIndexes[blockIndexes.length - 1][0],
+          blockIndexes[blockIndexes.length - 1][1]);
+      index = 0;
+      for (int j = blockIndexes[blockIndexes.length - 1][0];
+           j <= blockIndexes[blockIndexes.length - 1][1]; j++) {
+        dataChunks[j] = groupChunk[index++];
+      }
+    }
+    return dataChunks;
+  }
+
+  /**
+   * Below method will be used to read the chunk based on block index
+   *
+   * @param fileReader file reader to read the blocks from file
+   * @param blockIndex block to be read
+   * @return dimension column chunk
+   */
+  @Override public DimensionColumnDataChunk readDimensionChunk(FileHolder fileReader,
+      int blockIndex) {
+    byte[] dataPage = null;
+    int[] invertedIndexes = null;
+    int[] invertedIndexesReverse = null;
+    int[] rlePage = null;
+    DataChunk2 dimensionColumnChunk = null;
+    byte[] data = null;
+    int copySourcePoint = 0;
+    byte[] dimensionChunk = null;
+    if (dimensionChunksOffset.size() - 1 == blockIndex) {
+      dimensionChunk = fileReader.readByteArray(filePath, dimensionChunksOffset.get(blockIndex),
+          dimensionChunksLength.get(blockIndex));
+      dimensionColumnChunk = CarbonUtil.readDataChunk(dimensionChunk);
+      int totalDimensionDataLength =
+          dimensionColumnChunk.data_page_length + dimensionColumnChunk.rle_page_length
+              + dimensionColumnChunk.rowid_page_length;
+      data = fileReader.readByteArray(filePath,
+          dimensionChunksOffset.get(blockIndex) + dimensionChunksLength.get(blockIndex),
+          totalDimensionDataLength);
+    } else {
+      long currentDimensionOffset = dimensionChunksOffset.get(blockIndex);
+      data = fileReader.readByteArray(filePath, currentDimensionOffset,
+          (int) (dimensionChunksOffset.get(blockIndex + 1) - currentDimensionOffset));
+      dimensionChunk = new byte[dimensionChunksLength.get(blockIndex)];
+      System.arraycopy(data, copySourcePoint, dimensionChunk, 0,
+          dimensionChunksLength.get(blockIndex));
+      dimensionColumnChunk = CarbonUtil.readDataChunk(dimensionChunk);
+      copySourcePoint += dimensionChunksLength.get(blockIndex);
+    }
+
+    byte[] compressedDataPage = new byte[dimensionColumnChunk.data_page_length];
+    System.arraycopy(data, copySourcePoint, compressedDataPage, 0,
+        dimensionColumnChunk.data_page_length);
+    copySourcePoint += dimensionColumnChunk.data_page_length;
+    // first read the data and uncompressed it
+    dataPage = COMPRESSOR.unCompress(compressedDataPage);
+    // if row id block is present then read the row id chunk and uncompress it
+    if (hasEncoding(dimensionColumnChunk.encoders, Encoding.INVERTED_INDEX)) {
+      byte[] compressedIndexPage = new byte[dimensionColumnChunk.rowid_page_length];
+      System.arraycopy(data, copySourcePoint, compressedIndexPage, 0,
+          dimensionColumnChunk.rowid_page_length);
+      copySourcePoint += dimensionColumnChunk.rowid_page_length;
+      invertedIndexes = CarbonUtil
+          .getUnCompressColumnIndex(dimensionColumnChunk.rowid_page_length, compressedIndexPage,
+              numberComressor);
+      // get the reverse index
+      invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
+    }
+    // if rle is applied then read the rle block chunk and then uncompress
+    //then actual data based on rle block
+    if (hasEncoding(dimensionColumnChunk.encoders, Encoding.RLE)) {
+      // read and uncompress the rle block
+      byte[] compressedRLEPage = new byte[dimensionColumnChunk.rle_page_length];
+      System.arraycopy(data, copySourcePoint, compressedRLEPage, 0,
+          dimensionColumnChunk.rle_page_length);
+      rlePage = numberComressor.unCompress(compressedRLEPage);
+      // uncompress the data with rle indexes
+      dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[blockIndex]);
+      rlePage = null;
+    }
+    // fill chunk attributes
+    DimensionChunkAttributes chunkAttributes = new DimensionChunkAttributes();
+    chunkAttributes.setEachRowSize(eachColumnValueSize[blockIndex]);
+    chunkAttributes.setInvertedIndexes(invertedIndexes);
+    chunkAttributes.setInvertedIndexesReverse(invertedIndexesReverse);
+    DimensionColumnDataChunk columnDataChunk = null;
+
+    if (dimensionColumnChunk.isRowMajor()) {
+      // to store fixed length column chunk values
+      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, chunkAttributes);
+    }
+    // if no dictionary column then first create a no dictionary column chunk
+    // and set to data chunk instance
+    else if (!hasEncoding(dimensionColumnChunk.encoders, Encoding.DICTIONARY)) {
+      columnDataChunk =
+          new VariableLengthDimensionDataChunk(getNoDictionaryDataChunk(dataPage), chunkAttributes);
+      chunkAttributes.setNoDictionary(true);
+    } else {
+      // to store fixed length column chunk values
+      columnDataChunk = new FixedLengthDimensionDataChunk(dataPage, chunkAttributes);
+    }
+    return columnDataChunk;
+  }
+
+  /**
+   * Below method will be used to read the dimension chunks in group.
+   * This is to enhance the IO performance. Will read the data from start index
+   * to end index(including)
+   *
+   * @param fileReader      stream used for reading
+   * @param startBlockIndex start block index
+   * @param endBlockIndex   end block index
+   * @return dimension column chunk array
+   */
+  private DimensionColumnDataChunk[] readDimensionChunksInGroup(FileHolder fileReader,
+      int startBlockIndex, int endBlockIndex) {
+    long currentDimensionOffset = dimensionChunksOffset.get(startBlockIndex);
+    byte[] data = fileReader.readByteArray(filePath, currentDimensionOffset,
+        (int) (dimensionChunksOffset.get(endBlockIndex + 1) - currentDimensionOffset));
+    int copySourcePoint = 0;
+    // read the column chunk based on block index and add
+    DimensionColumnDataChunk[] dataChunks =
+        new DimensionColumnDataChunk[endBlockIndex - startBlockIndex + 1];
+    byte[] dataPage = null;
+    int[] invertedIndexes = null;
+    int[] invertedIndexesReverse = null;
+    int[] rlePage = null;
+    byte[] dimensionChunk = null;
+    DataChunk2 dimensionColumnChunk = null;
+    int index = 0;
+    for (int i = startBlockIndex; i <= endBlockIndex; i++) {
+      invertedIndexes = null;
+      invertedIndexesReverse = null;
+      dimensionChunk = new byte[dimensionChunksLength.get(i)];
+      System.arraycopy(data, copySourcePoint, dimensionChunk, 0, dimensionChunksLength.get(i));
+      dimensionColumnChunk = CarbonUtil.readDataChunk(dimensionChunk);
+      copySourcePoint += dimensionChunksLength.get(i);
+      byte[] compressedDataPage = new byte[dimensionColumnChunk.data_page_length];
+      System.arraycopy(data, copySourcePoint, compressedDataPage, 0,
+          dimensionColumnChunk.data_page_length);
+      copySourcePoint += dimensionColumnChunk.data_page_length;
+      // first read the data and uncompressed it
+      dataPage = COMPRESSOR.unCompress(compressedDataPage);
+      // if row id block is present then read the row id chunk and uncompress it
+      if (hasEncoding(dimensionColumnChunk.encoders, Encoding.INVERTED_INDEX)) {
+        byte[] compressedIndexPage = new byte[dimensionColumnChunk.rowid_page_length];
+        System.arraycopy(data, copySourcePoint, compressedIndexPage, 0,
+            dimensionColumnChunk.rowid_page_length);
+        copySourcePoint += dimensionColumnChunk.rowid_page_length;
+        invertedIndexes = CarbonUtil
+            .getUnCompressColumnIndex(dimensionColumnChunk.rowid_page_length, compressedIndexPage,
+                numberComressor);
+        // get the reverse index
+        invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
+      }
+      // if rle is applied then read the rle block chunk and then uncompress
+      //then actual data based on rle block
+      if (hasEncoding(dimensionColumnChunk.encoders, Encoding.RLE)) {
+        // read and uncompress the rle block
+        byte[] compressedRLEPage = new byte[dimensionColumnChunk.rle_page_length];
+        System.arraycopy(data, copySourcePoint, compressedRLEPage, 0,
+            dimensionColumnChunk.rle_page_length);
+        copySourcePoint += dimensionColumnChunk.rle_page_length;
+        rlePage = numberComressor.unCompress(compressedRLEPage);
+        // uncompress the data with rle indexes
+        dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[i]);
+        rlePage = null;
+      }
+      // fill chunk attributes
+      DimensionChunkAttributes chunkAttributes = new DimensionChunkAttributes();
+      chunkAttributes.setEachRowSize(eachColumnValueSize[i]);
+      chunkAttributes.setInvertedIndexes(invertedIndexes);
+      chunkAttributes.setInvertedIndexesReverse(invertedIndexesReverse);
+      DimensionColumnDataChunk columnDataChunk = null;
+      if (dimensionColumnChunk.isRowMajor()) {
+        // to store fixed length column chunk values
+        columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, chunkAttributes);
+      }
+      // if no dictionary column then first create a no dictionary column chunk
+      // and set to data chunk instance
+      else if (!hasEncoding(dimensionColumnChunk.encoders, Encoding.DICTIONARY)) {
+        columnDataChunk = new VariableLengthDimensionDataChunk(getNoDictionaryDataChunk(dataPage),
+            chunkAttributes);
+        chunkAttributes.setNoDictionary(true);
+      } else {
+        // to store fixed length column chunk values
+        columnDataChunk = new FixedLengthDimensionDataChunk(dataPage, chunkAttributes);
+      }
+      dataChunks[index++] = columnDataChunk;
+    }
+    return dataChunks;
+  }
+
+  /**
+   * Below method will be used to check whether particular encoding is present
+   * in the dimension or not
+   *
+   * @param encoding encoding to search
+   * @return if encoding is present in dimension
+   */
+  private boolean hasEncoding(List<Encoding> encodings, Encoding encoding) {
+    return encodings.contains(encoding);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/AbstractMeasureChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/AbstractMeasureChunkReader.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/AbstractMeasureChunkReader.java
index dc8771f..6c74379 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/AbstractMeasureChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/AbstractMeasureChunkReader.java
@@ -18,13 +18,7 @@
  */
 package org.apache.carbondata.core.carbon.datastore.chunk.reader.measure;
 
-import java.util.List;
-
 import org.apache.carbondata.core.carbon.datastore.chunk.reader.MeasureColumnChunkReader;
-import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
 
 /**
  * Measure block reader abstract class
@@ -32,44 +26,19 @@ import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonH
 public abstract class AbstractMeasureChunkReader implements MeasureColumnChunkReader {
 
   /**
-   * metadata which was to used to compress and uncompress the measure value
-   */
-  protected ValueCompressionModel compressionModel;
-
-  /**
    * file path from which blocks will be read
    */
   protected String filePath;
 
   /**
-   * measure chunk have the information about the metadata present in the file
-   */
-  protected List<DataChunk> measureColumnChunk;
-
-  /**
-   * type of valu comprssion model selected for each measure column
-   */
-  protected UnCompressValue[] values;
-
-  /**
    * Constructor to get minimum parameter to create instance of this class
    *
    * @param measureColumnChunk measure chunk metadata
    * @param compression        model metadata which was to used to compress and uncompress
    *                           the measure value
    * @param filePath           file from which data will be read
-   * @param isInMemory         in case of in memory it will read and holds the data and when
-   *                           query request will come it will uncompress and the data
    */
-  public AbstractMeasureChunkReader(List<DataChunk> measureColumnChunk,
-      ValueCompressionModel compressionModel, String filePath, boolean isInMemory) {
-    this.measureColumnChunk = measureColumnChunk;
-    this.compressionModel = compressionModel;
+  public AbstractMeasureChunkReader(String filePath) {
     this.filePath = filePath;
-    values =
-        new ValueCompressonHolder.UnCompressValue[compressionModel.getUnCompressValues().length];
-    for (int i = 0; i < values.length; i++) {
-      values[i] = compressionModel.getUnCompressValues()[i].getNew().getCompressorObject();
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReader.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReader.java
deleted file mode 100644
index 31c470d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReader.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.chunk.reader.measure;
-
-import java.util.List;
-
-import org.apache.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
-import org.apache.carbondata.core.datastorage.store.FileHolder;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
-import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
-
-/**
- * Compressed measure chunk reader
- */
-public class CompressedMeasureChunkFileBasedReader extends AbstractMeasureChunkReader {
-
-  /**
-   * Constructor to get minimum parameter to create instance of this class
-   *
-   * @param measureColumnChunk measure chunk metadata
-   * @param compression        model metadata which was to used to compress and uncompress
-   *                           the measure value
-   * @param filePath           file from which data will be read
-   */
-  public CompressedMeasureChunkFileBasedReader(List<DataChunk> measureColumnChunk,
-      ValueCompressionModel compressionModel, String filePath) {
-    super(measureColumnChunk, compressionModel, filePath, false);
-  }
-
-  /**
-   * Method to read the blocks data based on block indexes
-   *
-   * @param fileReader   file reader to read the blocks
-   * @param blockIndexes blocks to be read
-   * @return measure data chunks
-   */
-  @Override public MeasureColumnDataChunk[] readMeasureChunks(FileHolder fileReader,
-      int... blockIndexes) {
-    MeasureColumnDataChunk[] datChunk = new MeasureColumnDataChunk[values.length];
-    for (int i = 0; i < blockIndexes.length; i++) {
-      datChunk[blockIndexes[i]] = readMeasureChunk(fileReader, blockIndexes[i]);
-    }
-    return datChunk;
-  }
-
-  /**
-   * Method to read the blocks data based on block index
-   *
-   * @param fileReader file reader to read the blocks
-   * @param blockIndex block to be read
-   * @return measure data chunk
-   */
-  @Override public MeasureColumnDataChunk readMeasureChunk(FileHolder fileReader, int blockIndex) {
-    MeasureColumnDataChunk datChunk = new MeasureColumnDataChunk();
-    // create a new uncompressor
-    ValueCompressonHolder.UnCompressValue copy = values[blockIndex].getNew();
-    // read data from file and set to uncompressor
-    copy.setValue(fileReader
-        .readByteArray(filePath, measureColumnChunk.get(blockIndex).getDataPageOffset(),
-            measureColumnChunk.get(blockIndex).getDataPageLength()));
-    // get the data holder after uncompressing
-    CarbonReadDataHolder measureDataHolder =
-        copy.uncompress(compressionModel.getChangedDataType()[blockIndex])
-            .getValues(compressionModel.getDecimal()[blockIndex],
-                compressionModel.getMaxValue()[blockIndex]);
-    // set the data chunk
-    datChunk.setMeasureDataHolder(measureDataHolder);
-    // set the enun value indexes
-    datChunk
-        .setNullValueIndexHolder(measureColumnChunk.get(blockIndex).getNullValueIndexForColumn());
-    return datChunk;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
new file mode 100644
index 0000000..d989852
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.v1;
+
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.AbstractMeasureChunkReader;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+/**
+ * Compressed measure chunk reader
+ */
+public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChunkReader {
+
+  /**
+   * measure chunk have the information about the metadata present in the file
+   */
+  private final List<DataChunk> measureColumnChunks;
+
+  /**
+   * Constructor to get minimum parameter to create instance of this class
+   *
+   * @param blockletInfo BlockletInfo
+   * @param filePath     file from which data will be read
+   */
+  public CompressedMeasureChunkFileBasedReaderV1(final BlockletInfo blockletInfo,
+      final String filePath) {
+    super(filePath);
+    this.measureColumnChunks = blockletInfo.getMeasureColumnChunk();
+  }
+
+  /**
+   * Method to read the blocks data based on block indexes
+   *
+   * @param fileReader   file reader to read the blocks
+   * @param blockIndexes blocks to be read
+   * @return measure data chunks
+   */
+  @Override public MeasureColumnDataChunk[] readMeasureChunks(final FileHolder fileReader,
+      final int[][] blockIndexes) {
+    MeasureColumnDataChunk[] datChunk = new MeasureColumnDataChunk[measureColumnChunks.size()];
+    for (int i = 0; i < blockIndexes.length; i++) {
+      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+        datChunk[j] = readMeasureChunk(fileReader, j);
+      }
+    }
+    return datChunk;
+  }
+
+  /**
+   * Method to read the blocks data based on block index
+   *
+   * @param fileReader file reader to read the blocks
+   * @param blockIndex block to be read
+   * @return measure data chunk
+   */
+  @Override public MeasureColumnDataChunk readMeasureChunk(final FileHolder fileReader,
+      final int blockIndex) {
+    MeasureColumnDataChunk datChunk = new MeasureColumnDataChunk();
+    // create a new uncompressor
+    final ValueCompressionModel compressionModel = CarbonUtil
+        .getValueCompressionModel(measureColumnChunks.get(blockIndex).getValueEncoderMeta());
+    UnCompressValue values =
+        compressionModel.getUnCompressValues()[0].getNew().getCompressorObject();
+    // create a new uncompressor
+    // read data from file and set to uncompressor
+    // read data from file and set to uncompressor
+    values.setValue(fileReader
+        .readByteArray(filePath, measureColumnChunks.get(blockIndex).getDataPageOffset(),
+            measureColumnChunks.get(blockIndex).getDataPageLength()));
+    // get the data holder after uncompressing
+    CarbonReadDataHolder measureDataHolder =
+        values.uncompress(compressionModel.getChangedDataType()[0])
+            .getValues(compressionModel.getDecimal()[0], compressionModel.getMaxValue()[0]);
+    // set the data chunk
+    datChunk.setMeasureDataHolder(measureDataHolder);
+    // set the enun value indexes
+    datChunk
+        .setNullValueIndexHolder(measureColumnChunks.get(blockIndex).getNullValueIndexForColumn());
+    return datChunk;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
new file mode 100644
index 0000000..71b7c8f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.v2;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.AbstractMeasureChunkReader;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.PresenceMeta;
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression.SnappyByteCompression;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.metadata.ValueEncoderMeta;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.format.DataChunk2;
+
+/**
+ * Class to read the measure column data for version 2
+ */
+public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChunkReader {
+
+  /**
+   * measure column chunks offset
+   */
+  private List<Long> measureColumnChunkOffsets;
+
+  /**
+   * measure column chunks length
+   */
+  private List<Short> measureColumnChunkLength;
+
+  /**
+   * Constructor to get minimum parameter to create instance of this class
+   *
+   * @param blockletInfo BlockletInfo
+   * @param filePath     file from which data will be read
+   */
+  public CompressedMeasureChunkFileBasedReaderV2(final BlockletInfo blockletInfo,
+      final String filePath) {
+    super(filePath);
+    this.measureColumnChunkOffsets = blockletInfo.getMeasureChunkOffsets();
+    this.measureColumnChunkLength = blockletInfo.getMeasureChunksLength();
+  }
+
+  /**
+   * Below method will be used to convert the thrift presence meta to wrapper
+   * presence meta
+   *
+   * @param presentMetadataThrift
+   * @return wrapper presence meta
+   */
+  private static PresenceMeta getPresenceMeta(
+      org.apache.carbondata.format.PresenceMeta presentMetadataThrift) {
+    PresenceMeta presenceMeta = new PresenceMeta();
+    presenceMeta.setRepresentNullValues(presentMetadataThrift.isRepresents_presence());
+    presenceMeta.setBitSet(BitSet.valueOf(
+        SnappyByteCompression.INSTANCE.unCompress(presentMetadataThrift.getPresent_bit_stream())));
+    return presenceMeta;
+  }
+
+  /**
+   * Below method will be used to read the chunk based on block indexes
+   * Reading logic of below method is:
+   * Except last column all the column chunk can be read in group
+   * if not last column then read data of all the column present in block index
+   * together then process it.
+   * For last column read is separately and process
+   *
+   * @param fileReader   file reader to read the blocks from file
+   * @param blockIndexes blocks range to be read
+   * @return measure column chunks
+   */
+  public MeasureColumnDataChunk[] readMeasureChunks(FileHolder fileReader, int[][] blockIndexes) {
+    // read the column chunk based on block index and add
+    MeasureColumnDataChunk[] dataChunks =
+        new MeasureColumnDataChunk[measureColumnChunkOffsets.size()];
+    if (blockIndexes.length == 0) {
+      return dataChunks;
+    }
+    MeasureColumnDataChunk[] groupChunk = null;
+    int index = 0;
+    for (int i = 0; i < blockIndexes.length - 1; i++) {
+      index = 0;
+      groupChunk = readMeasureChunksInGroup(fileReader, blockIndexes[i][0], blockIndexes[i][1]);
+      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+        dataChunks[j] = groupChunk[index++];
+      }
+    }
+    if (blockIndexes[blockIndexes.length - 1][0] == measureColumnChunkOffsets.size() - 1) {
+      dataChunks[blockIndexes[blockIndexes.length - 1][0]] =
+          readMeasureChunk(fileReader, blockIndexes[blockIndexes.length - 1][0]);
+    } else {
+      groupChunk = readMeasureChunksInGroup(fileReader, blockIndexes[blockIndexes.length - 1][0],
+          blockIndexes[blockIndexes.length - 1][1]);
+      index = 0;
+      for (int j = blockIndexes[blockIndexes.length - 1][0];
+           j <= blockIndexes[blockIndexes.length - 1][1]; j++) {
+        dataChunks[j] = groupChunk[index++];
+      }
+    }
+    return dataChunks;
+  }
+
+  /**
+   * Method to read the blocks data based on block index
+   *
+   * @param fileReader file reader to read the blocks
+   * @param blockIndex block to be read
+   * @return measure data chunk
+   */
+  @Override public MeasureColumnDataChunk readMeasureChunk(FileHolder fileReader, int blockIndex) {
+    MeasureColumnDataChunk datChunk = new MeasureColumnDataChunk();
+    DataChunk2 measureColumnChunk = null;
+    byte[] measureDataChunk = null;
+    byte[] data = null;
+    byte[] dataPage = null;
+    if (measureColumnChunkOffsets.size() - 1 == blockIndex) {
+      measureDataChunk = fileReader
+          .readByteArray(filePath, measureColumnChunkOffsets.get(blockIndex),
+              measureColumnChunkLength.get(blockIndex));
+      measureColumnChunk = CarbonUtil.readDataChunk(measureDataChunk);
+      dataPage = fileReader.readByteArray(filePath,
+          measureColumnChunkOffsets.get(blockIndex) + measureColumnChunkLength.get(blockIndex),
+          measureColumnChunk.data_page_length);
+    } else {
+      long currentMeasureOffset = measureColumnChunkOffsets.get(blockIndex);
+      data = fileReader.readByteArray(filePath, currentMeasureOffset,
+          (int) (measureColumnChunkOffsets.get(blockIndex + 1) - currentMeasureOffset));
+      measureDataChunk = new byte[measureColumnChunkLength.get(blockIndex)];
+      System.arraycopy(data, 0, measureDataChunk, 0, measureColumnChunkLength.get(blockIndex));
+      measureColumnChunk = CarbonUtil.readDataChunk(measureDataChunk);
+      dataPage = new byte[measureColumnChunk.data_page_length];
+      System.arraycopy(data, measureColumnChunkLength.get(blockIndex), dataPage, 0,
+          measureColumnChunk.data_page_length);
+    }
+    List<ValueEncoderMeta> valueEncodeMeta = new ArrayList<>();
+    for (int i = 0; i < measureColumnChunk.getEncoder_meta().size(); i++) {
+      valueEncodeMeta.add(
+          CarbonUtil.deserializeEncoderMeta(measureColumnChunk.getEncoder_meta().get(i).array()));
+    }
+    ValueCompressionModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
+    UnCompressValue values =
+        compressionModel.getUnCompressValues()[0].getNew().getCompressorObject();
+    // create a new uncompressor
+    // read data from file and set to uncompressor
+    values.setValue(dataPage);
+    // get the data holder after uncompressing
+    CarbonReadDataHolder measureDataHolder =
+        values.uncompress(compressionModel.getChangedDataType()[0])
+            .getValues(compressionModel.getDecimal()[0], compressionModel.getMaxValue()[0]);
+    // set the data chunk
+    datChunk.setMeasureDataHolder(measureDataHolder);
+    // set the enun value indexes
+    datChunk.setNullValueIndexHolder(getPresenceMeta(measureColumnChunk.presence));
+    return datChunk;
+  }
+
+  /**
+   * Below method will be used to read the dimension chunks in group.
+   * This is to enhance the IO performance. Will read the data from start index
+   * to end index(including)
+   *
+   * @param fileReader      stream used for reading
+   * @param startBlockIndex start block index
+   * @param endBlockIndex   end block index
+   * @return measure column chunk array
+   */
+  private MeasureColumnDataChunk[] readMeasureChunksInGroup(FileHolder fileReader,
+      int startBlockIndex, int endBlockIndex) {
+    long currentMeasureOffset = measureColumnChunkOffsets.get(startBlockIndex);
+    byte[] data = fileReader.readByteArray(filePath, currentMeasureOffset,
+        (int) (measureColumnChunkOffsets.get(endBlockIndex + 1) - currentMeasureOffset));
+    MeasureColumnDataChunk[] dataChunks =
+        new MeasureColumnDataChunk[endBlockIndex - startBlockIndex + 1];
+    MeasureColumnDataChunk dataChunk = new MeasureColumnDataChunk();
+    int index = 0;
+    int copyPoint = 0;
+    byte[] measureDataChunk = null;
+    byte[] dataPage = null;
+    DataChunk2 measureColumnChunk = null;
+    for (int i = startBlockIndex; i <= endBlockIndex; i++) {
+      dataChunk = new MeasureColumnDataChunk();
+      measureDataChunk = new byte[measureColumnChunkLength.get(i)];
+      System.arraycopy(data, copyPoint, measureDataChunk, 0, measureColumnChunkLength.get(i));
+      measureColumnChunk = CarbonUtil.readDataChunk(measureDataChunk);
+      dataPage = new byte[measureColumnChunk.data_page_length];
+      copyPoint += measureColumnChunkLength.get(i);
+      System.arraycopy(data, copyPoint, dataPage, 0, measureColumnChunk.data_page_length);
+      copyPoint += measureColumnChunk.data_page_length;
+      List<ValueEncoderMeta> valueEncodeMeta = new ArrayList<>();
+      for (int j = 0; j < measureColumnChunk.getEncoder_meta().size(); j++) {
+        valueEncodeMeta.add(
+            CarbonUtil.deserializeEncoderMeta(measureColumnChunk.getEncoder_meta().get(j).array()));
+      }
+      ValueCompressionModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
+      UnCompressValue values =
+          compressionModel.getUnCompressValues()[0].getNew().getCompressorObject();
+      // create a new uncompressor
+      // read data from file and set to uncompressor
+      values.setValue(dataPage);
+      // get the data holder after uncompressing
+      CarbonReadDataHolder measureDataHolder =
+          values.uncompress(compressionModel.getChangedDataType()[0])
+              .getValues(compressionModel.getDecimal()[0], compressionModel.getMaxValue()[0]);
+      // set the data chunk
+      dataChunk.setMeasureDataHolder(measureDataHolder);
+      // set the enun value indexes
+      dataChunk.setNullValueIndexHolder(getPresenceMeta(measureColumnChunk.presence));
+      dataChunks[index++] = dataChunk;
+    }
+    return dataChunks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
index de476ad..bff3286 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
@@ -172,7 +172,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * @return dimension data chunks
    */
   @Override public DimensionColumnDataChunk[] getDimensionChunks(FileHolder fileReader,
-      int[] blockIndexes) {
+      int[][] blockIndexes) {
     // No required here as leaf which will will be use this class will implement its own get
     // dimension chunks
     return null;
@@ -200,7 +200,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * @return measure column data chunk
    */
   @Override public MeasureColumnDataChunk[] getMeasureChunks(FileHolder fileReader,
-      int[] blockIndexes) {
+      int[][] blockIndexes) {
     // No required here as leaf which will will be use this class will implement its own get
     // measure chunks
     return null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
index cfbe06d..c558763 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
@@ -169,7 +169,7 @@ public class BTreeNonLeafNode implements BTreeNode {
    * @return dimension data chunks
    */
   @Override public DimensionColumnDataChunk[] getDimensionChunks(FileHolder fileReader,
-      int[] blockIndexes) {
+      int[][] blockIndexes) {
 
     // operation of getting the dimension chunks is not supported as its a
     // non leaf node
@@ -204,7 +204,7 @@ public class BTreeNonLeafNode implements BTreeNode {
    * @return measure column data chunk
    */
   @Override public MeasureColumnDataChunk[] getMeasureChunks(FileHolder fileReader,
-      int[] blockIndexes) {
+      int[][] blockIndexes) {
     // operation of getting the measure chunk is not supported as its a non
     // leaf node
     // and in case of B+Tree data will be stored only in leaf node and

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
index 4293610..79ee008 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
@@ -21,14 +21,11 @@ package org.apache.carbondata.core.carbon.datastore.impl.btree;
 import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
 import org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
 import org.apache.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.CarbonDataReaderFactory;
 import org.apache.carbondata.core.carbon.datastore.chunk.reader.DimensionColumnChunkReader;
 import org.apache.carbondata.core.carbon.datastore.chunk.reader.MeasureColumnChunkReader;
-import org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.CompressedDimensionChunkFileBasedReader;
-import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.CompressedMeasureChunkFileBasedReader;
 import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
-import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * Leaf node class of a Blocklet btree
@@ -57,28 +54,25 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
   public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) {
     // get a lead node min max
     BlockletMinMaxIndex minMaxIndex =
-        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
-            .getBlockletIndex().getMinMaxIndex();
+        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex).getBlockletIndex()
+            .getMinMaxIndex();
     // max key of the columns
     maxKeyOfColumns = minMaxIndex.getMaxValues();
     // min keys of the columns
     minKeyOfColumns = minMaxIndex.getMinValues();
     // number of keys present in the leaf
-    numberOfKeys = builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
-        .getNumberOfRows();
+    numberOfKeys =
+        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex).getNumberOfRows();
     // create a instance of dimension chunk
-    dimensionChunksReader = new CompressedDimensionChunkFileBasedReader(
-        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
-            .getDimensionColumnChunk(), builderInfos.getDimensionColumnValueSize(),
-        builderInfos.getFooterList().get(0).getBlockInfo().getTableBlockInfo().getFilePath());
-    // get the value compression model which was used to compress the measure values
-    ValueCompressionModel valueCompressionModel = CarbonUtil.getValueCompressionModel(
-        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
-            .getMeasureColumnChunk());
+    dimensionChunksReader = CarbonDataReaderFactory.getInstance()
+        .getDimensionColumnChunkReader(builderInfos.getFooterList().get(0).getVersionId(),
+            builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex),
+            builderInfos.getDimensionColumnValueSize(),
+            builderInfos.getFooterList().get(0).getBlockInfo().getTableBlockInfo().getFilePath());
     // create a instance of measure column chunk reader
-    measureColumnChunkReader = new CompressedMeasureChunkFileBasedReader(
-        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
-            .getMeasureColumnChunk(), valueCompressionModel,
+    measureColumnChunkReader = CarbonDataReaderFactory.getInstance()
+        .getMeasureColumnChunkReader(builderInfos.getFooterList().get(0).getVersionId(),
+            builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex),
             builderInfos.getFooterList().get(0).getBlockInfo().getTableBlockInfo().getFilePath());
     this.nodeNumber = nodeNumber;
   }
@@ -91,7 +85,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * @return dimension data chunks
    */
   @Override public DimensionColumnDataChunk[] getDimensionChunks(FileHolder fileReader,
-      int[] blockIndexes) {
+      int[][] blockIndexes) {
     return dimensionChunksReader.readDimensionChunks(fileReader, blockIndexes);
   }
 
@@ -115,7 +109,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * @return measure column data chunk
    */
   @Override public MeasureColumnDataChunk[] getMeasureChunks(FileHolder fileReader,
-      int[] blockIndexes) {
+      int[][] blockIndexes) {
     return measureColumnChunkReader.readMeasureChunks(fileReader, blockIndexes);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
index b2c72aa..314f7e2 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
@@ -50,6 +50,14 @@ public class BlockletInfo implements Serializable {
    */
   private List<DataChunk> measureColumnChunk;
 
+  private List<Long> dimensionChunkOffsets;
+
+  private List<Short> dimensionChunksLength;
+
+  private List<Long> measureChunkOffsets;
+
+  private List<Short> measureChunksLength;
+
   /**
    * to store the index like min max and start and end key of each column of the blocklet
    */
@@ -111,4 +119,36 @@ public class BlockletInfo implements Serializable {
     this.blockletIndex = blockletIndex;
   }
 
+  public List<Long> getDimensionChunkOffsets() {
+    return dimensionChunkOffsets;
+  }
+
+  public void setDimensionChunkOffsets(List<Long> dimensionChunkOffsets) {
+    this.dimensionChunkOffsets = dimensionChunkOffsets;
+  }
+
+  public List<Short> getDimensionChunksLength() {
+    return dimensionChunksLength;
+  }
+
+  public void setDimensionChunksLength(List<Short> dimensionChunksLength) {
+    this.dimensionChunksLength = dimensionChunksLength;
+  }
+
+  public List<Long> getMeasureChunkOffsets() {
+    return measureChunkOffsets;
+  }
+
+  public void setMeasureChunkOffsets(List<Long> measureChunkOffsets) {
+    this.measureChunkOffsets = measureChunkOffsets;
+  }
+
+  public List<Short> getMeasureChunksLength() {
+    return measureChunksLength;
+  }
+
+  public void setMeasureChunksLength(List<Short> measureChunksLength) {
+    this.measureChunksLength = measureChunksLength;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
index d4741eb..be235ba 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
@@ -38,7 +38,7 @@ public class DataFileFooter implements Serializable {
   /**
    * version used for data compatibility
    */
-  private int versionId;
+  private short versionId;
 
   /**
    * total number of rows in this file
@@ -73,14 +73,14 @@ public class DataFileFooter implements Serializable {
   /**
    * @return the versionId
    */
-  public int getVersionId() {
+  public short getVersionId() {
     return versionId;
   }
 
   /**
    * @param versionId the versionId to set
    */
-  public void setVersionId(int versionId) {
+  public void setVersionId(short versionId) {
     this.versionId = versionId;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index e217d5d..a9e0442 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -893,7 +893,23 @@ public final class CarbonCommonConstants {
    * Default size of data load batch size.
    */
   public static final String DATA_LOAD_BATCH_SIZE_DEFAULT = "1000";
-
+  /**
+   * carbon data file version property
+   */
+  public static final String CARBON_DATA_FILE_VERSION = "carbon.data.file.version";
+  /**
+   * current data file version
+   */
+  public static final short CARBON_DATA_FILE_DEFAULT_VERSION = 2;
+  /**
+   * number of column data will read in IO operation
+   * during query execution
+   */
+  public static final short NUMBER_OF_COLUMN_READ_IN_IO = 10;
+  /**
+   * data file version header
+   */
+  public static final String CARBON_DATA_VERSION_HEADER = "CARBONDATAVERSION#";
   private CarbonCommonConstants() {
   }
 }



[3/5] incubator-carbondata git commit: Improve first time query performance

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
new file mode 100644
index 0000000..db9c9be
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
+import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.carbon.metadata.blocklet.SegmentInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.compressor.ChunkCompressorMeta;
+import org.apache.carbondata.core.carbon.metadata.blocklet.compressor.CompressionCodec;
+import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
+import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.PresenceMeta;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletBTreeIndex;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
+import org.apache.carbondata.core.carbon.metadata.blocklet.sort.SortState;
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.ValueEncoderMeta;
+import org.apache.carbondata.core.reader.CarbonIndexFileReader;
+import org.apache.carbondata.format.BlockIndex;
+
+/**
+ * Footer reader class
+ */
+public abstract class AbstractDataFileFooterConverter {
+
+  /**
+   * Below method will be used to convert the thrift presence meta to wrapper
+   * presence meta
+   *
+   * @param presentMetadataThrift
+   * @return wrapper presence meta
+   */
+  private static PresenceMeta getPresenceMeta(
+      org.apache.carbondata.format.PresenceMeta presentMetadataThrift) {
+    PresenceMeta presenceMeta = new PresenceMeta();
+    presenceMeta.setRepresentNullValues(presentMetadataThrift.isRepresents_presence());
+    presenceMeta.setBitSet(BitSet.valueOf(presentMetadataThrift.getPresent_bit_stream()));
+    return presenceMeta;
+  }
+
+  /**
+   * Below method will be used to get the index info from index file
+   *
+   * @param filePath           file path of the index file
+   * @param tableBlockInfoList table block index
+   * @return list of index info
+   * @throws IOException problem while reading the index file
+   */
+  public List<DataFileFooter> getIndexInfo(String filePath, List<TableBlockInfo> tableBlockInfoList)
+      throws IOException, CarbonUtilException {
+    CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
+    List<DataFileFooter> dataFileFooters = new ArrayList<DataFileFooter>();
+    try {
+      // open the reader
+      indexReader.openThriftReader(filePath);
+      // get the index header
+      org.apache.carbondata.format.IndexHeader readIndexHeader = indexReader.readIndexHeader();
+      List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
+      List<org.apache.carbondata.format.ColumnSchema> table_columns =
+          readIndexHeader.getTable_columns();
+      for (int i = 0; i < table_columns.size(); i++) {
+        columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
+      }
+      // get the segment info
+      SegmentInfo segmentInfo = getSegmentInfo(readIndexHeader.getSegment_info());
+      BlockletIndex blockletIndex = null;
+      int counter = 0;
+      DataFileFooter dataFileFooter = null;
+      // read the block info from file
+      while (indexReader.hasNext()) {
+        BlockIndex readBlockIndexInfo = indexReader.readBlockIndexInfo();
+        blockletIndex = getBlockletIndex(readBlockIndexInfo.getBlock_index());
+        dataFileFooter = new DataFileFooter();
+        TableBlockInfo tableBlockInfo = tableBlockInfoList.get(counter++);
+        tableBlockInfo.setBlockOffset(readBlockIndexInfo.getOffset());
+        tableBlockInfo.setVersion((short) readIndexHeader.getVersion());
+        int blockletSize = getBlockletSize(readBlockIndexInfo);
+        tableBlockInfo.getBlockletInfos().setNoOfBlockLets(blockletSize);
+        dataFileFooter.setBlockletIndex(blockletIndex);
+        dataFileFooter.setColumnInTable(columnSchemaList);
+        dataFileFooter.setNumberOfRows(readBlockIndexInfo.getNum_rows());
+        dataFileFooter.setBlockInfo(new BlockInfo(tableBlockInfo));
+        dataFileFooter.setSegmentInfo(segmentInfo);
+        dataFileFooters.add(dataFileFooter);
+      }
+    } finally {
+      indexReader.closeThriftReader();
+    }
+    return dataFileFooters;
+  }
+
+  /**
+   * the methods returns the number of blocklets in a block
+   *
+   * @param readBlockIndexInfo
+   * @return
+   */
+  protected int getBlockletSize(BlockIndex readBlockIndexInfo) {
+    long num_rows = readBlockIndexInfo.getNum_rows();
+    int blockletSize = Integer.parseInt(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
+            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
+    int remainder = (int) (num_rows % blockletSize);
+    int noOfBlockLet = (int) (num_rows / blockletSize);
+    // there could be some blocklets which will not
+    // contain the total records equal to the blockletSize
+    if (remainder > 0) {
+      noOfBlockLet = noOfBlockLet + 1;
+    }
+    return noOfBlockLet;
+  }
+
+  /**
+   * Below method will be used to convert thrift file meta to wrapper file meta
+   */
+  public abstract DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
+      throws IOException;
+
+  /**
+   * Below method will be used to get blocklet index for data file meta
+   *
+   * @param blockletIndexList
+   * @return blocklet index
+   */
+  protected BlockletIndex getBlockletIndexForDataFileFooter(List<BlockletIndex> blockletIndexList) {
+    BlockletIndex blockletIndex = new BlockletIndex();
+    BlockletBTreeIndex blockletBTreeIndex = new BlockletBTreeIndex();
+    blockletBTreeIndex.setStartKey(blockletIndexList.get(0).getBtreeIndex().getStartKey());
+    blockletBTreeIndex
+        .setEndKey(blockletIndexList.get(blockletIndexList.size() - 1).getBtreeIndex().getEndKey());
+    blockletIndex.setBtreeIndex(blockletBTreeIndex);
+    byte[][] currentMinValue = blockletIndexList.get(0).getMinMaxIndex().getMinValues().clone();
+    byte[][] currentMaxValue = blockletIndexList.get(0).getMinMaxIndex().getMaxValues().clone();
+    byte[][] minValue = null;
+    byte[][] maxValue = null;
+    for (int i = 1; i < blockletIndexList.size(); i++) {
+      minValue = blockletIndexList.get(i).getMinMaxIndex().getMinValues();
+      maxValue = blockletIndexList.get(i).getMinMaxIndex().getMaxValues();
+      for (int j = 0; j < maxValue.length; j++) {
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMinValue[j], minValue[j]) > 0) {
+          currentMinValue[j] = minValue[j].clone();
+        }
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMaxValue[j], maxValue[j]) < 0) {
+          currentMaxValue[j] = maxValue[j].clone();
+        }
+      }
+    }
+
+    BlockletMinMaxIndex minMax = new BlockletMinMaxIndex();
+    minMax.setMaxValues(currentMaxValue);
+    minMax.setMinValues(currentMinValue);
+    blockletIndex.setMinMaxIndex(minMax);
+    return blockletIndex;
+  }
+
+  protected ColumnSchema thriftColumnSchmeaToWrapperColumnSchema(
+      org.apache.carbondata.format.ColumnSchema externalColumnSchema) {
+    ColumnSchema wrapperColumnSchema = new ColumnSchema();
+    wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
+    wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
+    wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
+    wrapperColumnSchema
+        .setDataType(thriftDataTyopeToWrapperDataType(externalColumnSchema.data_type));
+    wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
+    List<Encoding> encoders = new ArrayList<Encoding>();
+    for (org.apache.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {
+      encoders.add(fromExternalToWrapperEncoding(encoder));
+    }
+    wrapperColumnSchema.setEncodingList(encoders);
+    wrapperColumnSchema.setNumberOfChild(externalColumnSchema.getNum_child());
+    wrapperColumnSchema.setPrecision(externalColumnSchema.getPrecision());
+    wrapperColumnSchema.setColumnGroup(externalColumnSchema.getColumn_group_id());
+    wrapperColumnSchema.setScale(externalColumnSchema.getScale());
+    wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
+    wrapperColumnSchema.setAggregateFunction(externalColumnSchema.getAggregate_function());
+    return wrapperColumnSchema;
+  }
+
+  /**
+   * Below method is convert the thrift encoding to wrapper encoding
+   *
+   * @param encoderThrift thrift encoding
+   * @return wrapper encoding
+   */
+  protected Encoding fromExternalToWrapperEncoding(
+      org.apache.carbondata.format.Encoding encoderThrift) {
+    switch (encoderThrift) {
+      case DICTIONARY:
+        return Encoding.DICTIONARY;
+      case DELTA:
+        return Encoding.DELTA;
+      case RLE:
+        return Encoding.RLE;
+      case INVERTED_INDEX:
+        return Encoding.INVERTED_INDEX;
+      case BIT_PACKED:
+        return Encoding.BIT_PACKED;
+      case DIRECT_DICTIONARY:
+        return Encoding.DIRECT_DICTIONARY;
+      default:
+        throw new IllegalArgumentException(encoderThrift.toString() + " is not supported");
+    }
+  }
+
+  /**
+   * Below method will be used to convert the thrift compression to wrapper
+   * compression codec
+   *
+   * @param compressionCodecThrift
+   * @return wrapper compression codec
+   */
+  protected CompressionCodec getCompressionCodec(
+      org.apache.carbondata.format.CompressionCodec compressionCodecThrift) {
+    switch (compressionCodecThrift) {
+      case SNAPPY:
+        return CompressionCodec.SNAPPY;
+      default:
+        return CompressionCodec.SNAPPY;
+    }
+  }
+
+  /**
+   * Below method will be used to convert thrift segment object to wrapper
+   * segment object
+   *
+   * @param segmentInfo thrift segment info object
+   * @return wrapper segment info object
+   */
+  protected SegmentInfo getSegmentInfo(org.apache.carbondata.format.SegmentInfo segmentInfo) {
+    SegmentInfo info = new SegmentInfo();
+    int[] cardinality = new int[segmentInfo.getColumn_cardinalities().size()];
+    for (int i = 0; i < cardinality.length; i++) {
+      cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
+    }
+    info.setColumnCardinality(cardinality);
+    info.setNumberOfColumns(segmentInfo.getNum_cols());
+    return info;
+  }
+
+  /**
+   * Below method will be used to convert the blocklet index of thrift to
+   * wrapper
+   *
+   * @param blockletIndexThrift
+   * @return blocklet index wrapper
+   */
+  protected BlockletIndex getBlockletIndex(
+      org.apache.carbondata.format.BlockletIndex blockletIndexThrift) {
+    org.apache.carbondata.format.BlockletBTreeIndex btreeIndex =
+        blockletIndexThrift.getB_tree_index();
+    org.apache.carbondata.format.BlockletMinMaxIndex minMaxIndex =
+        blockletIndexThrift.getMin_max_index();
+    return new BlockletIndex(
+        new BlockletBTreeIndex(btreeIndex.getStart_key(), btreeIndex.getEnd_key()),
+        new BlockletMinMaxIndex(minMaxIndex.getMin_values(), minMaxIndex.getMax_values()));
+  }
+
+  /**
+   * Below method will be used to convert the thrift compression meta to
+   * wrapper chunk compression meta
+   *
+   * @param chunkCompressionMetaThrift
+   * @return chunkCompressionMetaWrapper
+   */
+  protected ChunkCompressorMeta getChunkCompressionMeta(
+      org.apache.carbondata.format.ChunkCompressionMeta chunkCompressionMetaThrift) {
+    ChunkCompressorMeta compressorMeta = new ChunkCompressorMeta();
+    compressorMeta
+        .setCompressor(getCompressionCodec(chunkCompressionMetaThrift.getCompression_codec()));
+    compressorMeta.setCompressedSize(chunkCompressionMetaThrift.getTotal_compressed_size());
+    compressorMeta.setUncompressedSize(chunkCompressionMetaThrift.getTotal_uncompressed_size());
+    return compressorMeta;
+  }
+
+  /**
+   * Below method will be used to convert the thrift data type to wrapper data
+   * type
+   *
+   * @param dataTypeThrift
+   * @return dataType wrapper
+   */
+  protected DataType thriftDataTyopeToWrapperDataType(
+      org.apache.carbondata.format.DataType dataTypeThrift) {
+    switch (dataTypeThrift) {
+      case STRING:
+        return DataType.STRING;
+      case SHORT:
+        return DataType.SHORT;
+      case INT:
+        return DataType.INT;
+      case LONG:
+        return DataType.LONG;
+      case DOUBLE:
+        return DataType.DOUBLE;
+      case DECIMAL:
+        return DataType.DECIMAL;
+      case TIMESTAMP:
+        return DataType.TIMESTAMP;
+      case ARRAY:
+        return DataType.ARRAY;
+      case STRUCT:
+        return DataType.STRUCT;
+      default:
+        return DataType.STRING;
+    }
+  }
+
+  /**
+   * Below method will be used to convert the thrift object to wrapper object
+   *
+   * @param sortStateThrift
+   * @return wrapper sort state object
+   */
+  protected SortState getSortState(org.apache.carbondata.format.SortState sortStateThrift) {
+    if (sortStateThrift == org.apache.carbondata.format.SortState.SORT_EXPLICIT) {
+      return SortState.SORT_EXPLICT;
+    } else if (sortStateThrift == org.apache.carbondata.format.SortState.SORT_NATIVE) {
+      return SortState.SORT_NATIVE;
+    } else {
+      return SortState.SORT_NONE;
+    }
+  }
+
+  /**
+   * Below method will be used to convert the thrift data chunk to wrapper
+   * data chunk
+   *
+   * @param datachunkThrift
+   * @return wrapper data chunk
+   */
+  protected DataChunk getDataChunk(org.apache.carbondata.format.DataChunk datachunkThrift,
+      boolean isPresenceMetaPresent) {
+    DataChunk dataChunk = new DataChunk();
+    dataChunk.setColumnUniqueIdList(datachunkThrift.getColumn_ids());
+    dataChunk.setDataPageLength(datachunkThrift.getData_page_length());
+    dataChunk.setDataPageOffset(datachunkThrift.getData_page_offset());
+    if (isPresenceMetaPresent) {
+      dataChunk.setNullValueIndexForColumn(getPresenceMeta(datachunkThrift.getPresence()));
+    }
+    dataChunk.setRlePageLength(datachunkThrift.getRle_page_length());
+    dataChunk.setRlePageOffset(datachunkThrift.getRle_page_offset());
+    dataChunk.setRowMajor(datachunkThrift.isRowMajor());
+    dataChunk.setRowIdPageLength(datachunkThrift.getRowid_page_length());
+    dataChunk.setRowIdPageOffset(datachunkThrift.getRowid_page_offset());
+    dataChunk.setSortState(getSortState(datachunkThrift.getSort_state()));
+    dataChunk.setChunkCompressionMeta(getChunkCompressionMeta(datachunkThrift.getChunk_meta()));
+    List<Encoding> encodingList = new ArrayList<Encoding>(datachunkThrift.getEncoders().size());
+    for (int i = 0; i < datachunkThrift.getEncoders().size(); i++) {
+      encodingList.add(fromExternalToWrapperEncoding(datachunkThrift.getEncoders().get(i)));
+    }
+    dataChunk.setEncoderList(encodingList);
+    if (encodingList.contains(Encoding.DELTA)) {
+      List<ByteBuffer> thriftEncoderMeta = datachunkThrift.getEncoder_meta();
+      List<ValueEncoderMeta> encodeMetaList =
+          new ArrayList<ValueEncoderMeta>(thriftEncoderMeta.size());
+      for (int i = 0; i < thriftEncoderMeta.size(); i++) {
+        encodeMetaList.add(CarbonUtil.deserializeEncoderMeta(thriftEncoderMeta.get(i).array()));
+      }
+      dataChunk.setValueEncoderMeta(encodeMetaList);
+    }
+    return dataChunk;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/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 de0ea44..4f8a435 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
@@ -33,6 +33,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.carbon.metadata.index.BlockIndexInfo;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression.SnappyByteCompression;
 import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.apache.carbondata.core.metadata.BlockletInfoColumnar;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
@@ -40,11 +41,13 @@ import org.apache.carbondata.format.BlockIndex;
 import org.apache.carbondata.format.BlockletBTreeIndex;
 import org.apache.carbondata.format.BlockletIndex;
 import org.apache.carbondata.format.BlockletInfo;
+import org.apache.carbondata.format.BlockletInfo2;
 import org.apache.carbondata.format.BlockletMinMaxIndex;
 import org.apache.carbondata.format.ChunkCompressionMeta;
 import org.apache.carbondata.format.ColumnSchema;
 import org.apache.carbondata.format.CompressionCodec;
 import org.apache.carbondata.format.DataChunk;
+import org.apache.carbondata.format.DataChunk2;
 import org.apache.carbondata.format.Encoding;
 import org.apache.carbondata.format.FileFooter;
 import org.apache.carbondata.format.IndexHeader;
@@ -72,22 +75,60 @@ public class CarbonMetadataUtil {
    * @return FileFooter
    */
   public static FileFooter convertFileFooter(List<BlockletInfoColumnar> infoList, int numCols,
-      int[] cardinalities, List<ColumnSchema> columnSchemaList,
-      SegmentProperties segmentProperties) throws IOException {
+      int[] cardinalities, List<ColumnSchema> columnSchemaList, SegmentProperties segmentProperties)
+      throws IOException {
+    FileFooter footer = getFileFooter(infoList, cardinalities, columnSchemaList);
+    for (BlockletInfoColumnar info : infoList) {
+      footer.addToBlocklet_info_list(getBlockletInfo(info, columnSchemaList, segmentProperties));
+    }
+    return footer;
+  }
 
+  /**
+   * Below method will be used to get the file footer object
+   *
+   * @param infoList         blocklet info
+   * @param cardinalities    cardinlaity of dimension columns
+   * @param columnSchemaList column schema list
+   * @return file footer
+   */
+  private static FileFooter getFileFooter(List<BlockletInfoColumnar> infoList, int[] cardinalities,
+      List<ColumnSchema> columnSchemaList) {
     SegmentInfo segmentInfo = new SegmentInfo();
     segmentInfo.setNum_cols(columnSchemaList.size());
     segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(cardinalities));
-
+    short version = Short.parseShort(
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION));
     FileFooter footer = new FileFooter();
+    footer.setVersion(version);
     footer.setNum_rows(getTotalNumberOfRows(infoList));
     footer.setSegment_info(segmentInfo);
+    footer.setTable_columns(columnSchemaList);
     for (BlockletInfoColumnar info : infoList) {
       footer.addToBlocklet_index_list(getBlockletIndex(info));
     }
-    footer.setTable_columns(columnSchemaList);
+    return footer;
+  }
+
+  /**
+   * Below method will be used to get the file footer object for
+   *
+   * @param infoList         blocklet info
+   * @param cardinalities    cardinality of each column
+   * @param columnSchemaList column schema list
+   * @param dataChunksOffset data chunks offsets
+   * @param dataChunksLength data chunks length
+   * @return filefooter thrift object
+   */
+  public static FileFooter convertFilterFooter2(List<BlockletInfoColumnar> infoList,
+      int[] cardinalities, List<ColumnSchema> columnSchemaList, List<List<Long>> dataChunksOffset,
+      List<List<Short>> dataChunksLength) {
+    FileFooter footer = getFileFooter(infoList, cardinalities, columnSchemaList);
+    int index = 0;
     for (BlockletInfoColumnar info : infoList) {
-      footer.addToBlocklet_info_list(getBlockletInfo(info, columnSchemaList, segmentProperties));
+      footer.addToBlocklet_info_list2(
+          getBlockletInfo2(info, dataChunksOffset.get(index), dataChunksLength.get(index)));
+      index++;
     }
     return footer;
   }
@@ -142,15 +183,31 @@ public class CarbonMetadataUtil {
     return blockletIndex;
   }
 
+  /**
+   * Below method will be used to get the blocklet info object for
+   * data version 2 file
+   *
+   * @param blockletInfoColumnar blocklet info
+   * @param dataChunkOffsets     data chunks offsets
+   * @param dataChunksLength     data chunks length
+   * @return blocklet info version 2
+   */
+  private static BlockletInfo2 getBlockletInfo2(BlockletInfoColumnar blockletInfoColumnar,
+      List<Long> dataChunkOffsets, List<Short> dataChunksLength) {
+    BlockletInfo2 blockletInfo = new BlockletInfo2();
+    blockletInfo.setNum_rows(blockletInfoColumnar.getNumberOfKeys());
+    blockletInfo.setColumn_data_chunks_length(dataChunksLength);
+    blockletInfo.setColumn_data_chunks_offsets(dataChunkOffsets);
+    return blockletInfo;
+  }
+
   private static BlockletInfo getBlockletInfo(BlockletInfoColumnar blockletInfoColumnar,
-      List<ColumnSchema> columnSchenma,
-      SegmentProperties segmentProperties) throws IOException {
+      List<ColumnSchema> columnSchenma, SegmentProperties segmentProperties) throws IOException {
 
     BlockletInfo blockletInfo = new BlockletInfo();
     blockletInfo.setNum_rows(blockletInfoColumnar.getNumberOfKeys());
 
     List<DataChunk> colDataChunks = new ArrayList<DataChunk>();
-    blockletInfoColumnar.getKeyLengths();
     int j = 0;
     int aggregateIndex = 0;
     boolean[] isSortedKeyColumn = blockletInfoColumnar.getIsSortedKeyColumn();
@@ -419,6 +476,9 @@ public class CarbonMetadataUtil {
     segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(columnCardinality));
     // create index header object
     IndexHeader indexHeader = new IndexHeader();
+    short version = Short.parseShort(
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION));
+    indexHeader.setVersion(version);
     // set the segment info
     indexHeader.setSegment_info(segmentInfo);
     // set the column names
@@ -440,11 +500,91 @@ public class CarbonMetadataUtil {
     for (BlockIndexInfo blockIndexInfo : blockIndexInfoList) {
       blockIndex = new BlockIndex();
       blockIndex.setNum_rows(blockIndexInfo.getNumberOfRows());
-      blockIndex.setOffset(blockIndexInfo.getNumberOfRows());
+      blockIndex.setOffset(blockIndexInfo.getOffset());
       blockIndex.setFile_name(blockIndexInfo.getFileName());
       blockIndex.setBlock_index(getBlockletIndex(blockIndexInfo.getBlockletIndex()));
       thriftBlockIndexList.add(blockIndex);
     }
     return thriftBlockIndexList;
   }
+
+  /**
+   * Below method will be used to get the data chunk object for all the
+   * columns
+   *
+   * @param blockletInfoColumnar blocklet info
+   * @param columnSchenma        list of columns
+   * @param segmentProperties    segment properties
+   * @return list of data chunks
+   * @throws IOException
+   */
+  public static List<DataChunk2> getDatachunk2(BlockletInfoColumnar blockletInfoColumnar,
+      List<ColumnSchema> columnSchenma, SegmentProperties segmentProperties) throws IOException {
+    List<DataChunk2> colDataChunks = new ArrayList<DataChunk2>();
+    int rowIdIndex = 0;
+    int aggregateIndex = 0;
+    boolean[] isSortedKeyColumn = blockletInfoColumnar.getIsSortedKeyColumn();
+    boolean[] aggKeyBlock = blockletInfoColumnar.getAggKeyBlock();
+    boolean[] colGrpblock = blockletInfoColumnar.getColGrpBlocks();
+    for (int i = 0; i < blockletInfoColumnar.getKeyLengths().length; i++) {
+      DataChunk2 dataChunk = new DataChunk2();
+      dataChunk.setChunk_meta(getChunkCompressionMeta());
+      List<Encoding> encodings = new ArrayList<Encoding>();
+      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(colGrpblock[i]);
+      //TODO : Once schema PR is merged and information needs to be passed here.
+      dataChunk.setData_page_length(blockletInfoColumnar.getKeyLengths()[i]);
+      if (aggKeyBlock[i]) {
+        dataChunk.setRle_page_length(blockletInfoColumnar.getDataIndexMapLength()[aggregateIndex]);
+        encodings.add(Encoding.RLE);
+        aggregateIndex++;
+      }
+      dataChunk
+          .setSort_state(isSortedKeyColumn[i] ? SortState.SORT_EXPLICIT : SortState.SORT_NATIVE);
+
+      if (!isSortedKeyColumn[i]) {
+        dataChunk.setRowid_page_length(blockletInfoColumnar.getKeyBlockIndexLength()[rowIdIndex]);
+        encodings.add(Encoding.INVERTED_INDEX);
+        rowIdIndex++;
+      }
+
+      //TODO : Right now the encodings are happening at runtime. change as per this encoders.
+      dataChunk.setEncoders(encodings);
+
+      colDataChunks.add(dataChunk);
+    }
+
+    for (int i = 0; i < blockletInfoColumnar.getMeasureLength().length; i++) {
+      DataChunk2 dataChunk = new DataChunk2();
+      dataChunk.setChunk_meta(getChunkCompressionMeta());
+      dataChunk.setRowMajor(false);
+      //TODO : Once schema PR is merged and information needs to be passed here.
+      dataChunk.setData_page_length(blockletInfoColumnar.getMeasureLength()[i]);
+      //TODO : Right now the encodings are happening at runtime. change as per this encoders.
+      List<Encoding> encodings = new ArrayList<Encoding>();
+      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(SnappyByteCompression.INSTANCE
+          .compress(blockletInfoColumnar.getMeasureNullValueIndex()[i].toByteArray()));
+      dataChunk.setPresence(presenceMeta);
+      //TODO : PresenceMeta needs to be implemented and set here
+      // dataChunk.setPresence(new PresenceMeta());
+      //TODO : Need to write ValueCompression meta here.
+      List<ByteBuffer> encoderMetaList = new ArrayList<ByteBuffer>();
+      encoderMetaList.add(ByteBuffer.wrap(serializeEncoderMeta(
+          createValueEncoderMeta(blockletInfoColumnar.getCompressionModel(), i))));
+      dataChunk.setEncoder_meta(encoderMetaList);
+      colDataChunks.add(dataChunk);
+    }
+    return colDataChunks;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/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 b856928..adb0e6a 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
@@ -83,6 +83,7 @@ public final class CarbonProperties {
     validateHighCardinalityIdentify();
     validateHighCardinalityThreshold();
     validateHighCardinalityInRowCountPercentage();
+    validateCarbonDataFileVersion();
   }
 
   private void validateBadRecordsLocation() {
@@ -106,15 +107,15 @@ public final class CarbonProperties {
       if (blockletSize < CarbonCommonConstants.BLOCKLET_SIZE_MIN_VAL
           || blockletSize > CarbonCommonConstants.BLOCKLET_SIZE_MAX_VAL) {
         LOGGER.info("The blocklet size value \"" + blockletSizeStr
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
+            + "\" is invalid. Using the default value \""
+            + CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
         carbonProperties.setProperty(CarbonCommonConstants.BLOCKLET_SIZE,
             CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
       }
     } catch (NumberFormatException e) {
       LOGGER.info("The blocklet size value \"" + blockletSizeStr
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
+          + "\" is invalid. Using the default value \""
+          + CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
       carbonProperties.setProperty(CarbonCommonConstants.BLOCKLET_SIZE,
           CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
     }
@@ -131,16 +132,16 @@ public final class CarbonProperties {
 
       if (numCores < CarbonCommonConstants.NUM_CORES_MIN_VAL
           || numCores > CarbonCommonConstants.NUM_CORES_MAX_VAL) {
-        LOGGER.info("The num Cores  value \"" + numCoresStr
-            + "\" is invalid. Using the default value \""
-            + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
+        LOGGER.info(
+            "The num Cores  value \"" + numCoresStr + "\" is invalid. Using the default value \""
+                + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
         carbonProperties.setProperty(CarbonCommonConstants.NUM_CORES,
             CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
       }
     } catch (NumberFormatException e) {
-      LOGGER.info("The num Cores  value \"" + numCoresStr
-          + "\" is invalid. Using the default value \""
-          + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
+      LOGGER.info(
+          "The num Cores  value \"" + numCoresStr + "\" is invalid. Using the default value \""
+              + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
       carbonProperties.setProperty(CarbonCommonConstants.NUM_CORES,
           CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
     }
@@ -150,9 +151,8 @@ public final class CarbonProperties {
    * This method validates the number cores specified for mdk block sort
    */
   private void validateNumCoresBlockSort() {
-    String numCoresStr = carbonProperties
-        .getProperty(CarbonCommonConstants.NUM_CORES_BLOCK_SORT,
-            CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL);
+    String numCoresStr = carbonProperties.getProperty(CarbonCommonConstants.NUM_CORES_BLOCK_SORT,
+        CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL);
     try {
       int numCores = Integer.parseInt(numCoresStr);
 
@@ -183,25 +183,25 @@ public final class CarbonProperties {
       int sortSize = Integer.parseInt(sortSizeStr);
 
       if (sortSize < CarbonCommonConstants.SORT_SIZE_MIN_VAL) {
-        LOGGER.info("The batch size value \"" + sortSizeStr
-            + "\" is invalid. Using the default value \""
-            + CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
+        LOGGER.info(
+            "The batch size value \"" + sortSizeStr + "\" is invalid. Using the default value \""
+                + CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
         carbonProperties.setProperty(CarbonCommonConstants.SORT_SIZE,
             CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
       }
     } catch (NumberFormatException e) {
-      LOGGER.info("The batch size value \"" + sortSizeStr
-          + "\" is invalid. Using the default value \""
-          + CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
+      LOGGER.info(
+          "The batch size value \"" + sortSizeStr + "\" is invalid. Using the default value \""
+              + CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
       carbonProperties.setProperty(CarbonCommonConstants.SORT_SIZE,
           CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
     }
   }
 
   private void validateHighCardinalityIdentify() {
-    String highcardIdentifyStr = carbonProperties.getProperty(
-        CarbonCommonConstants.HIGH_CARDINALITY_IDENTIFY_ENABLE,
-        CarbonCommonConstants.HIGH_CARDINALITY_IDENTIFY_ENABLE_DEFAULT);
+    String highcardIdentifyStr = carbonProperties
+        .getProperty(CarbonCommonConstants.HIGH_CARDINALITY_IDENTIFY_ENABLE,
+            CarbonCommonConstants.HIGH_CARDINALITY_IDENTIFY_ENABLE_DEFAULT);
     try {
       Boolean.parseBoolean(highcardIdentifyStr);
     } catch (NumberFormatException e) {
@@ -214,12 +214,12 @@ public final class CarbonProperties {
   }
 
   private void validateHighCardinalityThreshold() {
-    String highcardThresholdStr = carbonProperties.getProperty(
-        CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD,
-        CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_DEFAULT);
+    String highcardThresholdStr = carbonProperties
+        .getProperty(CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD,
+            CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_DEFAULT);
     try {
       int highcardThreshold = Integer.parseInt(highcardThresholdStr);
-      if(highcardThreshold < CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_MIN){
+      if (highcardThreshold < CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_MIN) {
         LOGGER.info("The high cardinality threshold value \"" + highcardThresholdStr
             + "\" is invalid. Using the min value \""
             + CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_MIN);
@@ -236,22 +236,22 @@ public final class CarbonProperties {
   }
 
   private void validateHighCardinalityInRowCountPercentage() {
-    String highcardPercentageStr = carbonProperties.getProperty(
-        CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE,
-        CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE_DEFAULT);
+    String highcardPercentageStr = carbonProperties
+        .getProperty(CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE,
+            CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE_DEFAULT);
     try {
       double highcardPercentage = Double.parseDouble(highcardPercentageStr);
-      if(highcardPercentage <= 0){
-        LOGGER.info("The percentage of high cardinality in row count value \""
-            + highcardPercentageStr + "\" is invalid. Using the default value \""
-            + CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE_DEFAULT);
-        carbonProperties.setProperty(
-            CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE,
+      if (highcardPercentage <= 0) {
+        LOGGER.info(
+            "The percentage of high cardinality in row count value \"" + highcardPercentageStr
+                + "\" is invalid. Using the default value \""
+                + CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE_DEFAULT);
+        carbonProperties.setProperty(CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE,
             CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE_DEFAULT);
       }
     } catch (NumberFormatException e) {
-      LOGGER.info("The percentage of high cardinality in row count value \""
-          + highcardPercentageStr + "\" is invalid. Using the default value \""
+      LOGGER.info("The percentage of high cardinality in row count value \"" + highcardPercentageStr
+          + "\" is invalid. Using the default value \""
           + CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE_DEFAULT);
       carbonProperties.setProperty(CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE,
           CarbonCommonConstants.HIGH_CARDINALITY_IN_ROW_COUNT_PERCENTAGE_DEFAULT);
@@ -259,6 +259,34 @@ public final class CarbonProperties {
   }
 
   /**
+   * Below method will be used to validate the data file version parameter
+   * if parameter is invalid current version will be set
+   */
+  private void validateCarbonDataFileVersion() {
+    short carbondataFileVersion = CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION;
+    String carbondataFileVersionString =
+        carbonProperties.getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION);
+    try {
+      carbondataFileVersion = Short.parseShort(carbondataFileVersionString);
+    } catch (NumberFormatException e) {
+      carbondataFileVersion = CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION;
+      LOGGER.info("Current Data file version property is invalid  \"" + carbondataFileVersionString
+          + "\" is invalid. Using the Current data file version value \"" + carbondataFileVersion);
+      carbonProperties
+          .setProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, carbondataFileVersion + "");
+    }
+    if (carbondataFileVersion > CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION
+        || carbondataFileVersion < 0) {
+      LOGGER.info("Current Data file version property is invalid  \"" + carbondataFileVersionString
+          + "\" is invalid. Using the Current data file version value \"" + carbondataFileVersion);
+      carbondataFileVersion = CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION;
+      carbonProperties
+          .setProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, carbondataFileVersion + "");
+    }
+
+  }
+
+  /**
    * This method will read all the properties from file and load it into
    * memory
    */
@@ -278,18 +306,18 @@ public final class CarbonProperties {
         carbonProperties.load(fis);
       }
     } catch (FileNotFoundException e) {
-      LOGGER.error("The file: " + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH
-          + " does not exist");
+      LOGGER.error(
+          "The file: " + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH + " does not exist");
     } catch (IOException e) {
-      LOGGER.error("Error while reading the file: "
-          + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH);
+      LOGGER.error(
+          "Error while reading the file: " + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH);
     } finally {
       if (null != fis) {
         try {
           fis.close();
         } catch (IOException e) {
           LOGGER.error("Error while closing the file stream for file: "
-                  + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH);
+              + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH);
         }
       }
     }
@@ -402,6 +430,7 @@ public final class CarbonProperties {
 
   /**
    * gettting the unmerged segment numbers to be merged.
+   *
    * @return
    */
   public int[] getCompactionSegmentLevelCount() {
@@ -411,7 +440,7 @@ public final class CarbonProperties {
         CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD);
     int[] compactionSize = getIntArray(commaSeparatedLevels);
 
-    if(null == compactionSize){
+    if (null == compactionSize) {
       compactionSize = getIntArray(CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD);
     }
 
@@ -419,7 +448,6 @@ public final class CarbonProperties {
   }
 
   /**
-   *
    * @param commaSeparatedLevels
    * @return
    */
@@ -430,13 +458,12 @@ public final class CarbonProperties {
     for (String levelSize : levels) {
       try {
         int size = Integer.parseInt(levelSize.trim());
-        if(validate(size,100,0,-1) < 0 ){
+        if (validate(size, 100, 0, -1) < 0) {
           // if given size is out of boundary then take default value for all levels.
           return null;
         }
         compactionSize[i++] = size;
-      }
-      catch(NumberFormatException e){
+      } catch (NumberFormatException e) {
         LOGGER.error(
             "Given value for property" + CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD
                 + " is not proper. Taking the default value "

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 3c976db..162e9b9 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -21,6 +21,8 @@
 package org.apache.carbondata.core.util;
 
 import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.File;
@@ -28,6 +30,7 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.charset.Charset;
@@ -49,7 +52,6 @@ import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
@@ -67,11 +69,20 @@ import org.apache.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
 import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
+import org.apache.carbondata.core.reader.ThriftReader;
+import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
+import org.apache.carbondata.format.DataChunk2;
 import org.apache.carbondata.scan.model.QueryDimension;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.thrift.TBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TIOStreamTransport;
+
 import org.pentaho.di.core.exception.KettleException;
 
 
@@ -946,29 +957,25 @@ public final class CarbonUtil {
    * @return value compression model
    */
   public static ValueCompressionModel getValueCompressionModel(
-      List<DataChunk> measureDataChunkList) {
-    Object[] maxValue = new Object[measureDataChunkList.size()];
-    Object[] minValue = new Object[measureDataChunkList.size()];
-    Object[] uniqueValue = new Object[measureDataChunkList.size()];
-    int[] decimal = new int[measureDataChunkList.size()];
-    char[] type = new char[measureDataChunkList.size()];
-    byte[] dataTypeSelected = new byte[measureDataChunkList.size()];
+      List<ValueEncoderMeta> encodeMetaList) {
+    Object[] maxValue = new Object[encodeMetaList.size()];
+    Object[] minValue = new Object[encodeMetaList.size()];
+    Object[] uniqueValue = new Object[encodeMetaList.size()];
+    int[] decimal = new int[encodeMetaList.size()];
+    char[] type = new char[encodeMetaList.size()];
+    byte[] dataTypeSelected = new byte[encodeMetaList.size()];
 
     /**
      * to fill the meta data required for value compression model
      */
     for (int i = 0; i < dataTypeSelected.length; i++) {
-      int indexOf = measureDataChunkList.get(i).getEncodingList().indexOf(Encoding.DELTA);
-      if (indexOf > -1) {
-        ValueEncoderMeta valueEncoderMeta =
-            measureDataChunkList.get(i).getValueEncoderMeta().get(indexOf);
-        maxValue[i] = valueEncoderMeta.getMaxValue();
-        minValue[i] = valueEncoderMeta.getMinValue();
-        uniqueValue[i] = valueEncoderMeta.getUniqueValue();
-        decimal[i] = valueEncoderMeta.getDecimal();
-        type[i] = valueEncoderMeta.getType();
-        dataTypeSelected[i] = valueEncoderMeta.getDataTypeSelected();
-      }
+      ValueEncoderMeta valueEncoderMeta = encodeMetaList.get(i);
+      maxValue[i] = valueEncoderMeta.getMaxValue();
+      minValue[i] = valueEncoderMeta.getMinValue();
+      uniqueValue[i] = valueEncoderMeta.getUniqueValue();
+      decimal[i] = valueEncoderMeta.getDecimal();
+      type[i] = valueEncoderMeta.getType();
+      dataTypeSelected[i] = valueEncoderMeta.getDataTypeSelected();
     }
     MeasureMetaDataModel measureMetadataModel =
         new MeasureMetaDataModel(minValue, maxValue, decimal, dataTypeSelected.length, uniqueValue,
@@ -1055,11 +1062,13 @@ public final class CarbonUtil {
    * @return Data file metadata instance
    * @throws CarbonUtilException
    */
-  public static DataFileFooter readMetadatFile(String filePath, long blockOffset, long blockLength)
+  public static DataFileFooter readMetadatFile(TableBlockInfo tableBlockInfo)
       throws CarbonUtilException {
-    DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
+    AbstractDataFileFooterConverter fileFooterConverter =
+        DataFileFooterConverterFactory.getInstance()
+            .getDataFileFooterConverter(tableBlockInfo.getVersion());
     try {
-      return fileFooterConverter.readDataFileFooter(filePath, blockOffset, blockLength);
+      return fileFooterConverter.readDataFileFooter(tableBlockInfo);
     } catch (IOException e) {
       throw new CarbonUtilException("Problem while reading the file metadata", e);
     }
@@ -1462,5 +1471,161 @@ public final class CarbonUtil {
     return segmentStringbuilder.toString();
   }
 
+  /**
+   * Below method will be used to convert the thrift object to byte array.
+   */
+  public static byte[] getByteArray(TBase t) {
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    byte[] thriftByteArray = null;
+    TProtocol binaryOut = new TCompactProtocol(new TIOStreamTransport(stream));
+    try {
+      t.write(binaryOut);
+      stream.flush();
+      thriftByteArray = stream.toByteArray();
+    } catch (TException | IOException e) {
+      closeStreams(stream);
+    } finally {
+      closeStreams(stream);
+    }
+    return thriftByteArray;
+  }
+
+  /**
+   * Below method will be used to convert the bytearray to data chunk object
+   *
+   * @param dataChunkBytes datachunk thrift object in bytes
+   * @return data chunk thrift object
+   */
+  public static DataChunk2 readDataChunk(byte[] dataChunkBytes) {
+    try {
+      return (DataChunk2) read(dataChunkBytes, new ThriftReader.TBaseCreator() {
+        @Override public TBase create() {
+          return new DataChunk2();
+        }
+      });
+    } catch (IOException e) {
+      LOGGER.error(e);
+    }
+    return null;
+  }
+
+  /**
+   * Below method will be used to convert the byte array value to thrift object for
+   * data chunk
+   *
+   * @param data    thrift byte array
+   * @param creator type of thrift
+   * @return thrift object
+   * @throws IOException any problem while converting the object
+   */
+  private static TBase read(byte[] data, TBaseCreator creator) throws IOException {
+    ByteArrayInputStream stream = new ByteArrayInputStream(data);
+    TProtocol binaryIn = new TCompactProtocol(new TIOStreamTransport(stream));
+    TBase t = creator.create();
+    try {
+      t.read(binaryIn);
+    } catch (TException e) {
+      throw new IOException(e);
+    } finally {
+      CarbonUtil.closeStreams(stream);
+    }
+    return t;
+  }
+
+  /**
+   * Below method will be used to convert the encode metadata to
+   * ValueEncoderMeta object
+   *
+   * @param encoderMeta
+   * @return ValueEncoderMeta object
+   */
+  public static ValueEncoderMeta deserializeEncoderMeta(byte[] encoderMeta) {
+    // TODO : should remove the unnecessary fields.
+    ByteArrayInputStream aos = null;
+    ObjectInputStream objStream = null;
+    ValueEncoderMeta meta = null;
+    try {
+      aos = new ByteArrayInputStream(encoderMeta);
+      objStream = new ObjectInputStream(aos);
+      meta = (ValueEncoderMeta) objStream.readObject();
+    } catch (ClassNotFoundException e) {
+      LOGGER.error(e);
+    } catch (IOException e) {
+      CarbonUtil.closeStreams(objStream);
+    }
+    return meta;
+  }
+
+  /**
+   * Below method will be used to convert indexes in range
+   * Indexes=[0,1,2,3,4,5,6,7,8,9]
+   * Length=9
+   * number of element in group =5
+   * then output will be [0,1,2,3,4],[5,6,7,8],[9]
+   *
+   * @param indexes                indexes
+   * @param length                 number of element to be considered
+   * @param numberOfElementInGroup number of element in group
+   * @return range indexes
+   */
+  public static int[][] getRangeIndex(int[] indexes, int length, int numberOfElementInGroup) {
+    List<List<Integer>> rangeList = new ArrayList<>();
+    int[][] outputArray = null;
+    int k = 0;
+    int index = 1;
+    if (indexes.length == 1) {
+      outputArray = new int[1][2];
+      outputArray[0][0] = indexes[0];
+      outputArray[0][1] = indexes[0];
+      return outputArray;
+    }
+    while (index < length) {
+      if (indexes[index] - indexes[index - 1] == 1 && k < numberOfElementInGroup - 1) {
+        k++;
+      } else {
+        if (k > 0) {
+          List<Integer> range = new ArrayList<>();
+          rangeList.add(range);
+          range.add(indexes[index - k - 1]);
+          range.add(indexes[index - 1]);
+        } else {
+          List<Integer> range = new ArrayList<>();
+          rangeList.add(range);
+          range.add(indexes[index - 1]);
+        }
+        k = 0;
+      }
+      index++;
+    }
+    if (k > 0) {
+      List<Integer> range = new ArrayList<>();
+      rangeList.add(range);
+      range.add(indexes[index - k - 1]);
+      range.add(indexes[index - 1]);
+    } else {
+      List<Integer> range = new ArrayList<>();
+      rangeList.add(range);
+      range.add(indexes[index - 1]);
+
+    }
+    if (length != indexes.length) {
+      List<Integer> range = new ArrayList<>();
+      rangeList.add(range);
+      range.add(indexes[indexes.length - 1]);
+    }
+
+    // as diving in range so array size will be always 2
+    outputArray = new int[rangeList.size()][2];
+    for (int i = 0; i < outputArray.length; i++) {
+      if (rangeList.get(i).size() == 1) {
+        outputArray[i][0] = rangeList.get(i).get(0);
+        outputArray[i][1] = rangeList.get(i).get(0);
+      } else {
+        outputArray[i][0] = rangeList.get(i).get(0);
+        outputArray[i][1] = rangeList.get(i).get(1);
+      }
+    }
+    return outputArray;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
index 5f3565c..ea1324e 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
@@ -18,135 +18,44 @@
  */
 package org.apache.carbondata.core.util;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.carbon.metadata.blocklet.SegmentInfo;
-import org.apache.carbondata.core.carbon.metadata.blocklet.compressor.ChunkCompressorMeta;
-import org.apache.carbondata.core.carbon.metadata.blocklet.compressor.CompressionCodec;
 import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
-import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.PresenceMeta;
-import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletBTreeIndex;
 import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
-import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
-import org.apache.carbondata.core.carbon.metadata.blocklet.sort.SortState;
-import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
-import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
-import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.reader.CarbonFooterReader;
-import org.apache.carbondata.core.reader.CarbonIndexFileReader;
-import org.apache.carbondata.format.BlockIndex;
 import org.apache.carbondata.format.FileFooter;
 
 /**
  * Below class will be used to convert the thrift object of data file
  * meta data to wrapper object
  */
-public class DataFileFooterConverter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataFileFooterConverter.class.getName());
-
-  /**
-   * Below method will be used to get the index info from index file
-   *
-   * @param filePath           file path of the index file
-   * @param tableBlockInfoList table block index
-   * @return list of index info
-   * @throws IOException problem while reading the index file
-   */
-  public List<DataFileFooter> getIndexInfo(String filePath, List<TableBlockInfo> tableBlockInfoList)
-      throws IOException, CarbonUtilException {
-    CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
-    List<DataFileFooter> dataFileFooters = new ArrayList<DataFileFooter>();
-    try {
-      // open the reader
-      indexReader.openThriftReader(filePath);
-      // get the index header
-      org.apache.carbondata.format.IndexHeader readIndexHeader = indexReader.readIndexHeader();
-      List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
-      List<org.apache.carbondata.format.ColumnSchema> table_columns =
-          readIndexHeader.getTable_columns();
-      for (int i = 0; i < table_columns.size(); i++) {
-        columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
-      }
-      // get the segment info
-      SegmentInfo segmentInfo = getSegmentInfo(readIndexHeader.getSegment_info());
-      BlockletIndex blockletIndex = null;
-      int counter = 0;
-      DataFileFooter dataFileFooter = null;
-      // read the block info from file
-      while (indexReader.hasNext()) {
-        BlockIndex readBlockIndexInfo = indexReader.readBlockIndexInfo();
-        blockletIndex = getBlockletIndex(readBlockIndexInfo.getBlock_index());
-        dataFileFooter = new DataFileFooter();
-        TableBlockInfo tableBlockInfo = tableBlockInfoList.get(counter++);
-        int blockletSize = getBlockletSize(readBlockIndexInfo);
-        tableBlockInfo.getBlockletInfos().setNoOfBlockLets(blockletSize);
-        dataFileFooter.setBlockletIndex(blockletIndex);
-        dataFileFooter.setColumnInTable(columnSchemaList);
-        dataFileFooter.setNumberOfRows(readBlockIndexInfo.getNum_rows());
-        dataFileFooter.setBlockInfo(new BlockInfo(tableBlockInfo));
-        dataFileFooter.setSegmentInfo(segmentInfo);
-        dataFileFooters.add(dataFileFooter);
-      }
-    } finally {
-      indexReader.closeThriftReader();
-    }
-    return dataFileFooters;
-  }
-
-  /**
-   * the methods returns the number of blocklets in a block
-   * @param readBlockIndexInfo
-   * @return
-   */
-  private int getBlockletSize(BlockIndex readBlockIndexInfo) {
-    long num_rows = readBlockIndexInfo.getNum_rows();
-    int blockletSize = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    int remainder = (int) (num_rows % blockletSize);
-    int noOfBlockLet = (int) (num_rows / blockletSize);
-    // there could be some blocklets which will not
-    // contain the total records equal to the blockletSize
-    if (remainder > 0) {
-      noOfBlockLet = noOfBlockLet + 1;
-    }
-    return noOfBlockLet;
-  }
+public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
 
   /**
    * Below method will be used to convert thrift file meta to wrapper file meta
    */
-  public DataFileFooter readDataFileFooter(String filePath, long blockOffset, long blockLength)
+  @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
       throws IOException {
     DataFileFooter dataFileFooter = new DataFileFooter();
     FileHolder fileReader = null;
     try {
-      long completeBlockLength = blockOffset + blockLength;
+      long completeBlockLength = tableBlockInfo.getBlockLength();
       long footerPointer = completeBlockLength - 8;
-      fileReader = FileFactory.getFileHolder(FileFactory.getFileType(filePath));
-      long actualFooterOffset = fileReader.readLong(filePath, footerPointer);
-      CarbonFooterReader reader = new CarbonFooterReader(filePath, actualFooterOffset);
+      fileReader = FileFactory.getFileHolder(FileFactory.getFileType(tableBlockInfo.getFilePath()));
+      long actualFooterOffset = fileReader.readLong(tableBlockInfo.getFilePath(), footerPointer);
+      CarbonFooterReader reader =
+          new CarbonFooterReader(tableBlockInfo.getFilePath(), actualFooterOffset);
       FileFooter footer = reader.readFooter();
-      dataFileFooter.setVersionId(footer.getVersion());
+      dataFileFooter.setVersionId((short) footer.getVersion());
       dataFileFooter.setNumberOfRows(footer.getNum_rows());
       dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));
       List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
@@ -183,66 +92,6 @@ public class DataFileFooterConverter {
   }
 
   /**
-   * Below method will be used to get blocklet index for data file meta
-   *
-   * @param blockletIndexList
-   * @return blocklet index
-   */
-  private BlockletIndex getBlockletIndexForDataFileFooter(List<BlockletIndex> blockletIndexList) {
-    BlockletIndex blockletIndex = new BlockletIndex();
-    BlockletBTreeIndex blockletBTreeIndex = new BlockletBTreeIndex();
-    blockletBTreeIndex.setStartKey(blockletIndexList.get(0).getBtreeIndex().getStartKey());
-    blockletBTreeIndex
-        .setEndKey(blockletIndexList.get(blockletIndexList.size() - 1).getBtreeIndex().getEndKey());
-    blockletIndex.setBtreeIndex(blockletBTreeIndex);
-    byte[][] currentMinValue = blockletIndexList.get(0).getMinMaxIndex().getMinValues().clone();
-    byte[][] currentMaxValue = blockletIndexList.get(0).getMinMaxIndex().getMaxValues().clone();
-    byte[][] minValue = null;
-    byte[][] maxValue = null;
-    for (int i = 1; i < blockletIndexList.size(); i++) {
-      minValue = blockletIndexList.get(i).getMinMaxIndex().getMinValues();
-      maxValue = blockletIndexList.get(i).getMinMaxIndex().getMaxValues();
-      for (int j = 0; j < maxValue.length; j++) {
-        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMinValue[j], minValue[j]) > 0) {
-          currentMinValue[j] = minValue[j].clone();
-        }
-        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMaxValue[j], maxValue[j]) < 0) {
-          currentMaxValue[j] = maxValue[j].clone();
-        }
-      }
-    }
-
-    BlockletMinMaxIndex minMax = new BlockletMinMaxIndex();
-    minMax.setMaxValues(currentMaxValue);
-    minMax.setMinValues(currentMinValue);
-    blockletIndex.setMinMaxIndex(minMax);
-    return blockletIndex;
-  }
-
-  private ColumnSchema thriftColumnSchmeaToWrapperColumnSchema(
-      org.apache.carbondata.format.ColumnSchema externalColumnSchema) {
-    ColumnSchema wrapperColumnSchema = new ColumnSchema();
-    wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
-    wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
-    wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
-    wrapperColumnSchema
-        .setDataType(thriftDataTyopeToWrapperDataType(externalColumnSchema.data_type));
-    wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
-    List<Encoding> encoders = new ArrayList<Encoding>();
-    for (org.apache.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {
-      encoders.add(fromExternalToWrapperEncoding(encoder));
-    }
-    wrapperColumnSchema.setEncodingList(encoders);
-    wrapperColumnSchema.setNumberOfChild(externalColumnSchema.getNum_child());
-    wrapperColumnSchema.setPrecision(externalColumnSchema.getPrecision());
-    wrapperColumnSchema.setColumnGroup(externalColumnSchema.getColumn_group_id());
-    wrapperColumnSchema.setScale(externalColumnSchema.getScale());
-    wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
-    wrapperColumnSchema.setAggregateFunction(externalColumnSchema.getAggregate_function());
-    return wrapperColumnSchema;
-  }
-
-  /**
    * Below method is to convert the blocklet info of the thrift to wrapper
    * blocklet info
    *
@@ -273,228 +122,4 @@ public class DataFileFooterConverter {
     blockletInfo.setNumberOfRows(blockletInfoThrift.getNum_rows());
     return blockletInfo;
   }
-
-  /**
-   * Below method is convert the thrift encoding to wrapper encoding
-   *
-   * @param encoderThrift thrift encoding
-   * @return wrapper encoding
-   */
-  private Encoding fromExternalToWrapperEncoding(
-      org.apache.carbondata.format.Encoding encoderThrift) {
-    switch (encoderThrift) {
-      case DICTIONARY:
-        return Encoding.DICTIONARY;
-      case DELTA:
-        return Encoding.DELTA;
-      case RLE:
-        return Encoding.RLE;
-      case INVERTED_INDEX:
-        return Encoding.INVERTED_INDEX;
-      case BIT_PACKED:
-        return Encoding.BIT_PACKED;
-      case DIRECT_DICTIONARY:
-        return Encoding.DIRECT_DICTIONARY;
-      default:
-        return Encoding.DICTIONARY;
-    }
-  }
-
-  /**
-   * Below method will be used to convert the thrift compression to wrapper
-   * compression codec
-   *
-   * @param compressionCodecThrift
-   * @return wrapper compression codec
-   */
-  private CompressionCodec getCompressionCodec(
-      org.apache.carbondata.format.CompressionCodec compressionCodecThrift) {
-    switch (compressionCodecThrift) {
-      case SNAPPY:
-        return CompressionCodec.SNAPPY;
-      default:
-        return CompressionCodec.SNAPPY;
-    }
-  }
-
-  /**
-   * Below method will be used to convert thrift segment object to wrapper
-   * segment object
-   *
-   * @param segmentInfo thrift segment info object
-   * @return wrapper segment info object
-   */
-  private SegmentInfo getSegmentInfo(org.apache.carbondata.format.SegmentInfo segmentInfo) {
-    SegmentInfo info = new SegmentInfo();
-    int[] cardinality = new int[segmentInfo.getColumn_cardinalities().size()];
-    for (int i = 0; i < cardinality.length; i++) {
-      cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
-    }
-    info.setColumnCardinality(cardinality);
-    info.setNumberOfColumns(segmentInfo.getNum_cols());
-    return info;
-  }
-
-  /**
-   * Below method will be used to convert the blocklet index of thrift to
-   * wrapper
-   *
-   * @param blockletIndexThrift
-   * @return blocklet index wrapper
-   */
-  private BlockletIndex getBlockletIndex(
-      org.apache.carbondata.format.BlockletIndex blockletIndexThrift) {
-    org.apache.carbondata.format.BlockletBTreeIndex btreeIndex =
-        blockletIndexThrift.getB_tree_index();
-    org.apache.carbondata.format.BlockletMinMaxIndex minMaxIndex =
-        blockletIndexThrift.getMin_max_index();
-    return new BlockletIndex(
-        new BlockletBTreeIndex(btreeIndex.getStart_key(), btreeIndex.getEnd_key()),
-        new BlockletMinMaxIndex(minMaxIndex.getMin_values(), minMaxIndex.getMax_values()));
-  }
-
-  /**
-   * Below method will be used to convert the thrift compression meta to
-   * wrapper chunk compression meta
-   *
-   * @param chunkCompressionMetaThrift
-   * @return chunkCompressionMetaWrapper
-   */
-  private ChunkCompressorMeta getChunkCompressionMeta(
-      org.apache.carbondata.format.ChunkCompressionMeta chunkCompressionMetaThrift) {
-    ChunkCompressorMeta compressorMeta = new ChunkCompressorMeta();
-    compressorMeta
-        .setCompressor(getCompressionCodec(chunkCompressionMetaThrift.getCompression_codec()));
-    compressorMeta.setCompressedSize(chunkCompressionMetaThrift.getTotal_compressed_size());
-    compressorMeta.setUncompressedSize(chunkCompressionMetaThrift.getTotal_uncompressed_size());
-    return compressorMeta;
-  }
-
-  /**
-   * Below method will be used to convert the thrift data type to wrapper data
-   * type
-   *
-   * @param dataTypeThrift
-   * @return dataType wrapper
-   */
-  private DataType thriftDataTyopeToWrapperDataType(
-      org.apache.carbondata.format.DataType dataTypeThrift) {
-    switch (dataTypeThrift) {
-      case STRING:
-        return DataType.STRING;
-      case SHORT:
-        return DataType.SHORT;
-      case INT:
-        return DataType.INT;
-      case LONG:
-        return DataType.LONG;
-      case DOUBLE:
-        return DataType.DOUBLE;
-      case DECIMAL:
-        return DataType.DECIMAL;
-      case TIMESTAMP:
-        return DataType.TIMESTAMP;
-      case ARRAY:
-        return DataType.ARRAY;
-      case STRUCT:
-        return DataType.STRUCT;
-      default:
-        return DataType.STRING;
-    }
-  }
-
-  /**
-   * Below method will be used to convert the thrift presence meta to wrapper
-   * presence meta
-   *
-   * @param presentMetadataThrift
-   * @return wrapper presence meta
-   */
-  private PresenceMeta getPresenceMeta(
-      org.apache.carbondata.format.PresenceMeta presentMetadataThrift) {
-    PresenceMeta presenceMeta = new PresenceMeta();
-    presenceMeta.setRepresentNullValues(presentMetadataThrift.isRepresents_presence());
-    presenceMeta.setBitSet(BitSet.valueOf(presentMetadataThrift.getPresent_bit_stream()));
-    return presenceMeta;
-  }
-
-  /**
-   * Below method will be used to convert the thrift object to wrapper object
-   *
-   * @param sortStateThrift
-   * @return wrapper sort state object
-   */
-  private SortState getSortState(org.apache.carbondata.format.SortState sortStateThrift) {
-    if (sortStateThrift == org.apache.carbondata.format.SortState.SORT_EXPLICIT) {
-      return SortState.SORT_EXPLICT;
-    } else if (sortStateThrift == org.apache.carbondata.format.SortState.SORT_NATIVE) {
-      return SortState.SORT_NATIVE;
-    } else {
-      return SortState.SORT_NONE;
-    }
-  }
-
-  /**
-   * Below method will be used to convert the thrift data chunk to wrapper
-   * data chunk
-   *
-   * @param datachunkThrift
-   * @return wrapper data chunk
-   */
-  private DataChunk getDataChunk(org.apache.carbondata.format.DataChunk datachunkThrift,
-      boolean isPresenceMetaPresent) {
-    DataChunk dataChunk = new DataChunk();
-    dataChunk.setColumnUniqueIdList(datachunkThrift.getColumn_ids());
-    dataChunk.setDataPageLength(datachunkThrift.getData_page_length());
-    dataChunk.setDataPageOffset(datachunkThrift.getData_page_offset());
-    if (isPresenceMetaPresent) {
-      dataChunk.setNullValueIndexForColumn(getPresenceMeta(datachunkThrift.getPresence()));
-    }
-    dataChunk.setRlePageLength(datachunkThrift.getRle_page_length());
-    dataChunk.setRlePageOffset(datachunkThrift.getRle_page_offset());
-    dataChunk.setRowMajor(datachunkThrift.isRowMajor());
-    dataChunk.setRowIdPageLength(datachunkThrift.getRowid_page_length());
-    dataChunk.setRowIdPageOffset(datachunkThrift.getRowid_page_offset());
-    dataChunk.setSortState(getSortState(datachunkThrift.getSort_state()));
-    dataChunk.setChunkCompressionMeta(getChunkCompressionMeta(datachunkThrift.getChunk_meta()));
-    List<Encoding> encodingList = new ArrayList<Encoding>(datachunkThrift.getEncoders().size());
-    for (int i = 0; i < datachunkThrift.getEncoders().size(); i++) {
-      encodingList.add(fromExternalToWrapperEncoding(datachunkThrift.getEncoders().get(i)));
-    }
-    dataChunk.setEncoderList(encodingList);
-    if (encodingList.contains(Encoding.DELTA)) {
-      List<ByteBuffer> thriftEncoderMeta = datachunkThrift.getEncoder_meta();
-      List<ValueEncoderMeta> encodeMetaList =
-          new ArrayList<ValueEncoderMeta>(thriftEncoderMeta.size());
-      for (int i = 0; i < thriftEncoderMeta.size(); i++) {
-        encodeMetaList.add(deserializeEncoderMeta(thriftEncoderMeta.get(i).array()));
-      }
-      dataChunk.setValueEncoderMeta(encodeMetaList);
-    }
-    return dataChunk;
-  }
-
-  /**
-   * Below method will be used to convert the encode metadata to
-   * ValueEncoderMeta object
-   *
-   * @param encoderMeta
-   * @return ValueEncoderMeta object
-   */
-  private ValueEncoderMeta deserializeEncoderMeta(byte[] encoderMeta) {
-    // TODO : should remove the unnecessary fields.
-    ByteArrayInputStream aos = null;
-    ObjectInputStream objStream = null;
-    ValueEncoderMeta meta = null;
-    try {
-      aos = new ByteArrayInputStream(encoderMeta);
-      objStream = new ObjectInputStream(aos);
-      meta = (ValueEncoderMeta) objStream.readObject();
-    } catch (ClassNotFoundException e) {
-      LOGGER.error(e);
-    } catch (IOException e) {
-      CarbonUtil.closeStreams(objStream);
-    }
-    return meta;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter2.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter2.java
new file mode 100644
index 0000000..d971756
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter2.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.reader.CarbonFooterReader;
+import org.apache.carbondata.format.FileFooter;
+
+/**
+ * Below class will be used to convert the thrift object of data file
+ * meta data to wrapper object for version 2 data file
+ */
+
+public class DataFileFooterConverter2 extends AbstractDataFileFooterConverter {
+
+  /**
+   * Below method will be used to convert thrift file meta to wrapper file meta
+   */
+  @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
+      throws IOException {
+    DataFileFooter dataFileFooter = new DataFileFooter();
+    CarbonFooterReader reader =
+        new CarbonFooterReader(tableBlockInfo.getFilePath(), tableBlockInfo.getBlockOffset());
+    FileFooter footer = reader.readFooter();
+    dataFileFooter.setVersionId((short) footer.getVersion());
+    dataFileFooter.setNumberOfRows(footer.getNum_rows());
+    dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));
+    List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
+    List<org.apache.carbondata.format.ColumnSchema> table_columns = footer.getTable_columns();
+    for (int i = 0; i < table_columns.size(); i++) {
+      columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
+    }
+    dataFileFooter.setColumnInTable(columnSchemaList);
+
+    List<org.apache.carbondata.format.BlockletIndex> leaf_node_indices_Thrift =
+        footer.getBlocklet_index_list();
+    List<BlockletIndex> blockletIndexList = new ArrayList<BlockletIndex>();
+    for (int i = 0; i < leaf_node_indices_Thrift.size(); i++) {
+      BlockletIndex blockletIndex = getBlockletIndex(leaf_node_indices_Thrift.get(i));
+      blockletIndexList.add(blockletIndex);
+    }
+    List<org.apache.carbondata.format.BlockletInfo2> leaf_node_infos_Thrift =
+        footer.getBlocklet_info_list2();
+    List<BlockletInfo> blockletInfoList = new ArrayList<BlockletInfo>();
+    for (int i = 0; i < leaf_node_infos_Thrift.size(); i++) {
+      BlockletInfo blockletInfo = getBlockletInfo(leaf_node_infos_Thrift.get(i),
+          getNumberOfDimensionColumns(columnSchemaList));
+      blockletInfo.setBlockletIndex(blockletIndexList.get(i));
+      blockletInfoList.add(blockletInfo);
+    }
+    dataFileFooter.setBlockletList(blockletInfoList);
+    dataFileFooter.setBlockletIndex(getBlockletIndexForDataFileFooter(blockletIndexList));
+    return dataFileFooter;
+  }
+
+  /**
+   * Below method is to convert the blocklet info of the thrift to wrapper
+   * blocklet info
+   *
+   * @param blockletInfoThrift blocklet info of the thrift
+   * @return blocklet info wrapper
+   */
+  private BlockletInfo getBlockletInfo(
+      org.apache.carbondata.format.BlockletInfo2 blockletInfoThrift, int numberOfDimensionColumns) {
+    BlockletInfo blockletInfo = new BlockletInfo();
+    List<Long> dimensionColumnChunkOffsets =
+        blockletInfoThrift.getColumn_data_chunks_offsets().subList(0, numberOfDimensionColumns);
+    List<Long> measureColumnChunksOffsets = blockletInfoThrift.getColumn_data_chunks_offsets()
+        .subList(numberOfDimensionColumns,
+            blockletInfoThrift.getColumn_data_chunks_offsets().size());
+    List<Short> dimensionColumnChunkLength =
+        blockletInfoThrift.getColumn_data_chunks_length().subList(0, numberOfDimensionColumns);
+    List<Short> measureColumnChunksLength = blockletInfoThrift.getColumn_data_chunks_length()
+        .subList(numberOfDimensionColumns,
+            blockletInfoThrift.getColumn_data_chunks_offsets().size());
+    blockletInfo.setDimensionChunkOffsets(dimensionColumnChunkOffsets);
+    blockletInfo.setMeasureChunkOffsets(measureColumnChunksOffsets);
+    blockletInfo.setDimensionChunksLength(dimensionColumnChunkLength);
+    blockletInfo.setMeasureChunksLength(measureColumnChunksLength);
+    blockletInfo.setNumberOfRows(blockletInfoThrift.getNum_rows());
+    return blockletInfo;
+  }
+
+  /**
+   * Below method will be used to get the number of dimension column
+   * in carbon column schema
+   *
+   * @param columnSchemaList column schema list
+   * @return number of dimension column
+   */
+  private int getNumberOfDimensionColumns(List<ColumnSchema> columnSchemaList) {
+    int numberOfDimensionColumns = 0;
+    int previousColumnGroupId = -1;
+    ColumnSchema columnSchema = null;
+    for (int i = 0; i < columnSchemaList.size(); i++) {
+      columnSchema = columnSchemaList.get(i);
+      if (columnSchema.isDimensionColumn() && columnSchema.isColumnar()) {
+        numberOfDimensionColumns++;
+      } else if (columnSchema.isDimensionColumn()) {
+        if (previousColumnGroupId != columnSchema.getColumnGroupId()) {
+          previousColumnGroupId = columnSchema.getColumnGroupId();
+          numberOfDimensionColumns++;
+        }
+      } else {
+        break;
+      }
+    }
+    return numberOfDimensionColumns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterFactory.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterFactory.java
new file mode 100644
index 0000000..a079ad7
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterFactory.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.util;
+
+
+/**
+ * Factory class to get the thrift reader object based on version
+ */
+public class DataFileFooterConverterFactory {
+
+  /**
+   * static instance
+   */
+  private static final DataFileFooterConverterFactory FOOTER_CONVERTER_FACTORY =
+      new DataFileFooterConverterFactory();
+
+  /**
+   * private constructor
+   */
+  private DataFileFooterConverterFactory() {
+
+  }
+
+  /**
+   * Below method will be used to get the instance of this class
+   *
+   * @return DataFileFooterConverterFactory instance
+   */
+  public static DataFileFooterConverterFactory getInstance() {
+    return FOOTER_CONVERTER_FACTORY;
+  }
+
+  /**
+   * Method will be used to get the file footer converter instance based on version
+   *
+   * @param versionNumber
+   * @return footer reader instance
+   */
+  public AbstractDataFileFooterConverter getDataFileFooterConverter(final short versionNumber) {
+    switch (versionNumber) {
+      case 2:
+        return new DataFileFooterConverter2();
+      default:
+        return new DataFileFooterConverter();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/core/writer/CarbonFooterWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonFooterWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonFooterWriter.java
index 04d2b97..758c2d9 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonFooterWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonFooterWriter.java
@@ -28,9 +28,6 @@ import org.apache.carbondata.format.FileFooter;
  */
 public class CarbonFooterWriter {
 
-  // It is version number of this format class.
-  private static int VERSION_NUMBER = 1;
-
   // Fact file path
   private String filePath;
 
@@ -48,7 +45,6 @@ public class CarbonFooterWriter {
   public void writeFooter(FileFooter footer, long currentPosition) throws IOException {
 
     ThriftWriter thriftWriter = openThriftWriter(filePath);
-    footer.setVersion(VERSION_NUMBER);
     try {
       thriftWriter.write(footer);
       thriftWriter.writeOffset(currentPosition);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
index 85e979a..8289c8b 100644
--- a/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
@@ -18,7 +18,12 @@
  */
 package org.apache.carbondata.scan.executor.impl;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import java.util.concurrent.Executors;
 
 import org.apache.carbondata.common.logging.LogService;
@@ -33,7 +38,8 @@ import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.apache.carbondata.core.carbon.querystatistics.*;
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatistic;
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsConstants;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
@@ -90,7 +96,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // Initializing statistics list to record the query statistics
     // creating copy on write to handle concurrent scenario
     queryProperties.queryStatisticsRecorder =
-            CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId());
+        CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId());
     queryModel.setStatisticsRecorder(queryProperties.queryStatisticsRecorder);
     QueryUtil.resolveQueryModel(queryModel);
     QueryStatistic queryStatistic = new QueryStatistic();
@@ -143,9 +149,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // and measure column start index
     queryProperties.aggExpressionStartIndex = queryModel.getQueryMeasures().size();
     queryProperties.measureStartIndex = aggTypes.length - queryModel.getQueryMeasures().size();
+    queryProperties.filterMeasures = new HashSet<>();
+    queryProperties.complexFilterDimension = new HashSet<>();
+    QueryUtil.getAllFilterDimensions(queryModel.getFilterExpressionResolverTree(),
+        queryProperties.complexFilterDimension, queryProperties.filterMeasures);
 
-    queryProperties.complexFilterDimension =
-        QueryUtil.getAllFilterDimensions(queryModel.getFilterExpressionResolverTree());
     queryStatistic = new QueryStatistic();
     // dictionary column unique column id to dictionary mapping
     // which will be used to get column actual data
@@ -314,13 +322,38 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     List<CarbonMeasure> expressionMeasures =
         new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     // setting all the dimension chunk indexes to be read from file
-    blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(QueryUtil
-        .getDimensionsBlockIndexes(updatedQueryDimension,
-            segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions));
-    // setting all the measure chunk indexes to be read from file
-    blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(QueryUtil
+    int numberOfElementToConsider = 0;
+    int[] dimensionsBlockIndexes = QueryUtil.getDimensionsBlockIndexes(updatedQueryDimension,
+        segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions,
+        queryProperties.complexFilterDimension);
+    if (dimensionsBlockIndexes.length > 0) {
+      numberOfElementToConsider = dimensionsBlockIndexes[dimensionsBlockIndexes.length - 1]
+          == segmentProperties.getBlockTodimensionOrdinalMapping().size() - 1 ?
+          dimensionsBlockIndexes.length - 1 :
+          dimensionsBlockIndexes.length;
+      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(CarbonUtil
+          .getRangeIndex(dimensionsBlockIndexes, numberOfElementToConsider,
+              CarbonCommonConstants.NUMBER_OF_COLUMN_READ_IN_IO));
+    } else {
+      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[0][0]);
+    }
+
+    int[] measureBlockIndexes = QueryUtil
         .getMeasureBlockIndexes(queryModel.getQueryMeasures(), expressionMeasures,
-            segmentProperties.getMeasuresOrdinalToBlockMapping()));
+            segmentProperties.getMeasuresOrdinalToBlockMapping(), queryProperties.filterMeasures);
+    if (measureBlockIndexes.length > 0) {
+
+      numberOfElementToConsider = measureBlockIndexes[measureBlockIndexes.length - 1]
+          == segmentProperties.getMeasures().size() - 1 ?
+          measureBlockIndexes.length - 1 :
+          measureBlockIndexes.length;
+      // setting all the measure chunk indexes to be read from file
+      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(CarbonUtil
+          .getRangeIndex(measureBlockIndexes, numberOfElementToConsider,
+              CarbonCommonConstants.NUMBER_OF_COLUMN_READ_IN_IO));
+    } else {
+      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[0][0]);
+    }
     // setting the key structure info which will be required
     // to update the older block key with new key generator
     blockExecutionInfo.setKeyStructureInfo(queryProperties.keyStructureInfo);



[2/5] incubator-carbondata git commit: Improve first time query performance

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/scan/executor/impl/QueryExecutorProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/executor/impl/QueryExecutorProperties.java b/core/src/main/java/org/apache/carbondata/scan/executor/impl/QueryExecutorProperties.java
index 7663738..bf61be2 100644
--- a/core/src/main/java/org/apache/carbondata/scan/executor/impl/QueryExecutorProperties.java
+++ b/core/src/main/java/org/apache/carbondata/scan/executor/impl/QueryExecutorProperties.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.carbon.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorder;
 import org.apache.carbondata.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.scan.filter.GenericQueryType;
@@ -79,6 +80,8 @@ public class QueryExecutorProperties {
    * all the complex dimension which is on filter
    */
   public Set<CarbonDimension> complexFilterDimension;
+
+  public Set<CarbonMeasure> filterMeasures;
   /**
    * to record the query execution details phase wise
    */
@@ -91,5 +94,4 @@ public class QueryExecutorProperties {
    * list of blocks in which query will be executed
    */
   protected List<AbstractIndex> dataBlocks;
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/scan/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/apache/carbondata/scan/executor/infos/BlockExecutionInfo.java
index d84a183..2e80984 100644
--- a/core/src/main/java/org/apache/carbondata/scan/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/scan/executor/infos/BlockExecutionInfo.java
@@ -116,12 +116,12 @@ public class BlockExecutionInfo {
   /**
    * will be used to read the dimension block from file
    */
-  private int[] allSelectedDimensionBlocksIndexes;
+  private int[][] allSelectedDimensionBlocksIndexes;
 
   /**
    * will be used to read the measure block from file
    */
-  private int[] allSelectedMeasureBlocksIndexes;
+  private int[][] allSelectedMeasureBlocksIndexes;
 
   /**
    * this will be used to update the older block fixed length keys with the
@@ -407,28 +407,28 @@ public class BlockExecutionInfo {
   /**
    * @return the allSelectedDimensionBlocksIndexes
    */
-  public int[] getAllSelectedDimensionBlocksIndexes() {
+  public int[][] getAllSelectedDimensionBlocksIndexes() {
     return allSelectedDimensionBlocksIndexes;
   }
 
   /**
    * @param allSelectedDimensionBlocksIndexes the allSelectedDimensionBlocksIndexes to set
    */
-  public void setAllSelectedDimensionBlocksIndexes(int[] allSelectedDimensionBlocksIndexes) {
+  public void setAllSelectedDimensionBlocksIndexes(int[][] allSelectedDimensionBlocksIndexes) {
     this.allSelectedDimensionBlocksIndexes = allSelectedDimensionBlocksIndexes;
   }
 
   /**
    * @return the allSelectedMeasureBlocksIndexes
    */
-  public int[] getAllSelectedMeasureBlocksIndexes() {
+  public int[][] getAllSelectedMeasureBlocksIndexes() {
     return allSelectedMeasureBlocksIndexes;
   }
 
   /**
    * @param allSelectedMeasureBlocksIndexes the allSelectedMeasureBlocksIndexes to set
    */
-  public void setAllSelectedMeasureBlocksIndexes(int[] allSelectedMeasureBlocksIndexes) {
+  public void setAllSelectedMeasureBlocksIndexes(int[][] allSelectedMeasureBlocksIndexes) {
     this.allSelectedMeasureBlocksIndexes = allSelectedMeasureBlocksIndexes;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/scan/executor/util/QueryUtil.java
index 8837012..680876c 100644
--- a/core/src/main/java/org/apache/carbondata/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/scan/executor/util/QueryUtil.java
@@ -211,16 +211,19 @@ public class QueryUtil {
    */
   public static int[] getDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
       Map<Integer, Integer> dimensionOrdinalToBlockMapping,
-      List<CarbonDimension> customAggregationDimension) {
+      List<CarbonDimension> customAggregationDimension, Set<CarbonDimension> filterDimensions) {
     // using set as in row group columns will point to same block
     Set<Integer> dimensionBlockIndex = new HashSet<Integer>();
+    Set<Integer> filterDimensionOrdinal = getFilterDimensionOrdinal(filterDimensions);
     int blockIndex = 0;
     for (int i = 0; i < queryDimensions.size(); i++) {
-      blockIndex =
-          dimensionOrdinalToBlockMapping.get(queryDimensions.get(i).getDimension().getOrdinal());
-      dimensionBlockIndex.add(blockIndex);
-      if (queryDimensions.get(i).getDimension().numberOfChild() > 0) {
-        addChildrenBlockIndex(dimensionBlockIndex, queryDimensions.get(i).getDimension());
+      if (!filterDimensionOrdinal.contains(queryDimensions.get(i).getDimension().getOrdinal())) {
+        blockIndex =
+            dimensionOrdinalToBlockMapping.get(queryDimensions.get(i).getDimension().getOrdinal());
+        dimensionBlockIndex.add(blockIndex);
+        if (queryDimensions.get(i).getDimension().numberOfChild() > 0) {
+          addChildrenBlockIndex(dimensionBlockIndex, queryDimensions.get(i).getDimension());
+        }
       }
     }
     for (int i = 0; i < customAggregationDimension.size(); i++) {
@@ -230,8 +233,10 @@ public class QueryUtil {
       // is not push down in case of complex dimension
       dimensionBlockIndex.add(blockIndex);
     }
-    return ArrayUtils
+    int[] dimensionIndex = ArrayUtils
         .toPrimitive(dimensionBlockIndex.toArray(new Integer[dimensionBlockIndex.size()]));
+    Arrays.sort(dimensionIndex);
+    return dimensionIndex;
   }
 
   /**
@@ -252,15 +257,14 @@ public class QueryUtil {
    * Below method will be used to get the dictionary mapping for all the
    * dictionary encoded dimension present in the query
    *
-   * @param queryDimensions            query dimension present in the query this will be used to
-   *                                   convert the result from surrogate key to actual data
-   * @param absoluteTableIdentifier    absolute table identifier
+   * @param queryDimensions         query dimension present in the query this will be used to
+   *                                convert the result from surrogate key to actual data
+   * @param absoluteTableIdentifier absolute table identifier
    * @return dimension unique id to its dictionary map
    * @throws QueryExecutionException
    */
   public static Map<String, Dictionary> getDimensionDictionaryDetail(
-      List<QueryDimension> queryDimensions,
-      Set<CarbonDimension> filterComplexDimensions,
+      List<QueryDimension> queryDimensions, Set<CarbonDimension> filterComplexDimensions,
       AbsoluteTableIdentifier absoluteTableIdentifier) throws QueryExecutionException {
     // to store dimension unique column id list, this is required as
     // dimension can be present in
@@ -387,16 +391,23 @@ public class QueryUtil {
    * @return block indexes
    */
   public static int[] getMeasureBlockIndexes(List<QueryMeasure> queryMeasures,
-      List<CarbonMeasure> expressionMeasure, Map<Integer, Integer> ordinalToBlockIndexMapping) {
+      List<CarbonMeasure> expressionMeasure, Map<Integer, Integer> ordinalToBlockIndexMapping,
+      Set<CarbonMeasure> filterMeasures) {
     Set<Integer> measureBlockIndex = new HashSet<Integer>();
+    Set<Integer> filterMeasureOrdinal = getFilterMeasureOrdinal(filterMeasures);
     for (int i = 0; i < queryMeasures.size(); i++) {
-      measureBlockIndex
-          .add(ordinalToBlockIndexMapping.get(queryMeasures.get(i).getMeasure().getOrdinal()));
+      if (!filterMeasureOrdinal.contains(queryMeasures.get(i).getMeasure().getOrdinal())) {
+        measureBlockIndex
+            .add(ordinalToBlockIndexMapping.get(queryMeasures.get(i).getMeasure().getOrdinal()));
+      }
     }
     for (int i = 0; i < expressionMeasure.size(); i++) {
       measureBlockIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal()));
     }
-    return ArrayUtils.toPrimitive(measureBlockIndex.toArray(new Integer[measureBlockIndex.size()]));
+    int[] measureIndexes =
+        ArrayUtils.toPrimitive(measureBlockIndex.toArray(new Integer[measureBlockIndex.size()]));
+    Arrays.sort(measureIndexes);
+    return measureIndexes;
   }
 
   /**
@@ -912,21 +923,19 @@ public class QueryUtil {
     return parentBlockIndex;
   }
 
-  public static Set<CarbonDimension> getAllFilterDimensions(FilterResolverIntf filterResolverTree) {
-    Set<CarbonDimension> filterDimensions = new HashSet<CarbonDimension>();
+  public static void getAllFilterDimensions(FilterResolverIntf filterResolverTree,
+      Set<CarbonDimension> filterDimensions, Set<CarbonMeasure> filterMeasure) {
     if (null == filterResolverTree) {
-      return filterDimensions;
+      return;
     }
     List<ColumnExpression> dimensionResolvedInfos = new ArrayList<ColumnExpression>();
     Expression filterExpression = filterResolverTree.getFilterExpression();
-    addColumnDimensions(filterExpression, filterDimensions);
+    addColumnDimensions(filterExpression, filterDimensions, filterMeasure);
     for (ColumnExpression info : dimensionResolvedInfos) {
       if (info.isDimension() && info.getDimension().getNumberOfChild() > 0) {
         filterDimensions.add(info.getDimension());
       }
     }
-    return filterDimensions;
-
   }
 
   /**
@@ -938,14 +947,53 @@ public class QueryUtil {
    * @return
    */
   private static void addColumnDimensions(Expression expression,
-      Set<CarbonDimension> filterDimensions) {
-    if (null != expression && expression instanceof ColumnExpression
-        && ((ColumnExpression) expression).isDimension()) {
-      filterDimensions.add(((ColumnExpression) expression).getDimension());
+      Set<CarbonDimension> filterDimensions, Set<CarbonMeasure> filterMeasure) {
+    if (null != expression && expression instanceof ColumnExpression) {
+      if (((ColumnExpression) expression).isDimension()) {
+        filterDimensions.add(((ColumnExpression) expression).getDimension());
+      } else {
+        filterMeasure.add((CarbonMeasure) ((ColumnExpression) expression).getCarbonColumn());
+      }
       return;
     }
     for (Expression child : expression.getChildren()) {
-      addColumnDimensions(child, filterDimensions);
+      addColumnDimensions(child, filterDimensions, filterMeasure);
+    }
+  }
+
+  private static Set<Integer> getFilterMeasureOrdinal(Set<CarbonMeasure> filterMeasures) {
+    Set<Integer> filterMeasuresOrdinal = new HashSet<>();
+    for (CarbonMeasure filterMeasure : filterMeasures) {
+      filterMeasuresOrdinal.add(filterMeasure.getOrdinal());
+    }
+    return filterMeasuresOrdinal;
+  }
+
+  private static Set<Integer> getFilterDimensionOrdinal(Set<CarbonDimension> filterDimensions) {
+    Set<Integer> filterDimensionsOrdinal = new HashSet<>();
+    for (CarbonDimension filterDimension : filterDimensions) {
+      filterDimensionsOrdinal.add(filterDimension.getOrdinal());
+      getChildDimensionOrdinal(filterDimension, filterDimensionsOrdinal);
+    }
+    return filterDimensionsOrdinal;
+  }
+
+  /**
+   * Below method will be used to fill the children dimension column id
+   *
+   * @param queryDimensions              query dimension
+   * @param dictionaryDimensionFromQuery dictionary dimension for query
+   */
+  private static void getChildDimensionOrdinal(CarbonDimension queryDimensions,
+      Set<Integer> filterDimensionsOrdinal) {
+    for (int j = 0; j < queryDimensions.numberOfChild(); j++) {
+      List<Encoding> encodingList = queryDimensions.getListOfChildDimensions().get(j).getEncoder();
+      if (queryDimensions.getListOfChildDimensions().get(j).numberOfChild() > 0) {
+        getChildDimensionOrdinal(queryDimensions.getListOfChildDimensions().get(j),
+            filterDimensionsOrdinal);
+      } else if (!CarbonUtil.hasEncoding(encodingList, Encoding.DIRECT_DICTIONARY)) {
+        filterDimensionsOrdinal.add(queryDimensions.getListOfChildDimensions().get(j).getOrdinal());
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java b/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
index caee061..80e4837 100644
--- a/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
+++ b/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
@@ -50,13 +50,12 @@ public abstract class AbstractBlockletScanner implements BlockletScanner {
 
   protected void fillKeyValue(BlocksChunkHolder blocksChunkHolder) {
     scannedResult.reset();
-    scannedResult.setMeasureChunks(blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedMeasureBlocksIndexes()));
     scannedResult.setNumberOfRows(blocksChunkHolder.getDataBlock().nodeSize());
-
     scannedResult.setDimensionChunks(blocksChunkHolder.getDataBlock()
         .getDimensionChunks(blocksChunkHolder.getFileReader(),
             blockExecutionInfo.getAllSelectedDimensionBlocksIndexes()));
+    scannedResult.setMeasureChunks(blocksChunkHolder.getDataBlock()
+            .getMeasureChunks(blocksChunkHolder.getFileReader(),
+                blockExecutionInfo.getAllSelectedMeasureBlocksIndexes()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java b/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java
index 3aafd42..ac71100 100644
--- a/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java
+++ b/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java
@@ -62,7 +62,7 @@ public class FilterScanner extends AbstractBlockletScanner {
   private QueryStatisticsModel queryStatisticsModel;
 
   public FilterScanner(BlockExecutionInfo blockExecutionInfo,
-                       QueryStatisticsModel queryStatisticsModel) {
+      QueryStatisticsModel queryStatisticsModel) {
     super(blockExecutionInfo);
     scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
     // to check whether min max is enabled or not
@@ -116,8 +116,8 @@ public class FilterScanner extends AbstractBlockletScanner {
     scannedResult.reset();
     QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
         .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    totalBlockletStatistic.addCountStatistic(
-        QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, totalBlockletStatistic.getCount() + 1);
+    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
+        totalBlockletStatistic.getCount() + 1);
     queryStatisticsModel.getRecorder().recordStatistics(totalBlockletStatistic);
     // apply min max
     if (isMinMaxEnabled) {
@@ -153,35 +153,41 @@ public class FilterScanner extends AbstractBlockletScanner {
     }
 
     FileHolder fileReader = blocksChunkHolder.getFileReader();
-    int[] allSelectedDimensionBlocksIndexes =
+    int[][] allSelectedDimensionBlocksIndexes =
         blockExecutionInfo.getAllSelectedDimensionBlocksIndexes();
+    DimensionColumnDataChunk[] projectionListDimensionChunk = blocksChunkHolder.getDataBlock()
+        .getDimensionChunks(fileReader, allSelectedDimensionBlocksIndexes);
+
     DimensionColumnDataChunk[] dimensionColumnDataChunk =
         new DimensionColumnDataChunk[blockExecutionInfo.getTotalNumberDimensionBlock()];
     // read dimension chunk blocks from file which is not present
+    for (int i = 0; i < dimensionColumnDataChunk.length; i++) {
+      if (null != blocksChunkHolder.getDimensionDataChunk()[i]) {
+        dimensionColumnDataChunk[i] = blocksChunkHolder.getDimensionDataChunk()[i];
+      }
+    }
     for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) {
-      if (null == blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]]) {
-        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getDimensionChunk(fileReader, allSelectedDimensionBlocksIndexes[i]);
-      } else {
-        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
-            blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]];
+      for (int j = allSelectedDimensionBlocksIndexes[i][0];
+           j <= allSelectedDimensionBlocksIndexes[i][1]; j++) {
+        dimensionColumnDataChunk[j] = projectionListDimensionChunk[j];
       }
     }
     MeasureColumnDataChunk[] measureColumnDataChunk =
         new MeasureColumnDataChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()];
-    int[] allSelectedMeasureBlocksIndexes = blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
-
+    int[][] allSelectedMeasureBlocksIndexes =
+        blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
+    MeasureColumnDataChunk[] projectionListMeasureChunk = blocksChunkHolder.getDataBlock()
+        .getMeasureChunks(fileReader, allSelectedMeasureBlocksIndexes);
     // read the measure chunk blocks which is not present
+    for (int i = 0; i < measureColumnDataChunk.length; i++) {
+      if (null != blocksChunkHolder.getMeasureDataChunk()[i]) {
+        measureColumnDataChunk[i] = blocksChunkHolder.getMeasureDataChunk()[i];
+      }
+    }
     for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) {
-
-      if (null == blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]]) {
-        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getMeasureChunk(fileReader, allSelectedMeasureBlocksIndexes[i]);
-      } else {
-        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
-            blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]];
+      for (int j = allSelectedMeasureBlocksIndexes[i][0];
+           j <= allSelectedMeasureBlocksIndexes[i][1]; j++) {
+        measureColumnDataChunk[j] = projectionListMeasureChunk[j];
       }
     }
     scannedResult.setDimensionChunks(dimensionColumnDataChunk);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfoTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfoTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfoTest.java
index 7aa50ad..eabf688 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfoTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfoTest.java
@@ -28,7 +28,7 @@ public class BlockInfoTest {
   static BlockInfo blockInfo;
 
   @BeforeClass public static void setup() {
-    blockInfo = new BlockInfo(new TableBlockInfo("filePath", 6, "segmentId", null, 6));
+    blockInfo = new BlockInfo(new TableBlockInfo("filePath", 6, "segmentId", null, 6,(short)1));
   }
 
   @Test public void hashCodeTest() {
@@ -44,7 +44,7 @@ public class BlockInfoTest {
 
   @Test public void equalsTestWithSimilarObject() {
     BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("filePath", 6, "segmentId", null, 6));
+        new BlockInfo(new TableBlockInfo("filePath", 6, "segmentId", null, 6, (short)1));
     Boolean res = blockInfo.equals(blockInfoTest);
     assert (res);
   }
@@ -61,28 +61,28 @@ public class BlockInfoTest {
 
   @Test public void equalsTestWithDifferentSegmentId() {
     BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("filePath", 6, "diffSegmentId", null, 6));
+        new BlockInfo(new TableBlockInfo("filePath", 6, "diffSegmentId", null, 6,(short)1));
     Boolean res = blockInfo.equals(blockInfoTest);
     assert (!res);
   }
 
   @Test public void equalsTestWithDifferentOffset() {
     BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("filePath", 62, "segmentId", null, 6));
+        new BlockInfo(new TableBlockInfo("filePath", 62, "segmentId", null, 6, (short)1));
     Boolean res = blockInfo.equals(blockInfoTest);
     assert (!res);
   }
 
   @Test public void equalsTestWithDifferentBlockLength() {
     BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("filePath", 6, "segmentId", null, 62));
+        new BlockInfo(new TableBlockInfo("filePath", 6, "segmentId", null, 62, (short)1));
     Boolean res = blockInfo.equals(blockInfoTest);
     assert (!res);
   }
 
   @Test public void equalsTestWithDiffFilePath() {
     BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("diffFilePath", 6, "segmentId", null, 62));
+        new BlockInfo(new TableBlockInfo("diffFilePath", 6, "segmentId", null, 62, (short)1));
     Boolean res = blockInfoTest.equals(blockInfo);
     assert (!res);
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfoTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfoTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfoTest.java
index 7c1bbed..1b49f83 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfoTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfoTest.java
@@ -34,8 +34,8 @@ public class TableBlockInfoTest {
   static TableBlockInfo tableBlockInfos;
 
   @BeforeClass public static void setup() {
-    tableBlockInfo = new TableBlockInfo("filePath", 4, "segmentId", null, 6);
-    tableBlockInfos = new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 2, 2));
+    tableBlockInfo = new TableBlockInfo("filePath", 4, "segmentId", null, 6, (short) 1);
+    tableBlockInfos = new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 2, 2), (short) 1);
   }
 
   @Test public void equalTestWithSameObject() {
@@ -44,7 +44,7 @@ public class TableBlockInfoTest {
   }
 
   @Test public void equalTestWithSimilarObject() {
-    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 4, "segmentId", null, 6);
+    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 4, "segmentId", null, 6, (short) 1);
     Boolean res = tableBlockInfo.equals(tableBlockInfoTest);
     assert (res);
   }
@@ -60,52 +60,52 @@ public class TableBlockInfoTest {
   }
 
   @Test public void equlsTestWithDiffSegmentId() {
-    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 4, "diffsegmentId", null, 6);
+    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 4, "diffsegmentId", null, 6, (short) 1);
     Boolean res = tableBlockInfo.equals(tableBlockInfoTest);
     assert (!res);
   }
 
   @Test public void equlsTestWithDiffBlockOffset() {
-    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 6, "segmentId", null, 6);
+    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 6, "segmentId", null, 6, (short) 1);
     Boolean res = tableBlockInfo.equals(tableBlockInfoTest);
     assert (!res);
   }
 
   @Test public void equalsTestWithDiffBlockLength() {
-    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 4, "segmentId", null, 4);
+    TableBlockInfo tableBlockInfoTest = new TableBlockInfo("filePath", 4, "segmentId", null, 4, (short) 1);
     Boolean res = tableBlockInfo.equals(tableBlockInfoTest);
     assert (!res);
   }
 
   @Test public void equalsTestWithDiffBlockletNumber() {
     TableBlockInfo tableBlockInfoTest =
-        new TableBlockInfo("filepath", 6, "segmentId", null, 6, new BlockletInfos(6, 3, 2));
+        new TableBlockInfo("filepath", 6, "segmentId", null, 6, new BlockletInfos(6, 3, 2), (short) 1);
     Boolean res = tableBlockInfos.equals(tableBlockInfoTest);
     assert (!res);
   }
 
   @Test public void equalsTestWithDiffFilePath() {
     TableBlockInfo tableBlockInfoTest =
-        new TableBlockInfo("difffilepath", 6, "segmentId", null, 6, new BlockletInfos(6, 3, 2));
+        new TableBlockInfo("difffilepath", 6, "segmentId", null, 6, new BlockletInfos(6, 3, 2), (short) 1);
     Boolean res = tableBlockInfos.equals(tableBlockInfoTest);
     assert (!res);
   }
 
   @Test public void compareToTestForSegmentId() {
     TableBlockInfo tableBlockInfo =
-        new TableBlockInfo("difffilepath", 6, "5", null, 6, new BlockletInfos(6, 3, 2));
+        new TableBlockInfo("difffilepath", 6, "5", null, 6, new BlockletInfos(6, 3, 2), (short) 1);
     int res = tableBlockInfos.compareTo(tableBlockInfo);
     int expectedResult = 2;
     assertEquals(res, expectedResult);
 
     TableBlockInfo tableBlockInfo1 =
-        new TableBlockInfo("difffilepath", 6, "6", null, 6, new BlockletInfos(6, 3, 2));
+        new TableBlockInfo("difffilepath", 6, "6", null, 6, new BlockletInfos(6, 3, 2), (short) 1);
     int res1 = tableBlockInfos.compareTo(tableBlockInfo1);
     int expectedResult1 = -1;
     assertEquals(res1, expectedResult1);
 
     TableBlockInfo tableBlockInfo2 =
-        new TableBlockInfo("difffilepath", 6, "4", null, 6, new BlockletInfos(6, 3, 2));
+        new TableBlockInfo("difffilepath", 6, "4", null, 6, new BlockletInfos(6, 3, 2), (short) 1);
     int res2 = tableBlockInfos.compareTo(tableBlockInfo2);
     int expectedresult2 = 1;
     assertEquals(res2, expectedresult2);
@@ -130,18 +130,18 @@ public class TableBlockInfoTest {
 
     };
 
-    TableBlockInfo tableBlockInfo = new TableBlockInfo("difffilepaths", 6, "5", null, 3);
+    TableBlockInfo tableBlockInfo = new TableBlockInfo("difffilepaths", 6, "5", null, 3, (short) 1);
     int res = tableBlockInfos.compareTo(tableBlockInfo);
     int expectedResult = -5;
     assertEquals(res, expectedResult);
 
-    TableBlockInfo tableBlockInfo1 = new TableBlockInfo("filepath", 6, "5", null, 3);
+    TableBlockInfo tableBlockInfo1 = new TableBlockInfo("filepath", 6, "5", null, 3, (short) 1);
     int res1 = tableBlockInfos.compareTo(tableBlockInfo1);
     int expectedResult1 = 1;
     assertEquals(res1, expectedResult1);
 
     TableBlockInfo tableBlockInfoTest =
-        new TableBlockInfo("filePath", 6, "5", null, 7, new BlockletInfos(6, 2, 2));
+        new TableBlockInfo("filePath", 6, "5", null, 7, new BlockletInfos(6, 2, 2), (short) 1);
     int res2 = tableBlockInfos.compareTo(tableBlockInfoTest);
     int expectedResult2 = -1;
     assertEquals(res2, expectedResult2);
@@ -149,13 +149,13 @@ public class TableBlockInfoTest {
 
   @Test public void compareToTestWithStartBlockletNo() {
     TableBlockInfo tableBlockInfo =
-        new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 3, 2));
+        new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 3, 2), (short) 1);
     int res = tableBlockInfos.compareTo(tableBlockInfo);
     int expectedresult =-1;
     assertEquals(res, expectedresult);
 
     TableBlockInfo tableBlockInfo1 =
-        new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 1, 2));
+        new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 1, 2), (short) 1);
     int res1 = tableBlockInfos.compareTo(tableBlockInfo1);
     int expectedresult1 = 1;
     assertEquals(res1, expectedresult1);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfoTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfoTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfoTest.java
index 2d5750b..83b62a5 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfoTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfoTest.java
@@ -34,10 +34,10 @@ public class TableTaskInfoTest {
     tableBlockInfoList = new ArrayList<>(5);
 
     String[] locations = { "loc1", "loc2", "loc3" };
-    tableBlockInfoList.add(0, new TableBlockInfo("filePath", 2, "segmentID", locations, 6));
+    tableBlockInfoList.add(0, new TableBlockInfo("filePath", 2, "segmentID", locations, 6, (short) 1));
 
     String[] locs = { "loc4", "loc5" };
-    tableBlockInfoList.add(1, new TableBlockInfo("filepath", 2, "segmentId", locs, 6));
+    tableBlockInfoList.add(1, new TableBlockInfo("filepath", 2, "segmentId", locs, 6, (short) 1));
 
     tableTaskInfo = new TableTaskInfo("taskId", tableBlockInfoList);
   }
@@ -68,10 +68,10 @@ public class TableTaskInfoTest {
     List<TableBlockInfo> tableBlockInfoListTest = new ArrayList<>();
 
     String[] locations = { "loc1", "loc2", "loc3" };
-    tableBlockInfoListTest.add(0, new TableBlockInfo("filePath", 2, "segmentID", locations, 6));
+    tableBlockInfoListTest.add(0, new TableBlockInfo("filePath", 2, "segmentID", locations, 6, (short) 1));
 
     String[] locations1 = { "loc1", "loc2", "loc3" };
-    tableBlockInfoListTest.add(1, new TableBlockInfo("filePath", 2, "segmentID", locations1, 6));
+    tableBlockInfoListTest.add(1, new TableBlockInfo("filePath", 2, "segmentID", locations1, 6, (short) 1));
 
     List<String> res = TableTaskInfo.maxNoNodes(tableBlockInfoListTest);
     assert (res.equals(locs));

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReaderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReaderTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReaderTest.java
index 364a393..e1e4088 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReaderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReaderTest.java
@@ -19,10 +19,17 @@
 
 package org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension;
 
+import static junit.framework.TestCase.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
 
+import mockit.Mock;
+import mockit.MockUp;
+
 import org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.v1.CompressedDimensionChunkFileBasedReaderV1;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
@@ -30,17 +37,12 @@ import org.apache.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
 import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression;
 import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.util.CarbonUtil;
-
-import mockit.Mock;
-import mockit.MockUp;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static junit.framework.TestCase.assertEquals;
-
 public class CompressedDimensionChunkFileBasedReaderTest {
 
-  static CompressedDimensionChunkFileBasedReader compressedDimensionChunkFileBasedReader;
+  static CompressedDimensionChunkFileBasedReaderV1 compressedDimensionChunkFileBasedReader;
   static List<DataChunk> dataChunkList;
 
   @BeforeClass public static void setup() {
@@ -49,9 +51,10 @@ public class CompressedDimensionChunkFileBasedReaderTest {
 
     DataChunk dataChunk = new DataChunk();
     dataChunkList.add(dataChunk);
-
+    BlockletInfo info = new BlockletInfo();
+    info.setDimensionColumnChunk(dataChunkList);
     compressedDimensionChunkFileBasedReader =
-        new CompressedDimensionChunkFileBasedReader(dataChunkList, eachColumnBlockSize, "filePath");
+        new CompressedDimensionChunkFileBasedReaderV1(info, eachColumnBlockSize, "filePath");
   }
 
   @Test public void readDimensionChunksTest() {
@@ -88,7 +91,7 @@ public class CompressedDimensionChunkFileBasedReaderTest {
       }
     };
 
-    int blockIndexes[] = { 0 };
+    int[][] blockIndexes = {{0,0}};
     DimensionColumnDataChunk dimensionColumnDataChunk[] =
         compressedDimensionChunkFileBasedReader.readDimensionChunks(fileHolder, blockIndexes);
     byte expectedResult[] = { 1 };
@@ -137,7 +140,7 @@ public class CompressedDimensionChunkFileBasedReaderTest {
         return true;
       }
     };
-    int blockIndexes[] = { 0 };
+    int[][] blockIndexes = {{0,0}};
     DimensionColumnDataChunk dimensionColumnDataChunk[] =
         compressedDimensionChunkFileBasedReader.readDimensionChunks(fileHolder, blockIndexes);
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReaderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReaderTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReaderTest.java
index ea4eb23..e7342b1 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReaderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/CompressedMeasureChunkFileBasedReaderTest.java
@@ -1,27 +1,30 @@
 package org.apache.carbondata.core.carbon.datastore.chunk.reader.measure;
 
+import static junit.framework.TestCase.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
 
+import mockit.Mock;
+import mockit.MockUp;
+
 import org.apache.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.v1.CompressedMeasureChunkFileBasedReaderV1;
+import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
 import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
 import org.apache.carbondata.core.datastorage.store.compression.type.UnCompressByteArray;
 import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.util.ValueCompressionUtil;
-
-import mockit.Mock;
-import mockit.MockUp;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static junit.framework.TestCase.assertEquals;
-
 public class CompressedMeasureChunkFileBasedReaderTest {
 
-  static CompressedMeasureChunkFileBasedReader compressedMeasureChunkFileBasedReader;
+  static CompressedMeasureChunkFileBasedReaderV1 compressedMeasureChunkFileBasedReader;
 
   @BeforeClass public static void setup() {
     List<DataChunk> dataChunkList = new ArrayList<>();
@@ -41,9 +44,18 @@ public class CompressedMeasureChunkFileBasedReaderTest {
     valueCompressionModel.setDecimal(decimal);
     Object maxValue[] = { 8 };
     valueCompressionModel.setMaxValue(maxValue);
-
+    ValueEncoderMeta meta = new ValueEncoderMeta();
+    meta.setMaxValue(8.0);
+    meta.setMinValue(1.0);
+    meta.setDecimal(1);
+    meta.setType('b');
+    List<ValueEncoderMeta> valueEncoderMetaList = new ArrayList<>();
+    valueEncoderMetaList.add(meta);
+    dataChunkList.get(0).setValueEncoderMeta(valueEncoderMetaList);
+    BlockletInfo info = new BlockletInfo();
+    info.setMeasureColumnChunk(dataChunkList);
     compressedMeasureChunkFileBasedReader =
-        new CompressedMeasureChunkFileBasedReader(dataChunkList, valueCompressionModel, "filePath");
+        new CompressedMeasureChunkFileBasedReaderV1(info, "filePath");
   }
 
   @Test public void readMeasureChunkTest() {
@@ -98,7 +110,7 @@ public class CompressedMeasureChunkFileBasedReaderTest {
       }
     };
 
-    int blockIndexes[] = { 0 };
+    int[][] blockIndexes = {{0,0}};
     MeasureColumnDataChunk measureColumnDataChunks[] =
         compressedMeasureChunkFileBasedReader.readMeasureChunks(fileHolder, blockIndexes);
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
index 39123e2..d959a5c 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.carbon.metadata.blocklet.index.*;
 import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.carbon.metadata.index.BlockIndexInfo;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.*;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.apache.carbondata.core.metadata.BlockletInfoColumnar;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
@@ -34,7 +35,6 @@ import org.apache.carbondata.format.*;
 import org.apache.carbondata.format.BlockletBTreeIndex;
 import org.apache.carbondata.format.BlockletMinMaxIndex;
 import org.apache.carbondata.format.ColumnSchema;
-
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -166,6 +166,7 @@ public class CarbonMetadataUtilTest {
     segmentInfo.setNum_cols(0);
     segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(columnCardinality));
     IndexHeader indexHeader = new IndexHeader();
+    indexHeader.setVersion(CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION);
     indexHeader.setSegment_info(segmentInfo);
     indexHeader.setTable_columns(columnSchemaList);
     IndexHeader indexheaderResult = getIndexHeader(columnCardinality, columnSchemaList);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index 54910d8..c0d890c 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -20,6 +20,8 @@ package org.apache.carbondata.core.util;
 
 import mockit.Mock;
 import mockit.MockUp;
+
+import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
 import org.apache.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
@@ -41,11 +43,13 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.pentaho.di.core.exception.KettleException;
+
 import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+
 import static org.hamcrest.MatcherAssert.assertThat;
 import static junit.framework.TestCase.*;
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -549,19 +553,21 @@ public class CarbonUtilTest {
   @Test public void testToReadMetadatFile() throws CarbonUtilException {
     new MockUp<DataFileFooterConverter>() {
       @SuppressWarnings("unused") @Mock
-      public DataFileFooter readDataFileFooter(String filePath, long blockOffset,
-          long blockLength) {
+      public DataFileFooter readDataFileFooter(TableBlockInfo info) {
         DataFileFooter fileFooter = new DataFileFooter();
-        fileFooter.setVersionId(1);
+        fileFooter.setVersionId((short)1);
         return fileFooter;
       }
     };
-    assertEquals(CarbonUtil.readMetadatFile("", 1L, 1L).getVersionId(), 1);
+    TableBlockInfo info = new TableBlockInfo("file:/", 1, "0", new String[0], 1, (short)1);
+    
+    assertEquals(CarbonUtil.readMetadatFile(info).getVersionId(), 1);
   }
 
   @Test(expected = CarbonUtilException.class) public void testToReadMetadatFileWithException()
       throws Exception {
-    CarbonUtil.readMetadatFile("", 1L, 1L);
+	TableBlockInfo info = new TableBlockInfo("file:/", 1, "0", new String[0], 1, (short)1);
+    CarbonUtil.readMetadatFile(info);
   }
 
   @Test public void testToFindDimension() {
@@ -695,7 +701,7 @@ public class CarbonUtilTest {
     dataChunk.setValueEncoderMeta(valueEncoderMetas);
     dataChunkList.add(dataChunk);
     ValueCompressionModel valueCompressionModel =
-        CarbonUtil.getValueCompressionModel(dataChunkList);
+        CarbonUtil.getValueCompressionModel(dataChunkList.get(0).getValueEncoderMeta());
     assertEquals(1, valueCompressionModel.getMaxValue().length);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
index 1030d90..62d1ac7 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
@@ -152,7 +152,7 @@ public class DataFileFooterConverterTest {
       }
     };
     String[] arr = { "a", "b", "c" };
-    TableBlockInfo tableBlockInfo = new TableBlockInfo("file", 3, "id", arr, 3);
+    TableBlockInfo tableBlockInfo = new TableBlockInfo("file", 3, "id", arr, 3, (short) 1);
     tableBlockInfo.getBlockletInfos().setNoOfBlockLets(3);
     List<TableBlockInfo> tableBlockInfoList = new ArrayList<>();
     tableBlockInfoList.add(tableBlockInfo);
@@ -214,9 +214,9 @@ public class DataFileFooterConverterTest {
         new org.apache.carbondata.format.BlockletInfo();
     List<org.apache.carbondata.format.BlockletInfo> blockletInfoArrayList = new ArrayList<>();
     blockletInfoArrayList.add(blockletInfo);
-    final FileFooter fileFooter =
-        new FileFooter(1, 3, columnSchemas, segmentInfo1, blockletIndexArrayList,
-            blockletInfoArrayList);
+    final FileFooter fileFooter = 
+        new FileFooter(1, 3, columnSchemas, segmentInfo1, blockletIndexArrayList);
+    fileFooter.setBlocklet_info_list(blockletInfoArrayList);
     BlockletBTreeIndex blockletBTreeIndex = new BlockletBTreeIndex();
     blockletBTreeIndex.setStart_key("1".getBytes());
     blockletBTreeIndex.setEnd_key("3".getBytes());
@@ -254,7 +254,8 @@ public class DataFileFooterConverterTest {
     segmentInfo.setNumberOfColumns(segmentInfo1.getNum_cols());
     dataFileFooter.setNumberOfRows(3);
     dataFileFooter.setSegmentInfo(segmentInfo);
-    DataFileFooter result = dataFileFooterConverter.readDataFileFooter("file", 1, 1);
+    TableBlockInfo info = new TableBlockInfo("file", 1, "0", new String[0], 1, (short)1);
+    DataFileFooter result = dataFileFooterConverter.readDataFileFooter(info);
     assertEquals(result.getNumberOfRows(), 3);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/format/src/main/thrift/carbondata.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/carbondata.thrift b/format/src/main/thrift/carbondata.thrift
index dd4514d..759fbf7 100644
--- a/format/src/main/thrift/carbondata.thrift
+++ b/format/src/main/thrift/carbondata.thrift
@@ -113,6 +113,22 @@ struct DataChunk{
     13: optional list<binary> encoder_meta; // extra information required by encoders
 }
 
+/**
+* Represents a chunk of data. The chunk can be a single column stored in Column Major format or a group of columns stored in Row Major Format.
+**/
+struct DataChunk2{
+    1: required ChunkCompressionMeta chunk_meta; // the metadata of a chunk
+    2: required bool rowMajor; // whether this chunk is a row chunk or column chunk ? Decide whether this can be replace with counting od columnIDs
+	/** The column IDs in this chunk, in the order in which the data is physically stored, will have atleast one column ID for columnar format, many column ID for row major format**/
+    3: required i32 data_page_length; // length of data page
+    4: optional i32 rowid_page_length; //length of row id page, only if encoded using inverted index
+    5: optional i32 rle_page_length;	// length of rle page, only if RLE coded.
+    6: optional PresenceMeta presence; // information about presence of values in each row of this column chunk
+    7: optional SortState sort_state;
+    8: optional list<schema.Encoding> encoders; // The List of encoders overriden at node level
+    9: optional list<binary> encoder_meta; // extra information required by encoders
+}
+
 
 /**
 *	Information about a blocklet
@@ -123,6 +139,15 @@ struct BlockletInfo{
 }
 
 /**
+*	Information about a blocklet
+*/
+struct BlockletInfo2{
+    1: required i32 num_rows;	// Number of rows in this blocklet
+    2: required list<i64> column_data_chunks_offsets;	// Information about offsets all column chunks in this blocklet
+    3: required list<i16> column_data_chunks_length;	// Information about length all column chunks in this blocklet
+}
+
+/**
 * Footer for indexed carbon file
 */
 struct FileFooter{
@@ -131,8 +156,9 @@ struct FileFooter{
     3: required list<schema.ColumnSchema> table_columns;	// Description of columns in this file
     4: required SegmentInfo segment_info;	// Segment info (will be same/repeated for all files in this segment)
     5: required list<BlockletIndex> blocklet_index_list;	// blocklet index of all blocklets in this file
-    6: required list<BlockletInfo> blocklet_info_list;	// Information about blocklets of all columns in this file
-    7: optional dictionary.ColumnDictionaryChunk dictionary; // blocklet local dictionary
+    6: optional list<BlockletInfo> blocklet_info_list;	// Information about blocklets of all columns in this file
+    7: optional list<BlockletInfo2> blocklet_info_list2;	// Information about blocklets of all columns in this file
+    8: optional dictionary.ColumnDictionaryChunk dictionary; // blocklet local dictionary
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
index a44a78c..8b453c7 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
@@ -82,6 +82,7 @@ import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.util.StringUtils;
 
+
 /**
  * Carbon Input format class representing one carbon table
  */
@@ -180,8 +181,17 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
    * Set list of segments to access
    */
   public static void setSegmentsToAccess(Configuration configuration, List<String> validSegments) {
-    configuration.set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS,
-        CarbonUtil.getSegmentString(validSegments));
+    configuration
+        .set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, CarbonUtil.getSegmentString(validSegments));
+  }
+
+  private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration) {
+    String dirs = configuration.get(INPUT_DIR, "");
+    String[] inputPaths = StringUtils.split(dirs);
+    if (inputPaths.length == 0) {
+      throw new InvalidPathException("No input paths specified in job");
+    }
+    return AbsoluteTableIdentifier.fromTablePath(inputPaths[0]);
   }
 
   /**
@@ -193,8 +203,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
    * @return List<InputSplit> list of CarbonInputSplit
    * @throws IOException
    */
-  @Override
-  public List<InputSplit> getSplits(JobContext job) throws IOException {
+  @Override public List<InputSplit> getSplits(JobContext job) throws IOException {
     AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
     List<String> invalidSegments = new ArrayList<>();
 
@@ -245,7 +254,8 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       if (segmentId.equals(CarbonCommonConstants.INVALID_SEGMENT_ID)) {
         continue;
       }
-      carbonSplits.add(CarbonInputSplit.from(segmentId, fileSplit));
+      carbonSplits.add(CarbonInputSplit
+          .from(segmentId, fileSplit, CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION));
     }
     return carbonSplits;
   }
@@ -278,21 +288,13 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
         TableBlockInfo tableBlockInfo = leafNode.getTableBlockInfo();
         result.add(new CarbonInputSplit(segmentNo, new Path(tableBlockInfo.getFilePath()),
             tableBlockInfo.getBlockOffset(), tableBlockInfo.getBlockLength(),
-            tableBlockInfo.getLocations(), tableBlockInfo.getBlockletInfos().getNoOfBlockLets()));
+            tableBlockInfo.getLocations(), tableBlockInfo.getBlockletInfos().getNoOfBlockLets(),
+            tableBlockInfo.getVersion()));
       }
     }
     return result;
   }
 
-  private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration) {
-    String dirs = configuration.get(INPUT_DIR, "");
-    String[] inputPaths = StringUtils.split(dirs);
-    if (inputPaths.length == 0) {
-      throw new InvalidPathException("No input paths specified in job");
-    }
-    return AbsoluteTableIdentifier.fromTablePath(inputPaths[0]);
-  }
-
   private Expression getFilterPredicates(Configuration configuration) {
     try {
       String filterExprString = configuration.get(FILTER_PREDICATE);
@@ -313,8 +315,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       FilterExpressionProcessor filterExpressionProcessor,
       AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver,
       String segmentId) throws IndexBuilderException, IOException {
-    QueryStatisticsRecorder recorder =
-            CarbonTimeStatisticsFactory.createDriverRecorder();
+    QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
     QueryStatistic statistic = new QueryStatistic();
     Map<String, AbstractIndex> segmentIndexMap =
         getSegmentAbstractIndexs(job, absoluteTableIdentifier, segmentId);
@@ -340,8 +341,8 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       }
       resultFilterredBlocks.addAll(filterredBlocks);
     }
-    statistic.addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER,
-        System.currentTimeMillis());
+    statistic
+        .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
     recorder.recordStatisticsForDriver(statistic, job.getConfiguration().get("query.id"));
     return resultFilterredBlocks;
   }
@@ -349,8 +350,8 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   /**
    * Below method will be used to get the table block info
    *
-   * @param job                     job context
-   * @param segmentId               number of segment id
+   * @param job       job context
+   * @param segmentId number of segment id
    * @return list of table block
    * @throws IOException
    */
@@ -371,7 +372,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       tableBlockInfoList.add(
           new TableBlockInfo(carbonInputSplit.getPath().toString(), carbonInputSplit.getStart(),
               segmentId, carbonInputSplit.getLocations(), carbonInputSplit.getLength(),
-              blockletInfos));
+              blockletInfos, carbonInputSplit.getVersion()));
     }
     return tableBlockInfoList;
   }
@@ -384,8 +385,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
 
     // if segment tree is not loaded, load the segment tree
     if (segmentIndexMap == null) {
-      List<TableBlockInfo> tableBlockInfoList =
-          getTableBlockInfo(job, segmentId);
+      List<TableBlockInfo> tableBlockInfoList = getTableBlockInfo(job, segmentId);
 
       Map<String, List<TableBlockInfo>> segmentToTableBlocksInfos = new HashMap<>();
       segmentToTableBlocksInfos.put(segmentId, tableBlockInfoList);
@@ -428,8 +428,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     return blocks;
   }
 
-  @Override
-  public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
+  @Override public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
       TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     CarbonTable carbonTable = getCarbonTable(configuration);
@@ -482,18 +481,15 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     return readSupport;
   }
 
-  @Override
-  protected long computeSplitSize(long blockSize, long minSize, long maxSize) {
+  @Override protected long computeSplitSize(long blockSize, long minSize, long maxSize) {
     return super.computeSplitSize(blockSize, minSize, maxSize);
   }
 
-  @Override
-  protected int getBlockIndex(BlockLocation[] blkLocations, long offset) {
+  @Override protected int getBlockIndex(BlockLocation[] blkLocations, long offset) {
     return super.getBlockIndex(blkLocations, offset);
   }
 
-  @Override
-  protected List<FileStatus> listStatus(JobContext job) throws IOException {
+  @Override protected List<FileStatus> listStatus(JobContext job) throws IOException {
     List<FileStatus> result = new ArrayList<FileStatus>();
     String[] segmentsToConsider = getSegmentsToAccess(job);
     if (segmentsToConsider.length == 0) {
@@ -504,13 +500,11 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     return result;
   }
 
-  @Override
-  protected boolean isSplitable(JobContext context, Path filename) {
+  @Override protected boolean isSplitable(JobContext context, Path filename) {
     try {
       // Don't split the file if it is local file system
       FileSystem fileSystem = filename.getFileSystem(context.getConfiguration());
-      if (fileSystem instanceof LocalFileSystem)
-      {
+      if (fileSystem instanceof LocalFileSystem) {
         return false;
       }
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
index 132ee43..efc4f77 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
@@ -29,6 +29,7 @@ import org.apache.carbondata.core.carbon.datastore.block.BlockletInfos;
 import org.apache.carbondata.core.carbon.datastore.block.Distributable;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.path.CarbonTablePath;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.hadoop.internal.index.Block;
 
 import org.apache.hadoop.fs.Path;
@@ -38,13 +39,12 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 /**
  * Carbon input split to allow distributed read of CarbonInputFormat.
  */
-public class CarbonInputSplit extends FileSplit implements Distributable, Serializable, Writable,
-    Block {
+public class CarbonInputSplit extends FileSplit
+    implements Distributable, Serializable, Writable, Block {
 
   private static final long serialVersionUID = 3520344046772190207L;
-  private String segmentId;
   public String taskId;
-
+  private String segmentId;
   /*
    * Invalid segments that need to be removed in task side index
    */
@@ -55,40 +55,72 @@ public class CarbonInputSplit extends FileSplit implements Distributable, Serial
    */
   private int numberOfBlocklets;
 
-  public  CarbonInputSplit() {
+  private short version = CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION;
+
+  public CarbonInputSplit() {
     segmentId = null;
     taskId = "0";
     numberOfBlocklets = 0;
     invalidSegments = new ArrayList<>();
   }
 
-  private CarbonInputSplit(String segmentId, Path path, long start, long length,
-      String[] locations) {
+  private CarbonInputSplit(String segmentId, Path path, long start, long length, String[] locations,
+      short version) {
     super(path, start, length, locations);
     this.segmentId = segmentId;
     this.taskId = CarbonTablePath.DataFileUtil.getTaskNo(path.getName());
     this.invalidSegments = new ArrayList<>();
+    this.version = version;
   }
 
-  public CarbonInputSplit(String segmentId, Path path, long start, long length,
-      String[] locations, int numberOfBlocklets) {
-    this(segmentId, path, start, length, locations);
+  public CarbonInputSplit(String segmentId, Path path, long start, long length, String[] locations,
+      int numberOfBlocklets, short version) {
+    this(segmentId, path, start, length, locations, version);
     this.numberOfBlocklets = numberOfBlocklets;
   }
 
-  public static CarbonInputSplit from(String segmentId, FileSplit split) throws IOException {
+  public static CarbonInputSplit from(String segmentId, FileSplit split, short version)
+      throws IOException {
     return new CarbonInputSplit(segmentId, split.getPath(), split.getStart(), split.getLength(),
-        split.getLocations());
+        split.getLocations(), version);
+  }
+
+  public static List<TableBlockInfo> createBlocks(List<CarbonInputSplit> splitList) {
+    List<TableBlockInfo> tableBlockInfoList = new ArrayList<>();
+    for (CarbonInputSplit split : splitList) {
+      BlockletInfos blockletInfos =
+          new BlockletInfos(split.getNumberOfBlocklets(), 0, split.getNumberOfBlocklets());
+      try {
+        tableBlockInfoList.add(
+            new TableBlockInfo(split.getPath().toString(), split.getStart(), split.getSegmentId(),
+                split.getLocations(), split.getLength(), blockletInfos, split.getVersion()));
+      } catch (IOException e) {
+        throw new RuntimeException("fail to get location of split: " + split, e);
+      }
+    }
+    return tableBlockInfoList;
+  }
+
+  public static TableBlockInfo getTableBlockInfo(CarbonInputSplit inputSplit) {
+    BlockletInfos blockletInfos =
+        new BlockletInfos(inputSplit.getNumberOfBlocklets(), 0, inputSplit.getNumberOfBlocklets());
+    try {
+      return new TableBlockInfo(inputSplit.getPath().toString(), inputSplit.getStart(),
+          inputSplit.getSegmentId(), inputSplit.getLocations(), inputSplit.getLength(),
+          blockletInfos, inputSplit.getVersion());
+    } catch (IOException e) {
+      throw new RuntimeException("fail to get location of split: " + inputSplit, e);
+    }
   }
 
   public String getSegmentId() {
     return segmentId;
   }
 
-  @Override
-  public void readFields(DataInput in) throws IOException {
+  @Override public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     this.segmentId = in.readUTF();
+    this.version = in.readShort();
     int numInvalidSegment = in.readInt();
     invalidSegments = new ArrayList<>(numInvalidSegment);
     for (int i = 0; i < numInvalidSegment; i++) {
@@ -96,17 +128,17 @@ public class CarbonInputSplit extends FileSplit implements Distributable, Serial
     }
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
+  @Override public void write(DataOutput out) throws IOException {
     super.write(out);
     out.writeUTF(segmentId);
+    out.writeShort(version);
     out.writeInt(invalidSegments.size());
-    for (String invalidSegment: invalidSegments) {
+    for (String invalidSegment : invalidSegments) {
       out.writeUTF(invalidSegment);
     }
   }
 
-  public List<String> getInvalidSegments(){
+  public List<String> getInvalidSegments() {
     return invalidSegments;
   }
 
@@ -116,15 +148,23 @@ public class CarbonInputSplit extends FileSplit implements Distributable, Serial
 
   /**
    * returns the number of blocklets
+   *
    * @return
    */
   public int getNumberOfBlocklets() {
     return numberOfBlocklets;
   }
 
-  @Override
-  public int compareTo(Distributable o) {
-    CarbonInputSplit other = (CarbonInputSplit)o;
+  public short getVersion() {
+    return version;
+  }
+
+  public void setVersion(short version) {
+    this.version = version;
+  }
+
+  @Override public int compareTo(Distributable o) {
+    CarbonInputSplit other = (CarbonInputSplit) o;
     int compareResult = 0;
     // get the segment id
     // converr seg ID to double.
@@ -163,34 +203,15 @@ public class CarbonInputSplit extends FileSplit implements Distributable, Serial
     return 0;
   }
 
-  public static List<TableBlockInfo> createBlocks(List<CarbonInputSplit> splitList) {
-    List<TableBlockInfo> tableBlockInfoList = new ArrayList<>();
-    for (CarbonInputSplit split : splitList) {
-      BlockletInfos blockletInfos = new BlockletInfos(split.getNumberOfBlocklets(), 0,
-          split.getNumberOfBlocklets());
-      try {
-        tableBlockInfoList.add(
-            new TableBlockInfo(split.getPath().toString(), split.getStart(), split.getSegmentId(),
-                split.getLocations(), split.getLength(), blockletInfos));
-      } catch (IOException e) {
-        throw new RuntimeException("fail to get location of split: " + split, e);
-      }
-    }
-    return tableBlockInfoList;
-  }
-
-  @Override
-  public String getBlockPath() {
+  @Override public String getBlockPath() {
     return getPath().getName();
   }
 
-  @Override
-  public List<Long> getMatchedBlocklets() {
+  @Override public List<Long> getMatchedBlocklets() {
     return null;
   }
 
-  @Override
-  public boolean fullScan() {
+  @Override public boolean fullScan() {
     return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
index dbc71ec..c238e10 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
@@ -94,7 +94,8 @@ class InMemoryBTreeIndex implements Index {
       TableBlockInfo tableBlockInfo = leafNode.getTableBlockInfo();
       result.add(new CarbonInputSplit(segment.getId(), new Path(tableBlockInfo.getFilePath()),
           tableBlockInfo.getBlockOffset(), tableBlockInfo.getBlockLength(),
-          tableBlockInfo.getLocations(), tableBlockInfo.getBlockletInfos().getNoOfBlockLets()));
+          tableBlockInfo.getLocations(), tableBlockInfo.getBlockletInfos().getNoOfBlockLets(),
+          tableBlockInfo.getVersion()));
     }
     return result;
   }
@@ -138,7 +139,7 @@ class InMemoryBTreeIndex implements Index {
       tableBlockInfoList.add(
           new TableBlockInfo(carbonInputSplit.getPath().toString(), carbonInputSplit.getStart(),
               segment.getId(), carbonInputSplit.getLocations(), carbonInputSplit.getLength(),
-              blockletInfos));
+              blockletInfos, carbonInputSplit.getVersion()));
     }
     return tableBlockInfoList;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
index 753f485..404dd1a 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
@@ -113,14 +113,12 @@ public class CarbonCompactionUtil {
     for (TableBlockInfo blockInfo : tableBlockInfoList) {
       List<DataFileFooter> eachSegmentBlocks = new ArrayList<>();
       String segId = blockInfo.getSegmentId();
-
       DataFileFooter dataFileMatadata = null;
       // check if segId is already present in map
       List<DataFileFooter> metadataList = segmentBlockInfoMapping.get(segId);
       try {
         dataFileMatadata = CarbonUtil
-            .readMetadatFile(blockInfo.getFilePath(), blockInfo.getBlockOffset(),
-                blockInfo.getBlockLength());
+            .readMetadatFile(blockInfo);
       } catch (CarbonUtilException e) {
         throw new IndexBuilderException(e);
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index c9e3b6b..93e1590 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -250,8 +250,8 @@ class CarbonMergerRDD[K, V](
       var dataFileFooter: DataFileFooter = null
 
       try {
-        dataFileFooter = CarbonUtil.readMetadatFile(carbonInputSplit.getPath.toString(),
-          carbonInputSplit.getStart, carbonInputSplit.getLength)
+        dataFileFooter = CarbonUtil.readMetadatFile(
+            CarbonInputSplit.getTableBlockInfo(carbonInputSplit))
       } catch {
         case e: CarbonUtilException =>
           logError("Exception in preparing the data file footer for compaction " + e.getMessage)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index a750493..1801408 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -805,7 +805,7 @@ object CarbonDataRDDFactory {
             val fileSplit = inputSplit.asInstanceOf[FileSplit]
             new TableBlockInfo(fileSplit.getPath.toString,
               fileSplit.getStart, "1",
-              fileSplit.getLocations, fileSplit.getLength
+              fileSplit.getLocations, fileSplit.getLength, 0
             ).asInstanceOf[Distributable]
           }
           )

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/integration/spark/src/test/resources/OLDFORMATTABLE.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/OLDFORMATTABLE.csv b/integration/spark/src/test/resources/OLDFORMATTABLE.csv
new file mode 100644
index 0000000..12a6fe8
--- /dev/null
+++ b/integration/spark/src/test/resources/OLDFORMATTABLE.csv
@@ -0,0 +1,34 @@
+country,name,phonetype,serialname,salary
+china,aaa1,phone197,ASD69643,15000
+china,aaa2,phone756,ASD42892,15001
+china,aaa3,phone1904,ASD37014,15002
+china,aaa4,phone2435,ASD66902,15003
+china,aaa5,phone2441,ASD90633,15004
+china,aaa6,phone294,ASD59961,15005
+china,aaa7,phone610,ASD14875,15006
+china,aaa8,phone1848,ASD57308,15007
+china,aaa9,phone706,ASD86717,15008
+usa,aaa10,phone685,ASD30505,15009
+china,aaa11,phone1554,ASD26101,15010
+china,aaa12,phone1781,ASD85711,15011
+china,aaa13,phone943,ASD39200,15012
+china,aaa14,phone1954,ASD80468,15013
+china,aaa15,phone451,ASD1954,15014
+china,aaa16,phone390,ASD38513,15015
+china,aaa17,phone1929,ASD86213,15016
+usa,aaa18,phone910,ASD88812,15017
+china,aaa19,phone2151,ASD9316,15018
+china,aaa20,phone2625,ASD62597,15019
+china,aaa21,phone1371,ASD27896,15020
+china,aaa22,phone945,ASD79760,15021
+china,aaa23,phone2177,ASD45410,15022
+china,aaa24,phone1586,ASD80645,15023
+china,aaa25,phone1310,ASD36408,15024
+china,aaa26,phone1579,ASD14571,15025
+china,aaa27,phone2123,ASD36243,15026
+china,aaa28,phone2334,ASD57825,15027
+china,aaa29,phone1166,ASD26161,15028
+china,aaa30,phone2248,ASD47899,15029
+china,aaa31,phone475,ASD89811,15030
+china,aaa32,phone2499,ASD87974,15031
+china,aaa33,phone2333,ASD62408,15032
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/integration/spark/src/test/resources/OLDFORMATTABLEHIVE.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/OLDFORMATTABLEHIVE.csv b/integration/spark/src/test/resources/OLDFORMATTABLEHIVE.csv
new file mode 100644
index 0000000..55642aa
--- /dev/null
+++ b/integration/spark/src/test/resources/OLDFORMATTABLEHIVE.csv
@@ -0,0 +1,33 @@
+china,aaa1,phone197,ASD69643,15000
+china,aaa2,phone756,ASD42892,15001
+china,aaa3,phone1904,ASD37014,15002
+china,aaa4,phone2435,ASD66902,15003
+china,aaa5,phone2441,ASD90633,15004
+china,aaa6,phone294,ASD59961,15005
+china,aaa7,phone610,ASD14875,15006
+china,aaa8,phone1848,ASD57308,15007
+china,aaa9,phone706,ASD86717,15008
+usa,aaa10,phone685,ASD30505,15009
+china,aaa11,phone1554,ASD26101,15010
+china,aaa12,phone1781,ASD85711,15011
+china,aaa13,phone943,ASD39200,15012
+china,aaa14,phone1954,ASD80468,15013
+china,aaa15,phone451,ASD1954,15014
+china,aaa16,phone390,ASD38513,15015
+china,aaa17,phone1929,ASD86213,15016
+usa,aaa18,phone910,ASD88812,15017
+china,aaa19,phone2151,ASD9316,15018
+china,aaa20,phone2625,ASD62597,15019
+china,aaa21,phone1371,ASD27896,15020
+china,aaa22,phone945,ASD79760,15021
+china,aaa23,phone2177,ASD45410,15022
+china,aaa24,phone1586,ASD80645,15023
+china,aaa25,phone1310,ASD36408,15024
+china,aaa26,phone1579,ASD14571,15025
+china,aaa27,phone2123,ASD36243,15026
+china,aaa28,phone2334,ASD57825,15027
+china,aaa29,phone1166,ASD26161,15028
+china,aaa30,phone2248,ASD47899,15029
+china,aaa31,phone475,ASD89811,15030
+china,aaa32,phone2499,ASD87974,15031
+china,aaa33,phone2333,ASD62408,15032
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithOldCarbonDataFile.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithOldCarbonDataFile.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithOldCarbonDataFile.scala
new file mode 100644
index 0000000..431180c
--- /dev/null
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithOldCarbonDataFile.scala
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.allqueries
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
+/*
+ * Test Class for query without data load
+ *
+ */
+class TestQueryWithOldCarbonDataFile extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll {
+	  CarbonProperties.getInstance.addProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, "1");         
+    sql("drop table if exists OldFormatTable")
+    sql("drop table if exists OldFormatTableHIVE")
+     sql("""
+           CREATE TABLE IF NOT EXISTS OldFormatTable
+           (country String,
+           name String, phonetype String, serialname String, salary Int)
+           STORED BY 'carbondata'
+           """)
+      sql("""
+           CREATE TABLE IF NOT EXISTS OldFormatTableHIVE
+           (country String,
+           name String, phonetype String, serialname String, salary Int)
+          row format delimited fields terminated by ','
+           """)      
+    sql("LOAD DATA local inpath './src/test/resources/OLDFORMATTABLE.csv' INTO table OldFormatTable");       
+   sql(s"""
+           LOAD DATA LOCAL INPATH './src/test/resources/OLDFORMATTABLEHIVE.csv' into table OldFormatTableHIVE
+           """)
+
+  }
+
+  CarbonProperties.getInstance.addProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, "2");
+  test("Test select * query") {
+    checkAnswer(
+      sql("select * from OldFormatTable"), sql("select * from OldFormatTableHIVE")
+    )
+  }
+
+  override def afterAll {
+     CarbonProperties.getInstance.addProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION, "1");
+    sql("drop table if exists OldFormatTable")
+    sql("drop table if exists OldFormatTableHIVE")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java
new file mode 100644
index 0000000..2fbb00e
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.processing.store;
+
+import org.apache.carbondata.processing.store.writer.CarbonDataWriterVo;
+import org.apache.carbondata.processing.store.writer.CarbonFactDataWriter;
+import org.apache.carbondata.processing.store.writer.CarbonFactDataWriterImpl2;
+import org.apache.carbondata.processing.store.writer.CarbonFactDataWriterImplForIntIndexAndAggBlock;
+
+/**
+ * Factory class to get the writer instance
+ */
+public class CarbonDataWriterFactory {
+
+  /**
+   * static instance
+   */
+  private static final CarbonDataWriterFactory CARBON_DATA_WRITER_FACTORY =
+      new CarbonDataWriterFactory();
+
+  /**
+   * private constructor
+   */
+  private CarbonDataWriterFactory() {
+    // TODO Auto-generated constructor stub
+  }
+
+  /**
+   * Below method will be used to get the instance of factory class
+   *
+   * @return fact class instance
+   */
+  public static CarbonDataWriterFactory getInstance() {
+    return CARBON_DATA_WRITER_FACTORY;
+  }
+
+  /**
+   * Below method will be used to get the writer instance based on version
+   *
+   * @param version            writer version
+   * @param carbonDataWriterVo writer vo object
+   * @return writer instance
+   */
+  public CarbonFactDataWriter<?> getFactDataWriter(final short version,
+      final CarbonDataWriterVo carbonDataWriterVo) {
+    switch (version) {
+      case 2:
+        return new CarbonFactDataWriterImpl2(carbonDataWriterVo);
+      default:
+        return new CarbonFactDataWriterImplForIntIndexAndAggBlock(carbonDataWriterVo);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d54dc647/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 f3bd484..1fa14f0 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
@@ -70,8 +70,8 @@ import org.apache.carbondata.processing.store.colgroup.ColGroupDataHolder;
 import org.apache.carbondata.processing.store.colgroup.ColGroupMinMax;
 import org.apache.carbondata.processing.store.colgroup.ColumnDataHolder;
 import org.apache.carbondata.processing.store.colgroup.DataHolder;
+import org.apache.carbondata.processing.store.writer.CarbonDataWriterVo;
 import org.apache.carbondata.processing.store.writer.CarbonFactDataWriter;
-import org.apache.carbondata.processing.store.writer.CarbonFactDataWriterImplForIntIndexAndAggBlock;
 import org.apache.carbondata.processing.store.writer.NodeHolder;
 import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException;
 import org.apache.carbondata.processing.util.RemoveDictionaryUtil;
@@ -1306,8 +1306,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     System.arraycopy(blockKeySize, noOfColStore, keyBlockSize, noOfColStore,
         blockKeySize.length - noOfColStore);
     this.dataWriter =
-        getFactDataWriter(this.storeLocation, this.measureCount, this.mdkeyLength, this.tableName,
-            fileManager, keyBlockSize);
+        getFactDataWriter(keyBlockSize);
     this.dataWriter.setIsNoDictionary(isNoDictionary);
     // initialize the channel;
     this.dataWriter.initializeWriter();
@@ -1396,13 +1395,45 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     return nullvalueIndexBitset;
   }
 
-  private CarbonFactDataWriter<?> getFactDataWriter(String storeLocation, int measureCount,
-      int mdKeyLength, String tableName, IFileManagerComposite fileManager, int[] keyBlockSize) {
-    return new CarbonFactDataWriterImplForIntIndexAndAggBlock(storeLocation, measureCount,
-        mdKeyLength, tableName, fileManager, keyBlockSize, aggKeyBlock, isComplexTypes(),
-        noDictionaryCount, carbonDataFileAttributes, databaseName, wrapperColumnSchemaList,
-        noDictionaryCount, dimensionType, carbonDataDirectoryPath, colCardinality,
-        segmentProperties, tableBlockSize);
+  /**
+   * Below method will be used to get the fact data writer instance
+   *
+   * @param keyBlockSize
+   * @return data writer instance
+   */
+  private CarbonFactDataWriter<?> getFactDataWriter(int[] keyBlockSize) {
+    short version = Short.parseShort(
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION));
+    return CarbonDataWriterFactory.getInstance()
+        .getFactDataWriter(version, getDataWriterVo(keyBlockSize));
+  }
+
+  /**
+   * Below method will be used to get the writer vo
+   *
+   * @param keyBlockSize size of each key block
+   * @return data writer vo object
+   */
+  private CarbonDataWriterVo getDataWriterVo(int[] keyBlockSize) {
+    CarbonDataWriterVo carbonDataWriterVo = new CarbonDataWriterVo();
+    carbonDataWriterVo.setStoreLocation(storeLocation);
+    carbonDataWriterVo.setMeasureCount(measureCount);
+    carbonDataWriterVo.setMdKeyLength(mdkeyLength);
+    carbonDataWriterVo.setTableName(tableName);
+    carbonDataWriterVo.setKeyBlockSize(keyBlockSize);
+    carbonDataWriterVo.setFileManager(fileManager);
+    carbonDataWriterVo.setAggBlocks(aggKeyBlock);
+    carbonDataWriterVo.setIsComplexType(isComplexTypes());
+    carbonDataWriterVo.setNoDictionaryCount(noDictionaryCount);
+    carbonDataWriterVo.setCarbonDataFileAttributes(carbonDataFileAttributes);
+    carbonDataWriterVo.setDatabaseName(databaseName);
+    carbonDataWriterVo.setWrapperColumnSchemaList(wrapperColumnSchemaList);
+    carbonDataWriterVo.setIsDictionaryColumn(dimensionType);
+    carbonDataWriterVo.setCarbonDataDirectoryPath(carbonDataDirectoryPath);
+    carbonDataWriterVo.setColCardinality(colCardinality);
+    carbonDataWriterVo.setSegmentProperties(segmentProperties);
+    carbonDataWriterVo.setTableBlocksize(tableBlockSize);
+    return carbonDataWriterVo;
   }
 
   private boolean[] isComplexTypes() {
@@ -1571,7 +1602,8 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
           if (!processingComplete || blockletProcessingCount.get() > 0) {
             producerExecutorService.shutdownNow();
             resetBlockletProcessingCount();
-            throw new CarbonDataWriterException(throwable.getMessage(), throwable);
+            LOGGER.error(throwable, "Problem while writing the carbon data file");
+            throw new CarbonDataWriterException(throwable.getMessage());
           }
         } finally {
           semaphore.release();


[5/5] incubator-carbondata git commit: [CARBONDATA-458]Improving First time query performance. This closes #265

Posted by ra...@apache.org.
[CARBONDATA-458]Improving First time query performance. This closes #265


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

Branch: refs/heads/master
Commit: 7213ac0579ba2f8b171474b73487f484e2a2df50
Parents: 9ad98f4 d54dc64
Author: ravipesala <ra...@gmail.com>
Authored: Thu Dec 1 15:19:41 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Dec 1 15:19:41 2016 +0530

----------------------------------------------------------------------
 .../core/carbon/datastore/BlockIndexStore.java  |   4 +-
 .../core/carbon/datastore/DataRefNode.java      |  18 +-
 .../datastore/block/SegmentProperties.java      |   7 +
 .../carbon/datastore/block/TableBlockInfo.java  |  20 +-
 .../chunk/reader/CarbonDataReaderFactory.java   |  93 +++++
 .../reader/DimensionColumnChunkReader.java      |   2 +-
 .../chunk/reader/MeasureColumnChunkReader.java  |   2 +-
 .../reader/dimension/AbstractChunkReader.java   |  12 +-
 ...CompressedDimensionChunkFileBasedReader.java | 135 -------
 ...mpressedDimensionChunkFileBasedReaderV1.java | 146 +++++++
 ...mpressedDimensionChunkFileBasedReaderV2.java | 308 +++++++++++++++
 .../measure/AbstractMeasureChunkReader.java     |  33 +-
 .../CompressedMeasureChunkFileBasedReader.java  |  92 -----
 ...CompressedMeasureChunkFileBasedReaderV1.java | 106 +++++
 ...CompressedMeasureChunkFileBasedReaderV2.java | 234 +++++++++++
 .../impl/btree/AbstractBTreeLeafNode.java       |   4 +-
 .../datastore/impl/btree/BTreeNonLeafNode.java  |   4 +-
 .../impl/btree/BlockletBTreeLeafNode.java       |  36 +-
 .../carbon/metadata/blocklet/BlockletInfo.java  |  40 ++
 .../metadata/blocklet/DataFileFooter.java       |   6 +-
 .../core/constants/CarbonCommonConstants.java   |  18 +-
 .../util/AbstractDataFileFooterConverter.java   | 390 ++++++++++++++++++
 .../core/util/CarbonMetadataUtil.java           | 158 +++++++-
 .../carbondata/core/util/CarbonProperties.java  | 121 +++---
 .../apache/carbondata/core/util/CarbonUtil.java | 209 ++++++++--
 .../core/util/DataFileFooterConverter.java      | 391 +------------------
 .../core/util/DataFileFooterConverter2.java     | 135 +++++++
 .../util/DataFileFooterConverterFactory.java    |  64 +++
 .../core/writer/CarbonFooterWriter.java         |   4 -
 .../executor/impl/AbstractQueryExecutor.java    |  55 ++-
 .../executor/impl/QueryExecutorProperties.java  |   4 +-
 .../scan/executor/infos/BlockExecutionInfo.java |  12 +-
 .../scan/executor/util/QueryUtil.java           | 102 +++--
 .../scan/scanner/AbstractBlockletScanner.java   |   7 +-
 .../scan/scanner/impl/FilterScanner.java        |  48 ++-
 .../carbon/datastore/block/BlockInfoTest.java   |  12 +-
 .../datastore/block/TableBlockInfoTest.java     |  32 +-
 .../datastore/block/TableTaskInfoTest.java      |   8 +-
 ...ressedDimensionChunkFileBasedReaderTest.java |  23 +-
 ...mpressedMeasureChunkFileBasedReaderTest.java |  30 +-
 .../core/util/CarbonMetadataUtilTest.java       |   3 +-
 .../carbondata/core/util/CarbonUtilTest.java    |  18 +-
 .../core/util/DataFileFooterConverterTest.java  |  11 +-
 format/src/main/thrift/carbondata.thrift        |  30 +-
 .../carbondata/hadoop/CarbonInputFormat.java    |  66 ++--
 .../carbondata/hadoop/CarbonInputSplit.java     | 107 +++--
 .../internal/index/impl/InMemoryBTreeIndex.java |   5 +-
 .../spark/merger/CarbonCompactionUtil.java      |   4 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   4 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   2 +-
 .../spark/src/test/resources/OLDFORMATTABLE.csv |  34 ++
 .../src/test/resources/OLDFORMATTABLEHIVE.csv   |  33 ++
 .../TestQueryWithOldCarbonDataFile.scala        |  70 ++++
 .../store/CarbonDataWriterFactory.java          |  70 ++++
 .../store/CarbonFactDataHandlerColumnar.java    |  54 ++-
 .../store/writer/AbstractFactDataWriter.java    | 316 +++++++--------
 .../store/writer/CarbonDataWriterVo.java        | 321 +++++++++++++++
 .../store/writer/CarbonFactDataWriterImpl2.java | 285 ++++++++++++++
 ...actDataWriterImplForIntIndexAndAggBlock.java | 223 +++--------
 .../processing/store/writer/NodeHolder.java     |  38 +-
 .../carbon/datastore/BlockIndexStoreTest.java   |  43 +-
 61 files changed, 3504 insertions(+), 1358 deletions(-)
----------------------------------------------------------------------