You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/02/27 16:39:10 UTC

[24/50] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

[CARBONDATA-2099] Refactor query scan process to improve readability

Unified concepts in scan process flow:

1.QueryModel contains all parameter for scan, it is created by API in CarbonTable. (In future, CarbonTable will be the entry point for various table operations)
2.Use term ColumnChunk to represent one column in one blocklet, and use ChunkIndex in reader to read specified column chunk
3.Use term ColumnPage to represent one page in one ColumnChunk
4.QueryColumn => ProjectionColumn, indicating it is for projection

This closes #1874


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

Branch: refs/heads/carbonstore-rebase4
Commit: 2e3077c48435eb062b6c413fbd82e57f6386f0f7
Parents: e5acea9
Author: Jacky Li <ja...@qq.com>
Authored: Tue Jan 30 21:24:04 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 16:59:47 2018 +0800

----------------------------------------------------------------------
 .../dictionary/AbstractDictionaryCache.java     |   3 +-
 .../cache/dictionary/DictionaryCacheLoader.java |   7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  11 +-
 .../core/datastore/BTreeBuilderInfo.java        |   6 -
 .../carbondata/core/datastore/DataRefNode.java  |  81 +--
 .../carbondata/core/datastore/FileHolder.java   | 118 ----
 .../carbondata/core/datastore/FileReader.java   | 114 +++
 .../core/datastore/block/SegmentProperties.java |  50 +-
 .../chunk/DimensionColumnDataChunk.java         | 116 ---
 .../datastore/chunk/DimensionColumnPage.java    | 111 +++
 .../chunk/impl/AbstractDimensionColumnPage.java |  89 +++
 .../chunk/impl/AbstractDimensionDataChunk.java  |  95 ---
 .../impl/ColumnGroupDimensionColumnPage.java    | 194 ++++++
 .../impl/ColumnGroupDimensionDataChunk.java     | 194 ------
 .../chunk/impl/DimensionRawColumnChunk.java     |  46 +-
 .../impl/FixedLengthDimensionColumnPage.java    | 163 +++++
 .../impl/FixedLengthDimensionDataChunk.java     | 163 -----
 .../chunk/impl/MeasureRawColumnChunk.java       |  26 +-
 .../impl/VariableLengthDimensionColumnPage.java | 133 ++++
 .../impl/VariableLengthDimensionDataChunk.java  | 140 ----
 .../reader/DimensionColumnChunkReader.java      |  14 +-
 .../chunk/reader/MeasureColumnChunkReader.java  |  12 +-
 .../AbstractChunkReaderV2V3Format.java          |  34 +-
 ...mpressedDimensionChunkFileBasedReaderV1.java |  38 +-
 ...mpressedDimensionChunkFileBasedReaderV2.java |  30 +-
 ...essedDimChunkFileBasedPageLevelReaderV3.java |  11 +-
 ...mpressedDimensionChunkFileBasedReaderV3.java |  49 +-
 .../AbstractMeasureChunkReaderV2V3Format.java   |  42 +-
 ...CompressedMeasureChunkFileBasedReaderV1.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV2.java |  24 +-
 ...CompressedMeasureChunkFileBasedReaderV3.java |  45 +-
 ...essedMsrChunkFileBasedPageLevelReaderV3.java |   8 +-
 .../chunk/store/ColumnPageWrapper.java          |  30 +-
 .../chunk/store/DimensionDataChunkStore.java    |   8 +-
 .../SafeFixedLengthDimensionDataChunkStore.java |   6 +-
 ...feVariableLengthDimensionDataChunkStore.java |   8 +-
 ...nsafeFixedLengthDimensionDataChunkStore.java |  10 +-
 ...afeVariableLengthDimesionDataChunkStore.java |  10 +-
 .../datastore/columnar/ColumnGroupModel.java    |  26 -
 .../core/datastore/impl/DFSFileHolderImpl.java  | 166 -----
 .../core/datastore/impl/DFSFileReaderImpl.java  | 155 ++++
 .../datastore/impl/DefaultFileTypeProvider.java |  16 +-
 .../core/datastore/impl/FileFactory.java        |   4 +-
 .../core/datastore/impl/FileHolderImpl.java     | 224 ------
 .../core/datastore/impl/FileReaderImpl.java     | 215 ++++++
 .../core/datastore/impl/FileTypeInerface.java   |   4 +-
 .../impl/btree/AbstractBTreeLeafNode.java       |  60 +-
 .../impl/btree/BTreeDataRefNodeFinder.java      |   6 +-
 .../datastore/impl/btree/BTreeNonLeafNode.java  |  52 +-
 .../impl/btree/BlockBTreeLeafNode.java          |   6 +-
 .../impl/btree/BlockletBTreeLeafNode.java       |  46 +-
 .../page/encoding/EncodingFactory.java          |   8 +-
 .../server/NonSecureDictionaryServer.java       |   1 -
 .../core/indexstore/BlockletDetailInfo.java     |   4 -
 .../blockletindex/BlockletDataRefNode.java      | 228 ++++++
 .../BlockletDataRefNodeWrapper.java             | 241 -------
 .../indexstore/blockletindex/IndexWrapper.java  |   2 +-
 .../blockletindex/SegmentIndexFileStore.java    |   7 +-
 .../core/memory/HeapMemoryAllocator.java        |   2 +-
 .../core/metadata/blocklet/SegmentInfo.java     |  19 -
 .../core/metadata/schema/table/CarbonTable.java | 130 +++-
 .../schema/table/RelationIdentifier.java        |  16 -
 .../core/metadata/schema/table/TableInfo.java   |   6 +-
 .../schema/table/column/CarbonColumn.java       |   2 +-
 .../schema/table/column/CarbonDimension.java    |  12 -
 .../core/mutate/CarbonUpdateUtil.java           |  20 +-
 .../core/mutate/DeleteDeltaBlockDetails.java    |   8 -
 .../core/mutate/DeleteDeltaBlockletDetails.java |   8 -
 .../data/BlockletDeleteDeltaCacheLoader.java    |  87 ---
 .../scan/collector/ScannedResultCollector.java  |  16 +-
 .../impl/AbstractScannedResultCollector.java    |  29 +-
 .../impl/DictionaryBasedResultCollector.java    |  53 +-
 .../DictionaryBasedVectorResultCollector.java   |  63 +-
 .../collector/impl/RawBasedResultCollector.java |  32 +-
 ...structureBasedDictionaryResultCollector.java |  17 +-
 .../RestructureBasedRawResultCollector.java     |  27 +-
 .../RestructureBasedVectorResultCollector.java  |  33 +-
 .../core/scan/complextypes/ArrayQueryType.java  |   4 +-
 .../scan/complextypes/ComplexQueryType.java     |  12 +-
 .../scan/complextypes/PrimitiveQueryType.java   |   4 +-
 .../core/scan/complextypes/StructQueryType.java |   4 +-
 .../executor/impl/AbstractQueryExecutor.java    | 154 ++--
 .../scan/executor/impl/DetailQueryExecutor.java |   6 +-
 .../scan/executor/infos/BlockExecutionInfo.java | 179 ++---
 .../core/scan/executor/util/QueryUtil.java      | 141 ++--
 .../scan/executor/util/RestructureUtil.java     |  70 +-
 .../core/scan/expression/ColumnExpression.java  |   8 -
 .../scan/expression/FilterModificationNode.java |  35 +-
 .../expression/RangeExpressionEvaluator.java    |  11 +-
 .../scan/filter/FilterExpressionProcessor.java  |  22 +-
 .../core/scan/filter/FilterProcessor.java       |   2 +-
 .../carbondata/core/scan/filter/FilterUtil.java |  60 +-
 .../core/scan/filter/GenericQueryType.java      |   4 +-
 .../filter/executer/AndFilterExecuterImpl.java  |  22 +-
 .../ExcludeColGroupFilterExecuterImpl.java      |  85 ---
 .../executer/ExcludeFilterExecuterImpl.java     | 131 ++--
 .../scan/filter/executer/FilterExecuter.java    |  10 +-
 .../ImplicitIncludeFilterExecutorImpl.java      |  23 +-
 .../IncludeColGroupFilterExecuterImpl.java      |  31 +-
 .../executer/IncludeFilterExecuterImpl.java     | 142 ++--
 .../filter/executer/OrFilterExecuterImpl.java   |  25 +-
 .../executer/RangeValueFilterExecuterImpl.java  | 224 +++---
 .../RestructureExcludeFilterExecutorImpl.java   |  21 +-
 .../RestructureIncludeFilterExecutorImpl.java   |  22 +-
 .../executer/RowLevelFilterExecuterImpl.java    | 151 ++--
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  | 121 ++--
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java | 116 +--
 ...velRangeLessThanEqualFilterExecuterImpl.java | 113 +--
 .../RowLevelRangeLessThanFiterExecuterImpl.java | 122 ++--
 .../filter/executer/TrueFilterExecutor.java     |  17 +-
 .../executer/ValueBasedFilterExecuterImpl.java  |  16 +-
 .../scan/filter/intf/FilterOptimizerBasic.java  |  26 -
 .../filter/optimizer/RangeFilterOptmizer.java   |   3 +-
 .../resolver/ConditionalFilterResolverImpl.java |  18 +-
 .../resolver/RowLevelFilterResolverImpl.java    |   2 +-
 .../RowLevelRangeFilterResolverImpl.java        |   2 +-
 .../MeasureColumnResolvedFilterInfo.java        |  12 -
 .../TrueConditionalResolverImpl.java            |  16 +-
 .../visitor/RangeDictionaryColumnVisitor.java   |   2 -
 .../core/scan/model/CarbonQueryPlan.java        | 151 ----
 .../core/scan/model/ProjectionColumn.java       |  59 ++
 .../core/scan/model/ProjectionDimension.java    |  45 ++
 .../core/scan/model/ProjectionMeasure.java      |  46 ++
 .../carbondata/core/scan/model/QueryColumn.java |  66 --
 .../core/scan/model/QueryDimension.java         |  56 --
 .../core/scan/model/QueryMeasure.java           |  59 --
 .../carbondata/core/scan/model/QueryModel.java  | 120 +---
 .../core/scan/model/QueryProjection.java        |  83 +++
 .../processor/AbstractDataBlockIterator.java    | 251 -------
 .../core/scan/processor/BlockletIterator.java   |  14 +-
 .../core/scan/processor/BlocksChunkHolder.java  | 141 ----
 .../core/scan/processor/DataBlockIterator.java  | 269 +++++++
 .../scan/processor/RawBlockletColumnChunks.java | 113 +++
 .../processor/impl/DataBlockIteratorImpl.java   |  69 --
 .../core/scan/result/AbstractScannedResult.java | 698 -------------------
 .../core/scan/result/BatchResult.java           | 103 ---
 .../core/scan/result/BlockletScannedResult.java | 618 ++++++++++++++++
 .../carbondata/core/scan/result/RowBatch.java   | 103 +++
 .../result/impl/FilterQueryScannedResult.java   |  50 +-
 .../impl/NonFilterQueryScannedResult.java       |  14 +-
 .../AbstractDetailQueryResultIterator.java      |  44 +-
 .../scan/result/iterator/ChunkRowIterator.java  |   8 +-
 .../iterator/DetailQueryResultIterator.java     |  14 +-
 .../PartitionSpliterRawResultIterator.java      |  15 +-
 .../scan/result/iterator/RawResultIterator.java |  10 +-
 .../VectorDetailQueryResultIterator.java        |   4 +-
 .../scan/result/vector/CarbonColumnarBatch.java |   3 -
 .../scan/result/vector/ColumnVectorInfo.java    |   8 +-
 .../vector/MeasureDataVectorProcessor.java      |  36 +-
 .../scan/scanner/AbstractBlockletScanner.java   | 181 -----
 .../core/scan/scanner/BlockletScanner.java      |  22 +-
 .../scanner/impl/BlockletFilterScanner.java     | 329 +++++++++
 .../scan/scanner/impl/BlockletFullScanner.java  | 191 +++++
 .../core/scan/scanner/impl/FilterScanner.java   | 326 ---------
 .../scan/scanner/impl/NonFilterScanner.java     |  35 -
 .../core/scan/wrappers/ByteArrayWrapper.java    |  41 +-
 .../core/stats/PartitionStatistic.java          |  23 -
 .../core/stats/QueryStatisticsModel.java        |   3 -
 .../core/statusmanager/LoadMetadataDetails.java |   9 -
 .../statusmanager/SegmentStatusManager.java     |  16 -
 .../SegmentUpdateStatusManager.java             | 156 +----
 .../util/AbstractDataFileFooterConverter.java   |   1 -
 .../apache/carbondata/core/util/CarbonUtil.java |  34 +-
 .../core/util/DataFileFooterConverter.java      |   6 +-
 .../core/writer/CarbonDictionaryWriter.java     |  11 -
 .../core/writer/CarbonDictionaryWriterImpl.java |   2 +-
 .../DictionaryCacheLoaderImplTest.java          |   4 +-
 .../store/impl/DFSFileHolderImplUnitTest.java   | 149 ----
 .../store/impl/DFSFileReaderImplUnitTest.java   | 149 ++++
 .../store/impl/FileHolderImplUnitTest.java      | 154 ----
 .../store/impl/FileReaderImplUnitTest.java      | 154 ++++
 .../datastore/block/SegmentPropertiesTest.java  |   8 +-
 .../impl/ColumnGroupDimensionDataChunkTest.java |   8 +-
 .../impl/FixedLengthDimensionDataChunkTest.java |   8 +-
 .../impl/btree/BTreeBlockFinderTest.java        |  20 +-
 .../impl/RawBasedResultCollectorTest.java       |  32 +-
 .../core/scan/executor/util/QueryUtilTest.java  |  41 +-
 .../scan/executor/util/RestructureUtilTest.java |  31 +-
 .../core/scan/filter/FilterUtilTest.java        |   2 +-
 .../executer/ExcludeFilterExecuterImplTest.java |   6 +-
 .../executer/IncludeFilterExecuterImplTest.java |  32 +-
 .../core/scan/result/BatchResultTest.java       |  94 ---
 .../core/scan/result/RowBatchTest.java          |  94 +++
 .../carbondata/core/util/CarbonUtilTest.java    |  90 +--
 .../core/util/DataFileFooterConverterTest.java  |  11 +-
 .../core/util/RangeFilterProcessorTest.java     |   9 +-
 .../scanner/impl/FilterScannerTest.java         | 160 -----
 dev/findbugs-exclude.xml                        |   2 +-
 .../examples/CarbonSessionExample.scala         |   2 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  44 +-
 .../internal/index/impl/InMemoryBTreeIndex.java |   4 +-
 .../streaming/CarbonStreamRecordReader.java     |   2 +-
 .../hadoop/util/BlockLevelTraverser.java        |   2 +-
 .../hadoop/util/CarbonInputFormatUtil.java      |  73 +-
 .../hive/MapredCarbonInputFormat.java           |   9 +-
 .../presto/CarbonVectorizedRecordReader.java    | 243 -------
 .../carbondata/presto/CarbondataPageSource.java |   2 +-
 .../presto/CarbondataRecordCursor.java          |   6 +-
 .../carbondata/presto/CarbondataRecordSet.java  |   4 +-
 .../presto/CarbondataRecordSetProvider.java     |   2 +-
 .../PrestoCarbonVectorizedRecordReader.java     | 243 +++++++
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   3 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   2 +-
 .../vectorreader/ColumnarVectorWrapper.java     |   2 +-
 .../VectorizedCarbonRecordReader.java           |  51 +-
 .../merger/CarbonCompactionExecutor.java        |  64 +-
 .../partition/impl/QueryPartitionHelper.java    |   6 +-
 .../spliter/AbstractCarbonQueryExecutor.java    |  54 +-
 .../partition/spliter/CarbonSplitExecutor.java  |   4 +-
 .../processing/util/CarbonQueryUtil.java        |   6 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |   2 +-
 211 files changed, 5843 insertions(+), 7396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index 598d00e..fb67208 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -260,8 +260,7 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
     DictionaryCacheLoader dictionaryCacheLoader =
         new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier);
     dictionaryCacheLoader
-        .load(dictionaryInfo, dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
-            dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex);
+        .load(dictionaryInfo, dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
index 01c277d..8df1539 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
@@ -19,8 +19,6 @@ package org.apache.carbondata.core.cache.dictionary;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-
 public interface DictionaryCacheLoader {
 
   /**
@@ -28,7 +26,6 @@ public interface DictionaryCacheLoader {
    *
    * @param dictionaryInfo             dictionary info object which will hold the required data
    *                                   for a given column
-   * @param columnIdentifier           column unique identifier
    * @param dictionaryChunkStartOffset start offset from where dictionary file has to
    *                                   be read
    * @param dictionaryChunkEndOffset   end offset till where dictionary file has to
@@ -37,7 +34,7 @@ public interface DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
-      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
+  void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset,
+      long dictionaryChunkEndOffset, boolean loadSortIndex)
       throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index a603c01..899abf5 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.reader.CarbonDictionaryReader;
 import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
 import org.apache.carbondata.core.service.CarbonCommonFactory;
@@ -43,8 +42,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
   /**
    * @param dictionaryColumnUniqueIdentifier dictionary column identifier
    */
-  public DictionaryCacheLoaderImpl(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+  DictionaryCacheLoaderImpl(DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
   }
 
@@ -53,7 +51,6 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    *
    * @param dictionaryInfo             dictionary info object which will hold the required data
    *                                   for a given column
-   * @param columnIdentifier           column unique identifier
    * @param dictionaryChunkStartOffset start offset from where dictionary file has to
    *                                   be read
    * @param dictionaryChunkEndOffset   end offset till where dictionary file has to
@@ -62,9 +59,9 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  @Override public void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
-      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
-      throws IOException {
+  @Override
+  public void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset,
+      long dictionaryChunkEndOffset, boolean loadSortIndex) throws IOException {
     Iterator<byte[]> columnDictionaryChunkWrapper =
         load(dictionaryColumnUniqueIdentifier, dictionaryChunkStartOffset,
             dictionaryChunkEndOffset);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
index 3bfbc45..381e764 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
@@ -43,16 +43,10 @@ public class BTreeBuilderInfo {
     this.footerList = footerList;
   }
 
-  /**
-   * @return the eachDimensionBlockSize
-   */
   public int[] getDimensionColumnValueSize() {
     return dimensionColumnValueSize;
   }
 
-  /**
-   * @return the footerList
-   */
   public List<DataFileFooter> getFooterList() {
     return footerList;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
index 13d5f69..273f833 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
@@ -18,7 +18,6 @@ package org.apache.carbondata.core.datastore;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 
@@ -28,46 +27,47 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 public interface DataRefNode {
 
   /**
-   * Method to get the next block this can be used while scanning when
+   * Return the next data block in the tree, this can be used while scanning when
    * iterator of this class can be used iterate over blocks
-   *
-   * @return next block
    */
   DataRefNode getNextDataRefNode();
 
   /**
-   * to get the number of keys tuples present in the block
-   *
-   * @return number of keys in the block
+   * Return the number of rows in the data block
    */
-  int nodeSize();
+  int numRows();
 
   /**
-   * Method can be used to get the block index .This can be used when multiple
-   * thread can be used scan group of blocks in that can we can assign the
+   * Return the block index. This can be used when multiple
+   * thread can be used scan group of blocks in that can we can assign
    * some of the blocks to one thread and some to other
-   *
-   * @return block number
    */
-  long nodeNumber();
+  long nodeIndex();
+
+  /**
+   * Return the blocklet index in the node
+   */
+  short blockletIndex();
 
   /**
-   * Method is used for retreiving the BlockletId.
-   * @return the blockletid related to the data block.
+   * Return the number of pages
    */
-  String blockletId();
+  int numberOfPages();
 
   /**
-   * This method will be used to get the max value of all the columns this can
+   * Return the number of rows for a give page
+   */
+  int getPageRowCount(int pageNumber);
+
+  /**
+   * Return the max value of all the columns, this can
    * be used in case of filter query
-   *
    */
   byte[][] getColumnsMaxValue();
 
   /**
-   * This method will be used to get the min value of all the columns this can
+   * Return the min value of all the columns, this can
    * be used in case of filter query
-   *
    */
   byte[][] getColumnsMinValue();
 
@@ -75,15 +75,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 range indexes of the blocks need to be read
+   * @param columnIndexRange 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
+   *                     data in one IO operation
    * @return dimension data chunks
    */
-  DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, int[][] blockIndexes)
+  DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
@@ -92,54 +92,31 @@ public interface DataRefNode {
    * @param fileReader file reader to read the chunk from file
    * @return dimension data chunk
    */
-  DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndexes)
+  DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, int columnIndex)
       throws IOException;
 
   /**
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes range indexes of the blocks need to be read
+   * @param columnIndexRange 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
+   *                     data in one IO operation
    * @return measure column data chunk
    */
-  MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
+  MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
    * Below method will be used to read the measure chunk
    *
    * @param fileReader file read to read the file chunk
-   * @param blockIndex block index to be read from file
+   * @param columnIndex block index to be read from file
    * @return measure data chunk
    */
-  MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) throws IOException;
+  MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) throws IOException;
 
-  /**
-   * @param deleteDeltaDataCache
-   */
-  void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache);
-
-  /**
-   * @return
-   */
-  BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache();
-
-  /**
-   * number of pages in blocklet
-   * @return
-   */
-  int numberOfPages();
-
-  /**
-   * Return the number of rows for a give page
-   *
-   * @param pageNumber
-   * @return
-   */
-  int getPageRowCount(int pageNumber);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java b/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
deleted file mode 100644
index b4130a0..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
+++ /dev/null
@@ -1,118 +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.datastore;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public interface FileHolder {
-
-  /**
-   * This method will be used to reads the data to byteBuffer from file based on offset
-   * and length(number of bytes) need to read
-   *
-   * @param filePath fully qualified file path
-   * @param offset reading start position,
-   * @param length number of bytes to be read
-   * @return ByteBuffer
-   * @throws IOException
-   */
-  ByteBuffer readByteBuffer(String filePath, long offset, int length)
-      throws IOException;
-  /**
-   * This method will be used to read the byte array from file based on offset
-   * and length(number of bytes) need to read
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @param length   number of bytes to be read
-   * @return read byte array
-   */
-  byte[] readByteArray(String filePath, long offset, int length) throws IOException;
-
-  /**
-   * This method will be used to read the byte array from file based on length(number of bytes)
-   *
-   * @param filePath fully qualified file path
-   * @param length   number of bytes to be read
-   * @return read byte array
-   */
-  byte[] readByteArray(String filePath, int length) throws IOException;
-
-  /**
-   * This method will be used to read int from file from postion(offset), here
-   * length will be always 4 bacause int byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read int
-   */
-  int readInt(String filePath, long offset) throws IOException;
-
-  /**
-   * This method will be used to read long from file from postion(offset), here
-   * length will be always 8 bacause int byte size is 8
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read long
-   */
-  long readLong(String filePath, long offset) throws IOException;
-
-  /**
-   * This method will be used to read int from file from postion(offset), here
-   * length will be always 4 bacause int byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @return read int
-   */
-  int readInt(String filePath) throws IOException;
-
-  /**
-   * This method will be used to read long value from file from postion(offset), here
-   * length will be always 8 because long byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read long
-   */
-  long readDouble(String filePath, long offset) throws IOException;
-
-  /**
-   * This method will be used to close all the streams currently present in the cache
-   */
-  void finish() throws IOException;
-
-  void setQueryId(String queryId);
-
-  String getQueryId();
-
-  /**
-   * Set the flag to read data page by page instead of whole blocklet.
-   *
-   * @param isReadPageByPage
-   */
-  void setReadPageByPage(boolean isReadPageByPage);
-
-  /**
-   * Whether read the data page by page from carbondata file instead of reading whole
-   * blocklet to memory. It is need in case of memory constraint operations.
-   */
-  boolean isReadPageByPage();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java
new file mode 100644
index 0000000..df0d745
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java
@@ -0,0 +1,114 @@
+/*
+ * 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.datastore;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface FileReader {
+
+  /**
+   * This method will be used to reads the data to byteBuffer from file based on offset
+   * and length(number of bytes) need to read
+   *
+   * @param filePath fully qualified file path
+   * @param offset reading start position,
+   * @param length number of bytes to be read
+   * @return ByteBuffer
+   * @throws IOException
+   */
+  ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException;
+  /**
+   * This method will be used to read the byte array from file based on offset
+   * and length(number of bytes) need to read
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  byte[] readByteArray(String filePath, long offset, int length) throws IOException;
+
+  /**
+   * This method will be used to read the byte array from file based on length(number of bytes)
+   *
+   * @param filePath fully qualified file path
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  byte[] readByteArray(String filePath, int length) throws IOException;
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read int
+   */
+  int readInt(String filePath, long offset) throws IOException;
+
+  /**
+   * This method will be used to read long from file from postion(offset), here
+   * length will be always 8 bacause int byte size is 8
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read long
+   */
+  long readLong(String filePath, long offset) throws IOException;
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @return read int
+   */
+  int readInt(String filePath) throws IOException;
+
+  /**
+   * This method will be used to read long value from file from postion(offset), here
+   * length will be always 8 because long byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read long
+   */
+  long readDouble(String filePath, long offset) throws IOException;
+
+  /**
+   * This method will be used to close all the streams currently present in the cache
+   */
+  void finish() throws IOException;
+
+  /**
+   * Set the flag to read data page by page instead of whole blocklet.
+   *
+   * @param isReadPageByPage
+   */
+  void setReadPageByPage(boolean isReadPageByPage);
+
+  /**
+   * Whether read the data page by page from carbondata file instead of reading whole
+   * blocklet to memory. It is need in case of memory constraint operations.
+   */
+  boolean isReadPageByPage();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
index 5f82e87..c134db9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
@@ -94,10 +94,9 @@ public class SegmentProperties {
   private int[] complexDimColumnCardinality;
 
   /**
-   * mapping of dimension column to block in a file this will be used for
-   * reading the blocks from file
+   * mapping of dimension ordinal in schema to column chunk index in the data file
    */
-  private Map<Integer, Integer> dimensionOrdinalToBlockMapping;
+  private Map<Integer, Integer> dimensionOrdinalToChunkMapping;
 
   /**
    * a block can have multiple columns. This will have block index as key
@@ -106,10 +105,9 @@ public class SegmentProperties {
   private Map<Integer, Set<Integer>> blockTodimensionOrdinalMapping;
 
   /**
-   * mapping of measure column to block to in file this will be used while
-   * reading the block in a file
+   * mapping of measure ordinal in schema to column chunk index in the data file
    */
-  private Map<Integer, Integer> measuresOrdinalToBlockMapping;
+  private Map<Integer, Integer> measuresOrdinalToChunkMapping;
 
   /**
    * size of the each dimension column value in a block this can be used when
@@ -172,15 +170,15 @@ public class SegmentProperties {
         new ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     measures = new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     fillDimensionAndMeasureDetails(columnsInTable, columnCardinality);
-    dimensionOrdinalToBlockMapping =
+    dimensionOrdinalToChunkMapping =
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     blockTodimensionOrdinalMapping =
         new HashMap<Integer, Set<Integer>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    measuresOrdinalToBlockMapping =
+    measuresOrdinalToChunkMapping =
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     intialiseColGroups();
     fillOrdinalToBlockMappingForDimension();
-    fillOrdinalToBlockIndexMappingForMeasureColumns();
+    fillOrdinalToChunkIndexMappingForMeasureColumns();
     fillColumnGroupAndItsCardinality(columnCardinality);
     fillKeyGeneratorDetails();
   }
@@ -237,7 +235,7 @@ public class SegmentProperties {
       if (dimension.isColumnar() || dimension.columnGroupId() != prvcolumnGroupId) {
         blockOrdinal++;
       }
-      dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), blockOrdinal);
+      dimensionOrdinalToChunkMapping.put(dimension.getOrdinal(), blockOrdinal);
       prvcolumnGroupId = dimension.columnGroupId();
       index++;
     }
@@ -245,7 +243,7 @@ public class SegmentProperties {
     // complex dimension will be stored at last
     while (index < complexDimensions.size()) {
       dimension = complexDimensions.get(index);
-      dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), ++blockOrdinal);
+      dimensionOrdinalToChunkMapping.put(dimension.getOrdinal(), ++blockOrdinal);
       blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal, dimension);
       index++;
     }
@@ -256,7 +254,7 @@ public class SegmentProperties {
    *
    */
   private void fillBlockToDimensionOrdinalMapping() {
-    Set<Entry<Integer, Integer>> blocks = dimensionOrdinalToBlockMapping.entrySet();
+    Set<Entry<Integer, Integer>> blocks = dimensionOrdinalToChunkMapping.entrySet();
     Iterator<Entry<Integer, Integer>> blockItr = blocks.iterator();
     while (blockItr.hasNext()) {
       Entry<Integer, Integer> block = blockItr.next();
@@ -280,7 +278,7 @@ public class SegmentProperties {
    */
   private int fillComplexDimensionChildBlockIndex(int blockOrdinal, CarbonDimension dimension) {
     for (int i = 0; i < dimension.getNumberOfChild(); i++) {
-      dimensionOrdinalToBlockMapping
+      dimensionOrdinalToChunkMapping
           .put(dimension.getListOfChildDimensions().get(i).getOrdinal(), ++blockOrdinal);
       if (dimension.getListOfChildDimensions().get(i).getNumberOfChild() > 0) {
         blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal,
@@ -295,11 +293,11 @@ public class SegmentProperties {
    * of measure ordinal to its block index mapping in
    * file
    */
-  private void fillOrdinalToBlockIndexMappingForMeasureColumns() {
+  private void fillOrdinalToChunkIndexMappingForMeasureColumns() {
     int blockOrdinal = 0;
     int index = 0;
     while (index < measures.size()) {
-      measuresOrdinalToBlockMapping.put(measures.get(index).getOrdinal(), blockOrdinal);
+      measuresOrdinalToChunkMapping.put(measures.get(index).getOrdinal(), blockOrdinal);
       blockOrdinal++;
       index++;
     }
@@ -731,17 +729,17 @@ public class SegmentProperties {
   }
 
   /**
-   * @return the dimensionOrdinalToBlockMapping
+   * @return the dimensionOrdinalToChunkMapping
    */
-  public Map<Integer, Integer> getDimensionOrdinalToBlockMapping() {
-    return dimensionOrdinalToBlockMapping;
+  public Map<Integer, Integer> getDimensionOrdinalToChunkMapping() {
+    return dimensionOrdinalToChunkMapping;
   }
 
   /**
-   * @return the measuresOrdinalToBlockMapping
+   * @return the measuresOrdinalToChunkMapping
    */
-  public Map<Integer, Integer> getMeasuresOrdinalToBlockMapping() {
-    return measuresOrdinalToBlockMapping;
+  public Map<Integer, Integer> getMeasuresOrdinalToChunkMapping() {
+    return measuresOrdinalToChunkMapping;
   }
 
   /**
@@ -805,16 +803,6 @@ public class SegmentProperties {
   }
 
   /**
-   * It returns no of column availble in given column group
-   *
-   * @param colGrpId
-   * @return no of column in given column group
-   */
-  public int getNoOfColumnsInColumnGroup(int colGrpId) {
-    return columnGroupOrdinalToMdkeymapping.get(colGrpId).size();
-  }
-
-  /**
    * @param blockIndex
    * @return It returns all dimension present in given block index
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java
deleted file mode 100644
index 3791314..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java
+++ /dev/null
@@ -1,116 +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.datastore.chunk;
-
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-
-/**
- * Interface for dimension column chunk.
- */
-public interface DimensionColumnDataChunk {
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data   data to filed
-   * @param offset offset from which data need to be filed
-   * @return how many bytes was copied
-   */
-  int fillChunkData(byte[] data, int offset, int columnIndex, KeyStructureInfo restructuringInfo);
-
-  /**
-   * It uses to convert column data to dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo @return
-   */
-  int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Fill the data to vector
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Fill the data to vector
-   * @param rowMapping
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Below method to get  the data based in row id
-   *
-   * @return chunk
-   */
-  byte[] getChunkData(int columnIndex);
-
-  /**
-   * @return inverted index
-   */
-  int getInvertedIndex(int index);
-
-  /**
-   *
-   * @param invertedIndex
-   * @return index reverse index
-   */
-  int getInvertedReverseIndex(int invertedIndex);
-
-  /**
-   * @return whether column is dictionary column or not
-   */
-  boolean isNoDicitionaryColumn();
-
-  /**
-   * @return length of each column
-   */
-  int getColumnValueSize();
-
-  /**
-   * @return whether columns where explictly sorted or not
-   */
-  boolean isExplicitSorted();
-
-  /**
-   * to compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  int compareTo(int index, byte[] compareValue);
-
-  /**
-   * below method will be used to free the allocated memory
-   */
-  void freeMemory();
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java
new file mode 100644
index 0000000..15840bc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java
@@ -0,0 +1,111 @@
+/*
+ * 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.datastore.chunk;
+
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+
+/**
+ * Interface for dimension column chunk.
+ */
+public interface DimensionColumnPage {
+
+  /**
+   * Below method will be used to fill the data based on offset and row id
+   *
+   * @param offset offset from which data need to be filed
+   * @param data   data to filed
+   * @return how many bytes was copied
+   */
+  int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo);
+
+  /**
+   * It uses to convert column data to dictionary integer value
+   *
+   * @param rowId
+   * @param chunkIndex
+   * @param outputSurrogateKey
+   * @param restructuringInfo @return
+   */
+  int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
+      KeyStructureInfo restructuringInfo);
+
+  /**
+   * Fill the data to vector
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo);
+
+  /**
+   * Fill the data to vector
+   * @param filteredRowId
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo);
+
+  /**
+   * Below method to get  the data based in row id
+   *
+   * @return chunk
+   */
+  byte[] getChunkData(int rowId);
+
+  /**
+   * @return inverted index
+   */
+  int getInvertedIndex(int rowId);
+
+  /**
+   *
+   * @param rowId
+   * @return index reverse index
+   */
+  int getInvertedReverseIndex(int rowId);
+
+  /**
+   * @return whether column is dictionary column or not
+   */
+  boolean isNoDicitionaryColumn();
+
+  /**
+   * @return whether columns where explictly sorted or not
+   */
+  boolean isExplicitSorted();
+
+  /**
+   * to compare the data
+   *
+   * @param rowId        row index to be compared
+   * @param compareValue value to compare
+   * @return compare result
+   */
+  int compareTo(int rowId, byte[] compareValue);
+
+  /**
+   * below method will be used to free the allocated memory
+   */
+  void freeMemory();
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java
new file mode 100644
index 0000000..6f316c5
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java
@@ -0,0 +1,89 @@
+/*
+ * 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.datastore.chunk.impl;
+
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionDataChunkStore;
+
+/**
+ * Class responsibility is to give access to dimension column data chunk store
+ */
+public abstract class AbstractDimensionColumnPage implements DimensionColumnPage {
+
+  /**
+   * data chunks
+   */
+  DimensionDataChunkStore dataChunkStore;
+
+  /**
+   * @return whether columns where explicitly sorted or not
+   */
+  @Override public boolean isExplicitSorted() {
+    return dataChunkStore.isExplicitSorted();
+  }
+
+  /**
+   * Below method to get the data based in row id
+   *
+   * @param rowId row id of the data
+   * @return chunk
+   */
+  @Override public byte[] getChunkData(int rowId) {
+    return dataChunkStore.getRow(rowId);
+  }
+
+  /**
+   * @return inverted index
+   */
+  @Override public int getInvertedIndex(int rowId) {
+    return dataChunkStore.getInvertedIndex(rowId);
+  }
+
+  /**
+   * @param rowId
+   * @return inverted index reverse
+   */
+  @Override public int getInvertedReverseIndex(int rowId) {
+    return dataChunkStore.getInvertedReverseIndex(rowId);
+  }
+
+  /**
+   * To compare the data
+   *
+   * @param rowId        row index to be compared
+   * @param compareValue value to compare
+   * @return compare result
+   */
+  @Override public int compareTo(int rowId, byte[] compareValue) {
+    // TODO Auto-generated method stub
+    return dataChunkStore.compareTo(rowId, compareValue);
+  }
+
+  /**
+   * below method will be used to free the allocated memory
+   */
+  @Override public void freeMemory() {
+    dataChunkStore.freeMemory();
+  }
+
+  /**
+   * @return column is dictionary column or not
+   */
+  @Override public boolean isNoDicitionaryColumn() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java
deleted file mode 100644
index eac062f..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java
+++ /dev/null
@@ -1,95 +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.datastore.chunk.impl;
-
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionDataChunkStore;
-
-/**
- * Class responsibility is to give access to dimension column data chunk store
- */
-public abstract class AbstractDimensionDataChunk implements DimensionColumnDataChunk {
-
-  /**
-   * data chunks
-   */
-  protected DimensionDataChunkStore dataChunkStore;
-
-  /**
-   * @return whether columns where explicitly sorted or not
-   */
-  @Override public boolean isExplicitSorted() {
-    return dataChunkStore.isExplicitSorted();
-  }
-
-  /**
-   * Below method to get the data based in row id
-   *
-   * @param index row id of the data
-   * @return chunk
-   */
-  @Override public byte[] getChunkData(int index) {
-    return dataChunkStore.getRow(index);
-  }
-
-  /**
-   * @return inverted index
-   */
-  @Override public int getInvertedIndex(int index) {
-    return dataChunkStore.getInvertedIndex(index);
-  }
-
-  /**
-   * @param invertedIndex
-   * @return inverted index reverse
-   */
-  @Override public int getInvertedReverseIndex(int invertedIndex) {
-    return dataChunkStore.getInvertedReverseIndex(invertedIndex);
-  }
-  /**
-   * @return length of each column
-   */
-  @Override public int getColumnValueSize() {
-    return dataChunkStore.getColumnValueSize();
-  }
-
-  /**
-   * To compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  @Override public int compareTo(int index, byte[] compareValue) {
-    // TODO Auto-generated method stub
-    return dataChunkStore.compareTo(index, compareValue);
-  }
-
-  /**
-   * below method will be used to free the allocated memory
-   */
-  @Override public void freeMemory() {
-    dataChunkStore.freeMemory();
-  }
-
-  /**
-   * @return column is dictionary column or not
-   */
-  @Override public boolean isNoDicitionaryColumn() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java
new file mode 100644
index 0000000..741c13d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java
@@ -0,0 +1,194 @@
+/*
+ * 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.datastore.chunk.impl;
+
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+
+/**
+ * This class is gives access to column group dimension data chunk store
+ */
+public class ColumnGroupDimensionColumnPage extends AbstractDimensionColumnPage {
+
+  /**
+   * Constructor for this class
+   *
+   * @param dataChunk       data chunk
+   * @param columnValueSize chunk attributes
+   * @param numberOfRows
+   */
+  public ColumnGroupDimensionColumnPage(byte[] dataChunk, int columnValueSize, int numberOfRows) {
+    this.dataChunkStore = DimensionChunkStoreFactory.INSTANCE
+        .getDimensionChunkStore(columnValueSize, false, numberOfRows, dataChunk.length,
+        DimensionStoreType.FIXEDLENGTH);
+    this.dataChunkStore.putArray(null, null, dataChunk);
+  }
+
+  /**
+   * Below method will be used to fill the data based on offset and row id
+   *
+   * @param rowId             row id of the chunk
+   * @param offset            offset from which data need to be filed
+   * @param data              data to filed
+   * @param restructuringInfo define the structure of the key
+   * @return how many bytes was copied
+   */
+  @Override public int fillRawData(int rowId, int offset, byte[] data,
+      KeyStructureInfo restructuringInfo) {
+    byte[] row = dataChunkStore.getRow(rowId);
+    byte[] maskedKey = getMaskedKey(row, restructuringInfo);
+    System.arraycopy(maskedKey, 0, data, offset, maskedKey.length);
+    return maskedKey.length;
+  }
+
+  /**
+   * Converts to column dictionary integer value
+   *
+   * @param rowId
+   * @param chunkIndex
+   * @param outputSurrogateKey
+   * @param info          KeyStructureInfo
+   * @return
+   */
+  @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
+      KeyStructureInfo info) {
+    byte[] data = dataChunkStore.getRow(rowId);
+    long[] keyArray = info.getKeyGenerator().getKeyArray(data);
+    int[] ordinal = info.getMdkeyQueryDimensionOrdinal();
+    for (int i = 0; i < ordinal.length; i++) {
+      outputSurrogateKey[chunkIndex++] = (int) keyArray[ordinal[i]];
+    }
+    return chunkIndex;
+  }
+
+  /**
+   * Below method will be used to get the masked key
+   *
+   * @param data   data
+   * @param info
+   * @return
+   */
+  private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) {
+    byte[] maskedKey = new byte[info.getMaskByteRanges().length];
+    int counter = 0;
+    int byteRange = 0;
+    for (int i = 0; i < info.getMaskByteRanges().length; i++) {
+      byteRange = info.getMaskByteRanges()[i];
+      maskedKey[counter++] = (byte) (data[byteRange] & info.getMaxKey()[byteRange]);
+    }
+    return maskedKey;
+  }
+
+  /**
+   * @return inverted index
+   */
+  @Override public int getInvertedIndex(int rowId) {
+    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
+  }
+
+  /**
+   * @param rowId
+   * @return inverted index reverse
+   */
+  @Override public int getInvertedReverseIndex(int rowId) {
+    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
+  }
+
+  /**
+   * @return whether columns where explictly sorted or not
+   */
+  @Override public boolean isExplicitSorted() {
+    return false;
+  }
+
+  /**
+   * to compare the data
+   *
+   * @param rowId        row index to be compared
+   * @param compareValue value to compare
+   * @return compare result
+   */
+  @Override public int compareTo(int rowId, byte[] compareValue) {
+    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
+    for (int k = offset; k < len; k++) {
+      long[] keyArray = restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(k));
+      int index = 0;
+      for (int i = chunkIndex; i < chunkIndex + ordinal.length; i++) {
+        if (vectorInfo[i].directDictionaryGenerator == null) {
+          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
+        } else {
+          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
+              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
+        }
+      }
+      vectorOffset++;
+    }
+    return chunkIndex + ordinal.length;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param filteredRowId
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo,
+      int chunkIndex, KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
+    for (int k = offset; k < len; k++) {
+      long[] keyArray =
+          restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(filteredRowId[k]));
+      int index = 0;
+      for (int i = chunkIndex; i < chunkIndex + ordinal.length; i++) {
+        if (vectorInfo[i].directDictionaryGenerator == null) {
+          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
+        } else {
+          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
+              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
+        }
+      }
+      vectorOffset++;
+    }
+    return chunkIndex + ordinal.length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
deleted file mode 100644
index b76ae53..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
+++ /dev/null
@@ -1,194 +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.datastore.chunk.impl;
-
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-
-/**
- * This class is gives access to column group dimension data chunk store
- */
-public class ColumnGroupDimensionDataChunk extends AbstractDimensionDataChunk {
-
-  /**
-   * Constructor for this class
-   *
-   * @param dataChunk       data chunk
-   * @param columnValueSize chunk attributes
-   * @param numberOfRows
-   */
-  public ColumnGroupDimensionDataChunk(byte[] dataChunk, int columnValueSize, int numberOfRows) {
-    this.dataChunkStore = DimensionChunkStoreFactory.INSTANCE
-        .getDimensionChunkStore(columnValueSize, false, numberOfRows, dataChunk.length,
-        DimensionStoreType.FIXEDLENGTH);
-    this.dataChunkStore.putArray(null, null, dataChunk);
-  }
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data              data to filed
-   * @param offset            offset from which data need to be filed
-   * @param rowId             row id of the chunk
-   * @param restructuringInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int rowId,
-      KeyStructureInfo restructuringInfo) {
-    byte[] row = dataChunkStore.getRow(rowId);
-    byte[] maskedKey = getMaskedKey(row, restructuringInfo);
-    System.arraycopy(maskedKey, 0, data, offset, maskedKey.length);
-    return maskedKey.length;
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param info          KeyStructureInfo
-   * @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo info) {
-    byte[] data = dataChunkStore.getRow(rowId);
-    long[] keyArray = info.getKeyGenerator().getKeyArray(data);
-    int[] ordinal = info.getMdkeyQueryDimensionOrdinal();
-    for (int i = 0; i < ordinal.length; i++) {
-      row[columnIndex++] = (int) keyArray[ordinal[i]];
-    }
-    return columnIndex;
-  }
-
-  /**
-   * Below method will be used to get the masked key
-   *
-   * @param data   data
-   * @param info
-   * @return
-   */
-  private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) {
-    byte[] maskedKey = new byte[info.getMaskByteRanges().length];
-    int counter = 0;
-    int byteRange = 0;
-    for (int i = 0; i < info.getMaskByteRanges().length; i++) {
-      byteRange = info.getMaskByteRanges()[i];
-      maskedKey[counter++] = (byte) (data[byteRange] & info.getMaxKey()[byteRange]);
-    }
-    return maskedKey;
-  }
-
-  /**
-   * @return inverted index
-   */
-  @Override public int getInvertedIndex(int index) {
-    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
-  }
-
-  /**
-   * @param invertedIndex
-   * @return inverted index reverse
-   */
-  @Override public int getInvertedReverseIndex(int invertedIndex) {
-    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
-  }
-
-  /**
-   * @return whether columns where explictly sorted or not
-   */
-  @Override public boolean isExplicitSorted() {
-    return false;
-  }
-
-  /**
-   * to compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  @Override public int compareTo(int index, byte[] compareValue) {
-    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
-    for (int k = offset; k < len; k++) {
-      long[] keyArray = restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(k));
-      int index = 0;
-      for (int i = column; i < column + ordinal.length; i++) {
-        if (vectorInfo[i].directDictionaryGenerator == null) {
-          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
-        } else {
-          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
-              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
-        }
-      }
-      vectorOffset++;
-    }
-    return column + ordinal.length;
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param rowMapping
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo,
-      int column, KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
-    for (int k = offset; k < len; k++) {
-      long[] keyArray =
-          restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(rowMapping[k]));
-      int index = 0;
-      for (int i = column; i < column + ordinal.length; i++) {
-        if (vectorInfo[i].directDictionaryGenerator == null) {
-          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
-        } else {
-          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
-              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
-        }
-      }
-      vectorOffset++;
-    }
-    return column + ordinal.length;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
index 705c13c..f9bb590 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
@@ -19,25 +19,25 @@ package org.apache.carbondata.core.datastore.chunk.impl;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.AbstractRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
 import org.apache.carbondata.core.memory.MemoryException;
 
 /**
  * Contains raw dimension data,
  * 1. The read uncompressed raw data of column chunk with all pages is stored in this instance.
- * 2. The raw data can be converted to processed chunk using convertToDimColDataChunk method
+ * 2. The raw data can be converted to processed chunk using decodeColumnPage method
  *  by specifying page number.
  */
 public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
 
-  private DimensionColumnDataChunk[] dataChunks;
+  private DimensionColumnPage[] dataChunks;
 
   private DimensionColumnChunkReader chunkReader;
 
-  private FileHolder fileHolder;
+  private FileReader fileReader;
 
   public DimensionRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, int length,
       DimensionColumnChunkReader columnChunkReader) {
@@ -46,17 +46,17 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
   }
 
   /**
-   * Convert all raw data with all pages to processed DimensionColumnDataChunk's
+   * Convert all raw data with all pages to processed DimensionColumnPage's
    * @return
    */
-  public DimensionColumnDataChunk[] convertToDimColDataChunks() {
+  public DimensionColumnPage[] decodeAllColumnPages() {
     if (dataChunks == null) {
-      dataChunks = new DimensionColumnDataChunk[pagesCount];
+      dataChunks = new DimensionColumnPage[pagesCount];
     }
     for (int i = 0; i < pagesCount; i++) {
       try {
         if (dataChunks[i] == null) {
-          dataChunks[i] = chunkReader.convertToDimensionChunk(this, i);
+          dataChunks[i] = chunkReader.decodeColumnPage(this, i);
         }
       } catch (IOException | MemoryException e) {
         throw new RuntimeException(e);
@@ -66,24 +66,24 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
   }
 
   /**
-   * Convert raw data with specified page number processed to DimensionColumnDataChunk
-   * @param index
+   * Convert raw data with specified page number processed to DimensionColumnPage
+   * @param pageNumber
    * @return
    */
-  public DimensionColumnDataChunk convertToDimColDataChunk(int index) {
-    assert index < pagesCount;
+  public DimensionColumnPage decodeColumnPage(int pageNumber) {
+    assert pageNumber < pagesCount;
     if (dataChunks == null) {
-      dataChunks = new DimensionColumnDataChunk[pagesCount];
+      dataChunks = new DimensionColumnPage[pagesCount];
     }
-    if (dataChunks[index] == null) {
+    if (dataChunks[pageNumber] == null) {
       try {
-        dataChunks[index] = chunkReader.convertToDimensionChunk(this, index);
+        dataChunks[pageNumber] = chunkReader.decodeColumnPage(this, pageNumber);
       } catch (IOException | MemoryException e) {
         throw new RuntimeException(e);
       }
     }
 
-    return dataChunks[index];
+    return dataChunks[pageNumber];
   }
 
   /**
@@ -92,10 +92,10 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
    * @param index
    * @return
    */
-  public DimensionColumnDataChunk convertToDimColDataChunkWithOutCache(int index) {
+  public DimensionColumnPage convertToDimColDataChunkWithOutCache(int index) {
     assert index < pagesCount;
     try {
-      return chunkReader.convertToDimensionChunk(this, index);
+      return chunkReader.decodeColumnPage(this, index);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -111,11 +111,11 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
     }
   }
 
-  public void setFileHolder(FileHolder fileHolder) {
-    this.fileHolder = fileHolder;
+  public void setFileReader(FileReader fileReader) {
+    this.fileReader = fileReader;
   }
 
-  public FileHolder getFileReader() {
-    return fileHolder;
+  public FileReader getFileReader() {
+    return fileReader;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e3077c4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java
new file mode 100644
index 0000000..ff54b12
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java
@@ -0,0 +1,163 @@
+/*
+ * 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.datastore.chunk.impl;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+
+/**
+ * This class is gives access to fixed length dimension data chunk store
+ */
+public class FixedLengthDimensionColumnPage extends AbstractDimensionColumnPage {
+
+  /**
+   * Constructor
+   *
+   * @param dataChunk            data chunk
+   * @param invertedIndex        inverted index
+   * @param invertedIndexReverse reverse inverted index
+   * @param numberOfRows         number of rows
+   * @param columnValueSize      size of each column value
+   */
+  public FixedLengthDimensionColumnPage(byte[] dataChunk, int[] invertedIndex,
+      int[] invertedIndexReverse, int numberOfRows, int columnValueSize) {
+    long totalSize = null != invertedIndex ?
+        dataChunk.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) :
+        dataChunk.length;
+    dataChunkStore = DimensionChunkStoreFactory.INSTANCE
+        .getDimensionChunkStore(columnValueSize, null != invertedIndex, numberOfRows, totalSize,
+            DimensionStoreType.FIXEDLENGTH);
+    dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunk);
+  }
+
+  /**
+   * Below method will be used to fill the data based on offset and row id
+   *
+   * @param rowId            row id of the chunk
+   * @param offset           offset from which data need to be filed
+   * @param data             data to filed
+   * @param keyStructureInfo define the structure of the key
+   * @return how many bytes was copied
+   */
+  @Override public int fillRawData(int rowId, int offset, byte[] data,
+      KeyStructureInfo keyStructureInfo) {
+    dataChunkStore.fillRow(rowId, data, offset);
+    return dataChunkStore.getColumnValueSize();
+  }
+
+  /**
+   * Converts to column dictionary integer value
+   *
+   * @param rowId
+   * @param chunkIndex
+   * @param outputSurrogateKey
+   * @param restructuringInfo
+   * @return
+   */
+  @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
+      KeyStructureInfo restructuringInfo) {
+    outputSurrogateKey[chunkIndex] = dataChunkStore.getSurrogate(rowId);
+    return chunkIndex + 1;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = columnVectorInfo.size + offset;
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    for (int j = offset; j < len; j++) {
+      int dict = dataChunkStore.getSurrogate(j);
+      if (columnVectorInfo.directDictionaryGenerator == null) {
+        vector.putInt(vectorOffset++, dict);
+      } else {
+        Object valueFromSurrogate =
+            columnVectorInfo.directDictionaryGenerator.getValueFromSurrogate(dict);
+        if (valueFromSurrogate == null) {
+          vector.putNull(vectorOffset++);
+        } else {
+          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
+          if (dataType == DataTypes.INT) {
+            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
+          } else if (dataType == DataTypes.LONG) {
+            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
+          } else {
+            throw new IllegalArgumentException("unsupported data type: " +
+                columnVectorInfo.directDictionaryGenerator.getReturnType());
+          }
+        }
+      }
+    }
+    return chunkIndex + 1;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param filteredRowId
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo,
+      int chunkIndex, KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = columnVectorInfo.size + offset;
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    for (int j = offset; j < len; j++) {
+      int dict = dataChunkStore.getSurrogate(filteredRowId[j]);
+      if (columnVectorInfo.directDictionaryGenerator == null) {
+        vector.putInt(vectorOffset++, dict);
+      } else {
+        Object valueFromSurrogate =
+            columnVectorInfo.directDictionaryGenerator.getValueFromSurrogate(dict);
+        if (valueFromSurrogate == null) {
+          vector.putNull(vectorOffset++);
+        } else {
+          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
+          if (dataType == DataTypes.INT) {
+            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
+          } else if (dataType == DataTypes.LONG) {
+            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
+          } else {
+            throw new IllegalArgumentException("unsupported data type: " +
+                columnVectorInfo.directDictionaryGenerator.getReturnType());
+          }
+        }
+      }
+    }
+    return chunkIndex + 1;
+  }
+}