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/03/02 08:01:36 UTC

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index 69f5ceb..22d1df1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -43,10 +43,9 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode;
 import org.apache.carbondata.core.indexstore.blockletindex.IndexWrapper;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -64,8 +63,8 @@ import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -121,7 +120,6 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     queryProperties.queryStatisticsRecorder =
         CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId());
     queryModel.setStatisticsRecorder(queryProperties.queryStatisticsRecorder);
-    QueryUtil.resolveQueryModel(queryModel);
     QueryStatistic queryStatistic = new QueryStatistic();
     // sort the block info
     // so block will be loaded in sorted order this will be required for
@@ -168,12 +166,12 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
         .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, System.currentTimeMillis());
     queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
     // calculating the total number of aggeragted columns
-    int measureCount = queryModel.getQueryMeasures().size();
+    int measureCount = queryModel.getProjectionMeasures().size();
 
     int currentIndex = 0;
     DataType[] dataTypes = new DataType[measureCount];
 
-    for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
+    for (ProjectionMeasure carbonMeasure : queryModel.getProjectionMeasures()) {
       // adding the data type and aggregation type of all the measure this
       // can be used
       // to select the aggregator
@@ -198,9 +196,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     queryStatistic = new QueryStatistic();
     // dictionary column unique column id to dictionary mapping
     // which will be used to get column actual data
-    queryProperties.columnToDictionayMapping = QueryUtil
-        .getDimensionDictionaryDetail(queryModel.getQueryDimension(),
-            queryProperties.complexFilterDimension, queryModel.getAbsoluteTableIdentifier(),
+    queryProperties.columnToDictionayMapping =
+        QueryUtil.getDimensionDictionaryDetail(
+            queryModel.getProjectionDimensions(),
+            queryProperties.complexFilterDimension,
+            queryModel.getAbsoluteTableIdentifier(),
             tableProvider);
     queryStatistic
         .addStatistics(QueryStatisticsConstants.LOAD_DICTIONARY, System.currentTimeMillis());
@@ -263,8 +263,8 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // and query will be executed based on that infos
     for (int i = 0; i < queryProperties.dataBlocks.size(); i++) {
       AbstractIndex abstractIndex = queryProperties.dataBlocks.get(i);
-      BlockletDataRefNodeWrapper dataRefNode =
-          (BlockletDataRefNodeWrapper) abstractIndex.getDataRefNode();
+      BlockletDataRefNode dataRefNode =
+          (BlockletDataRefNode) abstractIndex.getDataRefNode();
       blockExecutionInfoList.add(getBlockExecutionInfoForBlock(queryModel, abstractIndex,
           dataRefNode.getBlockInfos().get(0).getBlockletInfos().getStartBlockletNumber(),
           dataRefNode.numberOfNodes(), dataRefNode.getBlockInfos().get(0).getFilePath(),
@@ -289,47 +289,45 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
    * @return block execution info
    * @throws QueryExecutionException any failure during block info creation
    */
-  protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
+  private BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
       AbstractIndex blockIndex, int startBlockletIndex, int numberOfBlockletToScan, String filePath,
       String[] deleteDeltaFiles, String segmentId)
       throws QueryExecutionException {
     BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
     SegmentProperties segmentProperties = blockIndex.getSegmentProperties();
     List<CarbonDimension> tableBlockDimensions = segmentProperties.getDimensions();
-    KeyGenerator blockKeyGenerator = segmentProperties.getDimensionKeyGenerator();
 
     // below is to get only those dimension in query which is present in the
     // table block
-    List<QueryDimension> currentBlockQueryDimensions = RestructureUtil
+    List<ProjectionDimension> projectDimensions = RestructureUtil
         .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo,
-            queryModel.getQueryDimension(), tableBlockDimensions,
-            segmentProperties.getComplexDimensions(), queryModel.getQueryMeasures().size());
+            queryModel.getProjectionDimensions(), tableBlockDimensions,
+            segmentProperties.getComplexDimensions(), queryModel.getProjectionMeasures().size());
     blockExecutionInfo.setBlockId(
         CarbonUtil.getBlockId(queryModel.getAbsoluteTableIdentifier(), filePath, segmentId));
     blockExecutionInfo.setDeleteDeltaFilePath(deleteDeltaFiles);
     blockExecutionInfo.setStartBlockletIndex(startBlockletIndex);
     blockExecutionInfo.setNumberOfBlockletToScan(numberOfBlockletToScan);
-    blockExecutionInfo.setQueryDimensions(currentBlockQueryDimensions
-        .toArray(new QueryDimension[currentBlockQueryDimensions.size()]));
+    blockExecutionInfo.setProjectionDimensions(projectDimensions
+        .toArray(new ProjectionDimension[projectDimensions.size()]));
     // get measures present in the current block
-    List<QueryMeasure> currentBlockQueryMeasures =
+    List<ProjectionMeasure> currentBlockQueryMeasures =
         getCurrentBlockQueryMeasures(blockExecutionInfo, queryModel, blockIndex);
-    blockExecutionInfo.setQueryMeasures(
-        currentBlockQueryMeasures.toArray(new QueryMeasure[currentBlockQueryMeasures.size()]));
+    blockExecutionInfo.setProjectionMeasures(
+        currentBlockQueryMeasures.toArray(new ProjectionMeasure[currentBlockQueryMeasures.size()]));
     blockExecutionInfo.setDataBlock(blockIndex);
-    blockExecutionInfo.setBlockKeyGenerator(blockKeyGenerator);
     // setting whether raw record query or not
     blockExecutionInfo.setRawRecordDetailQuery(queryModel.isForcedDetailRawQuery());
     // total number dimension
     blockExecutionInfo
-        .setTotalNumberDimensionBlock(segmentProperties.getDimensionOrdinalToBlockMapping().size());
+        .setTotalNumberDimensionToRead(
+            segmentProperties.getDimensionOrdinalToChunkMapping().size());
     blockExecutionInfo.setPrefetchBlocklet(!queryModel.isReadPageByPage());
     blockExecutionInfo
-        .setTotalNumberOfMeasureBlock(segmentProperties.getMeasuresOrdinalToBlockMapping().size());
-    blockExecutionInfo.setAbsoluteTableIdentifier(queryModel.getAbsoluteTableIdentifier());
+        .setTotalNumberOfMeasureToRead(segmentProperties.getMeasuresOrdinalToChunkMapping().size());
     blockExecutionInfo.setComplexDimensionInfoMap(QueryUtil
-        .getComplexDimensionsMap(currentBlockQueryDimensions,
-            segmentProperties.getDimensionOrdinalToBlockMapping(),
+        .getComplexDimensionsMap(projectDimensions,
+            segmentProperties.getDimensionOrdinalToChunkMapping(),
             segmentProperties.getEachComplexDimColumnValueSize(),
             queryProperties.columnToDictionayMapping, queryProperties.complexFilterDimension));
     IndexKey startIndexKey = null;
@@ -363,45 +361,46 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // create a list of filter dimensions present in the current block
     Set<CarbonDimension> currentBlockFilterDimensions =
         getCurrentBlockFilterDimensions(queryProperties.complexFilterDimension, segmentProperties);
-    int[] dimensionsBlockIndexes = QueryUtil.getDimensionsBlockIndexes(currentBlockQueryDimensions,
-        segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions,
-        currentBlockFilterDimensions, allProjectionListDimensionIdexes);
+    int[] dimensionChunkIndexes = QueryUtil.getDimensionChunkIndexes(
+        projectDimensions, segmentProperties.getDimensionOrdinalToChunkMapping(),
+        expressionDimensions, currentBlockFilterDimensions, allProjectionListDimensionIdexes);
     int numberOfColumnToBeReadInOneIO = Integer.parseInt(CarbonProperties.getInstance()
         .getProperty(CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO,
             CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE));
 
-    if (dimensionsBlockIndexes.length > 0) {
-      numberOfElementToConsider = dimensionsBlockIndexes[dimensionsBlockIndexes.length - 1]
+    if (dimensionChunkIndexes.length > 0) {
+      numberOfElementToConsider = dimensionChunkIndexes[dimensionChunkIndexes.length - 1]
           == segmentProperties.getBlockTodimensionOrdinalMapping().size() - 1 ?
-          dimensionsBlockIndexes.length - 1 :
-          dimensionsBlockIndexes.length;
-      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(CarbonUtil
-          .getRangeIndex(dimensionsBlockIndexes, numberOfElementToConsider,
+          dimensionChunkIndexes.length - 1 :
+          dimensionChunkIndexes.length;
+      blockExecutionInfo.setAllSelectedDimensionColumnIndexRange(
+          CarbonUtil.getRangeIndex(dimensionChunkIndexes, numberOfElementToConsider,
               numberOfColumnToBeReadInOneIO));
     } else {
-      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[0][0]);
+      blockExecutionInfo.setAllSelectedDimensionColumnIndexRange(new int[0][0]);
     }
     // get the list of updated filter measures present in the current block
-    Set<CarbonMeasure> currentBlockFilterMeasures =
+    Set<CarbonMeasure> filterMeasures =
         getCurrentBlockFilterMeasures(queryProperties.filterMeasures, segmentProperties);
     // list of measures to be projected
     List<Integer> allProjectionListMeasureIndexes = new ArrayList<>();
-    int[] measureBlockIndexes = QueryUtil
-        .getMeasureBlockIndexes(currentBlockQueryMeasures, expressionMeasures,
-            segmentProperties.getMeasuresOrdinalToBlockMapping(), currentBlockFilterMeasures,
-            allProjectionListMeasureIndexes);
-    if (measureBlockIndexes.length > 0) {
+    int[] measureChunkIndexes = QueryUtil.getMeasureChunkIndexes(
+        currentBlockQueryMeasures, expressionMeasures,
+        segmentProperties.getMeasuresOrdinalToChunkMapping(), filterMeasures,
+        allProjectionListMeasureIndexes);
+    if (measureChunkIndexes.length > 0) {
 
-      numberOfElementToConsider = measureBlockIndexes[measureBlockIndexes.length - 1]
+      numberOfElementToConsider = measureChunkIndexes[measureChunkIndexes.length - 1]
           == segmentProperties.getMeasures().size() - 1 ?
-          measureBlockIndexes.length - 1 :
-          measureBlockIndexes.length;
+          measureChunkIndexes.length - 1 :
+          measureChunkIndexes.length;
       // setting all the measure chunk indexes to be read from file
-      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(CarbonUtil
-          .getRangeIndex(measureBlockIndexes, numberOfElementToConsider,
+      blockExecutionInfo.setAllSelectedMeasureIndexRange(
+          CarbonUtil.getRangeIndex(
+              measureChunkIndexes, numberOfElementToConsider,
               numberOfColumnToBeReadInOneIO));
     } else {
-      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[0][0]);
+      blockExecutionInfo.setAllSelectedMeasureIndexRange(new int[0][0]);
     }
     // setting the indexes of list of dimension in projection list
     blockExecutionInfo.setProjectionListDimensionIndexes(ArrayUtils.toPrimitive(
@@ -413,27 +412,27 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
             .toArray(new Integer[allProjectionListMeasureIndexes.size()])));
     // setting the size of fixed key column (dictionary column)
     blockExecutionInfo
-        .setFixedLengthKeySize(getKeySize(currentBlockQueryDimensions, segmentProperties));
-    Set<Integer> dictionaryColumnBlockIndex = new HashSet<Integer>();
-    List<Integer> noDictionaryColumnBlockIndex = new ArrayList<Integer>();
+        .setFixedLengthKeySize(getKeySize(projectDimensions, segmentProperties));
+    Set<Integer> dictionaryColumnChunkIndex = new HashSet<Integer>();
+    List<Integer> noDictionaryColumnChunkIndex = new ArrayList<Integer>();
     // get the block index to be read from file for query dimension
     // for both dictionary columns and no dictionary columns
-    QueryUtil.fillQueryDimensionsBlockIndexes(currentBlockQueryDimensions,
-        segmentProperties.getDimensionOrdinalToBlockMapping(), dictionaryColumnBlockIndex,
-        noDictionaryColumnBlockIndex);
-    int[] queryDictionaryColumnBlockIndexes = ArrayUtils.toPrimitive(
-        dictionaryColumnBlockIndex.toArray(new Integer[dictionaryColumnBlockIndex.size()]));
+    QueryUtil.fillQueryDimensionChunkIndexes(projectDimensions,
+        segmentProperties.getDimensionOrdinalToChunkMapping(), dictionaryColumnChunkIndex,
+        noDictionaryColumnChunkIndex);
+    int[] queryDictionaryColumnChunkIndexes = ArrayUtils.toPrimitive(
+        dictionaryColumnChunkIndex.toArray(new Integer[dictionaryColumnChunkIndex.size()]));
     // need to sort the dictionary column as for all dimension
     // column key will be filled based on key order
-    Arrays.sort(queryDictionaryColumnBlockIndexes);
-    blockExecutionInfo.setDictionaryColumnBlockIndex(queryDictionaryColumnBlockIndexes);
+    Arrays.sort(queryDictionaryColumnChunkIndexes);
+    blockExecutionInfo.setDictionaryColumnChunkIndex(queryDictionaryColumnChunkIndexes);
     // setting the no dictionary column block indexes
-    blockExecutionInfo.setNoDictionaryBlockIndexes(ArrayUtils.toPrimitive(
-        noDictionaryColumnBlockIndex.toArray(new Integer[noDictionaryColumnBlockIndex.size()])));
+    blockExecutionInfo.setNoDictionaryColumnChunkIndexes(ArrayUtils.toPrimitive(
+        noDictionaryColumnChunkIndex.toArray(new Integer[noDictionaryColumnChunkIndex.size()])));
     // setting each column value size
     blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize());
     blockExecutionInfo.setComplexColumnParentBlockIndexes(
-        getComplexDimensionParentBlockIndexes(currentBlockQueryDimensions));
+        getComplexDimensionParentBlockIndexes(projectDimensions));
     blockExecutionInfo.setVectorBatchCollector(queryModel.isVectorReader());
     try {
       // to set column group and its key structure info which will be used
@@ -441,15 +440,15 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       // for getting the column group column data in case of final row
       // and in case of dimension aggregation
       blockExecutionInfo.setColumnGroupToKeyStructureInfo(
-          QueryUtil.getColumnGroupKeyStructureInfo(currentBlockQueryDimensions, segmentProperties));
+          QueryUtil.getColumnGroupKeyStructureInfo(projectDimensions, segmentProperties));
     } catch (KeyGenException e) {
       throw new QueryExecutionException(e);
     }
     // set actual query dimensions and measures. It may differ in case of restructure scenarios
-    blockExecutionInfo.setActualQueryDimensions(queryModel.getQueryDimension()
-        .toArray(new QueryDimension[queryModel.getQueryDimension().size()]));
-    blockExecutionInfo.setActualQueryMeasures(queryModel.getQueryMeasures()
-        .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
+    blockExecutionInfo.setActualQueryDimensions(queryModel.getProjectionDimensions()
+        .toArray(new ProjectionDimension[queryModel.getProjectionDimensions().size()]));
+    blockExecutionInfo.setActualQueryMeasures(queryModel.getProjectionMeasures()
+        .toArray(new ProjectionMeasure[queryModel.getProjectionMeasures().size()]));
     DataTypeUtil.setDataTypeConverter(queryModel.getConverter());
     return blockExecutionInfo;
   }
@@ -464,7 +463,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
    * @param blockMetadataInfo block metadata info
    * @return key size
    */
-  private int getKeySize(List<QueryDimension> queryDimension,
+  private int getKeySize(List<ProjectionDimension> queryDimension,
       SegmentProperties blockMetadataInfo) {
     // add the dimension block ordinal for each dictionary column
     // existing in the current block dimensions. Set is used because in case of column groups
@@ -475,12 +474,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     while (counter < queryDimension.size()) {
       if (queryDimension.get(counter).getDimension().getNumberOfChild() > 0) {
         counter += queryDimension.get(counter).getDimension().getNumberOfChild();
-        continue;
       } else if (!CarbonUtil.hasEncoding(queryDimension.get(counter).getDimension().getEncoder(),
           Encoding.DICTIONARY)) {
         counter++;
       } else {
-        fixedLengthDimensionOrdinal.add(blockMetadataInfo.getDimensionOrdinalToBlockMapping()
+        fixedLengthDimensionOrdinal.add(blockMetadataInfo.getDimensionOrdinalToChunkMapping()
             .get(queryDimension.get(counter).getDimension().getOrdinal()));
         counter++;
       }
@@ -502,26 +500,26 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
   /**
    * Below method will be used to get the measures present in the current block
    *
-   * @param blockExecutionInfo
+   * @param executionInfo
    * @param queryModel         query model
    * @param tableBlock         table block
    * @return
    */
-  private List<QueryMeasure> getCurrentBlockQueryMeasures(BlockExecutionInfo blockExecutionInfo,
+  private List<ProjectionMeasure> getCurrentBlockQueryMeasures(BlockExecutionInfo executionInfo,
       QueryModel queryModel, AbstractIndex tableBlock) throws QueryExecutionException {
     // getting the measure info which will be used while filling up measure data
-    List<QueryMeasure> updatedQueryMeasures = RestructureUtil
-        .createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo,
-            queryModel.getQueryMeasures(), tableBlock.getSegmentProperties().getMeasures());
+    List<ProjectionMeasure> updatedQueryMeasures = RestructureUtil
+        .createMeasureInfoAndGetCurrentBlockQueryMeasures(executionInfo,
+            queryModel.getProjectionMeasures(), tableBlock.getSegmentProperties().getMeasures());
     // setting the measure aggregator for all aggregation function selected
     // in query
-    blockExecutionInfo.getMeasureInfo().setMeasureDataTypes(queryProperties.measureDataTypes);
+    executionInfo.getMeasureInfo().setMeasureDataTypes(queryProperties.measureDataTypes);
     return updatedQueryMeasures;
   }
 
-  private int[] getComplexDimensionParentBlockIndexes(List<QueryDimension> queryDimensions) {
+  private int[] getComplexDimensionParentBlockIndexes(List<ProjectionDimension> queryDimensions) {
     List<Integer> parentBlockIndexList = new ArrayList<Integer>();
-    for (QueryDimension queryDimension : queryDimensions) {
+    for (ProjectionDimension queryDimension : queryDimensions) {
       if (queryDimension.getDimension().getDataType().isComplexType()) {
         parentBlockIndexList.add(queryDimension.getDimension().getOrdinal());
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
index 931b1ab..93d696b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
@@ -23,7 +23,7 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.result.iterator.DetailQueryResultIterator;
 
 /**
@@ -31,10 +31,10 @@ import org.apache.carbondata.core.scan.result.iterator.DetailQueryResultIterator
  * For executing the detail query it will pass all the block execution
  * info to detail query result iterator and iterator will be returned
  */
-public class DetailQueryExecutor extends AbstractQueryExecutor<BatchResult> {
+public class DetailQueryExecutor extends AbstractQueryExecutor<RowBatch> {
 
   @Override
-  public CarbonIterator<BatchResult> execute(QueryModel queryModel)
+  public CarbonIterator<RowBatch> execute(QueryModel queryModel)
       throws QueryExecutionException, IOException {
     List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
     this.queryIterator = new DetailQueryResultIterator(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
index 65f2838..06adf8f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
@@ -21,13 +21,11 @@ import java.util.Map;
 import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.mutate.DeleteDeltaVo;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 
 /**
  * Below class will have all the properties which needed during query execution
@@ -41,14 +39,6 @@ public class BlockExecutionInfo {
   private AbstractIndex blockIndex;
 
   /**
-   * each segment key size can be different and in that case we need to update
-   * the fixed key with latest segment key generator. so this property will
-   * tell whether this is required or not if key size is same then it is not
-   * required
-   */
-  private boolean isFixedKeyUpdateRequired;
-
-  /**
    * below to store all the information required for measures during query
    * execution
    */
@@ -73,22 +63,22 @@ public class BlockExecutionInfo {
   /**
    * total number of dimension in block
    */
-  private int totalNumberDimensionBlock;
+  private int totalNumberDimensionToRead;
 
   /**
    * total number of measure in block
    */
-  private int totalNumberOfMeasureBlock;
+  private int totalNumberOfMeasureToRead;
 
   /**
    * will be used to read the dimension block from file
    */
-  private int[][] allSelectedDimensionBlocksIndexes;
+  private int[][] allSelectedDimensionColumnIndexRange;
 
   /**
    * will be used to read the measure block from file
    */
-  private int[][] allSelectedMeasureBlocksIndexes;
+  private int[][] allSelectedMeasureIndexRange;
 
   /**
    * list of dimension present in the projection
@@ -118,16 +108,11 @@ public class BlockExecutionInfo {
   /**
    * dictionary column block indexes based on query
    */
-  private int[] dictionaryColumnBlockIndex;
+  private int[] dictionaryColumnChunkIndex;
   /**
    * no dictionary column block indexes in based on the query order
    */
-  private int[] noDictionaryBlockIndexes;
-
-  /**
-   * key generator used for generating the table block fixed length key
-   */
-  private KeyGenerator blockKeyGenerator;
+  private int[] noDictionaryColumnChunkIndexes;
 
   /**
    * each column value size
@@ -180,23 +165,23 @@ public class BlockExecutionInfo {
    * list of dimension present in the current block. This will be
    * different in case of restructured block
    */
-  private QueryDimension[] queryDimensions;
+  private ProjectionDimension[] projectionDimensions;
 
   /**
    * list of dimension selected for in query
    */
-  private QueryDimension[] actualQueryDimensions;
+  private ProjectionDimension[] actualQueryDimensions;
 
   /**
    * list of dimension present in the current block. This will be
    * different in case of restructured block
    */
-  private QueryMeasure[] queryMeasures;
+  private ProjectionMeasure[] projectionMeasures;
 
   /**
    * list of measure selected in query
    */
-  private QueryMeasure[] actualQueryMeasures;
+  private ProjectionMeasure[] actualQueryMeasures;
 
   /**
    * variable to maintain dimension existence and default value info
@@ -214,11 +199,6 @@ public class BlockExecutionInfo {
   private boolean isRestructuredBlock;
 
   /**
-   * absolute table identifier
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-
-  /**
    * delete delta file path
    */
   private String[] deleteDeltaFilePath;
@@ -229,13 +209,6 @@ public class BlockExecutionInfo {
   private boolean prefetchBlocklet = true;
 
   private Map<String, DeleteDeltaVo> deletedRecordsMap;
-  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-  }
 
   /**
    * @param blockIndex the tableBlock to set
@@ -245,20 +218,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the isFixedKeyUpdateRequired
-   */
-  public boolean isFixedKeyUpdateRequired() {
-    return isFixedKeyUpdateRequired;
-  }
-
-  /**
-   * @param isFixedKeyUpdateRequired the isFixedKeyUpdateRequired to set
-   */
-  public void setFixedKeyUpdateRequired(boolean isFixedKeyUpdateRequired) {
-    this.isFixedKeyUpdateRequired = isFixedKeyUpdateRequired;
-  }
-
-  /**
    * @return the aggregatorInfos
    */
   public MeasureInfo getMeasureInfo() {
@@ -301,59 +260,60 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the totalNumberDimensionBlock
+   * @return the totalNumberDimensionToRead
    */
-  public int getTotalNumberDimensionBlock() {
-    return totalNumberDimensionBlock;
+  public int getTotalNumberDimensionToRead() {
+    return totalNumberDimensionToRead;
   }
 
   /**
-   * @param totalNumberDimensionBlock the totalNumberDimensionBlock to set
+   * @param totalNumberDimensionToRead the totalNumberDimensionToRead to set
    */
-  public void setTotalNumberDimensionBlock(int totalNumberDimensionBlock) {
-    this.totalNumberDimensionBlock = totalNumberDimensionBlock;
+  public void setTotalNumberDimensionToRead(int totalNumberDimensionToRead) {
+    this.totalNumberDimensionToRead = totalNumberDimensionToRead;
   }
 
   /**
-   * @return the totalNumberOfMeasureBlock
+   * @return the totalNumberOfMeasureToRead
    */
-  public int getTotalNumberOfMeasureBlock() {
-    return totalNumberOfMeasureBlock;
+  public int getTotalNumberOfMeasureToRead() {
+    return totalNumberOfMeasureToRead;
   }
 
   /**
-   * @param totalNumberOfMeasureBlock the totalNumberOfMeasureBlock to set
+   * @param totalNumberOfMeasureToRead the totalNumberOfMeasureToRead to set
    */
-  public void setTotalNumberOfMeasureBlock(int totalNumberOfMeasureBlock) {
-    this.totalNumberOfMeasureBlock = totalNumberOfMeasureBlock;
+  public void setTotalNumberOfMeasureToRead(int totalNumberOfMeasureToRead) {
+    this.totalNumberOfMeasureToRead = totalNumberOfMeasureToRead;
   }
 
   /**
-   * @return the allSelectedDimensionBlocksIndexes
+   * @return the allSelectedDimensionColumnIndexRange
    */
-  public int[][] getAllSelectedDimensionBlocksIndexes() {
-    return allSelectedDimensionBlocksIndexes;
+  public int[][] getAllSelectedDimensionColumnIndexRange() {
+    return allSelectedDimensionColumnIndexRange;
   }
 
   /**
-   * @param allSelectedDimensionBlocksIndexes the allSelectedDimensionBlocksIndexes to set
+   * @param allSelectedDimensionColumnIndexRange the allSelectedDimensionColumnIndexRange to set
    */
-  public void setAllSelectedDimensionBlocksIndexes(int[][] allSelectedDimensionBlocksIndexes) {
-    this.allSelectedDimensionBlocksIndexes = allSelectedDimensionBlocksIndexes;
+  public void setAllSelectedDimensionColumnIndexRange(int[][] allSelectedDimensionColumnIndexRange)
+  {
+    this.allSelectedDimensionColumnIndexRange = allSelectedDimensionColumnIndexRange;
   }
 
   /**
-   * @return the allSelectedMeasureBlocksIndexes
+   * @return the allSelectedMeasureIndexRange
    */
-  public int[][] getAllSelectedMeasureBlocksIndexes() {
-    return allSelectedMeasureBlocksIndexes;
+  public int[][] getAllSelectedMeasureIndexRange() {
+    return allSelectedMeasureIndexRange;
   }
 
   /**
-   * @param allSelectedMeasureBlocksIndexes the allSelectedMeasureBlocksIndexes to set
+   * @param allSelectedMeasureIndexRange the allSelectedMeasureIndexRange to set
    */
-  public void setAllSelectedMeasureBlocksIndexes(int[][] allSelectedMeasureBlocksIndexes) {
-    this.allSelectedMeasureBlocksIndexes = allSelectedMeasureBlocksIndexes;
+  public void setAllSelectedMeasureIndexRange(int[][] allSelectedMeasureIndexRange) {
+    this.allSelectedMeasureIndexRange = allSelectedMeasureIndexRange;
   }
 
   /**
@@ -413,20 +373,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the tableBlockKeyGenerator
-   */
-  public KeyGenerator getBlockKeyGenerator() {
-    return blockKeyGenerator;
-  }
-
-  /**
-   * @param tableBlockKeyGenerator the tableBlockKeyGenerator to set
-   */
-  public void setBlockKeyGenerator(KeyGenerator tableBlockKeyGenerator) {
-    this.blockKeyGenerator = tableBlockKeyGenerator;
-  }
-
-  /**
    * @return the eachColumnValueSize
    */
   public int[] getEachColumnValueSize() {
@@ -441,31 +387,31 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the dictionaryColumnBlockIndex
+   * @return the dictionaryColumnChunkIndex
    */
-  public int[] getDictionaryColumnBlockIndex() {
-    return dictionaryColumnBlockIndex;
+  public int[] getDictionaryColumnChunkIndex() {
+    return dictionaryColumnChunkIndex;
   }
 
   /**
-   * @param dictionaryColumnBlockIndex the dictionaryColumnBlockIndex to set
+   * @param dictionaryColumnChunkIndex the dictionaryColumnChunkIndex to set
    */
-  public void setDictionaryColumnBlockIndex(int[] dictionaryColumnBlockIndex) {
-    this.dictionaryColumnBlockIndex = dictionaryColumnBlockIndex;
+  public void setDictionaryColumnChunkIndex(int[] dictionaryColumnChunkIndex) {
+    this.dictionaryColumnChunkIndex = dictionaryColumnChunkIndex;
   }
 
   /**
-   * @return the noDictionaryBlockIndexes
+   * @return the noDictionaryColumnChunkIndexes
    */
-  public int[] getNoDictionaryBlockIndexes() {
-    return noDictionaryBlockIndexes;
+  public int[] getNoDictionaryColumnChunkIndexes() {
+    return noDictionaryColumnChunkIndexes;
   }
 
   /**
-   * @param noDictionaryBlockIndexes the noDictionaryBlockIndexes to set
+   * @param noDictionaryColumnChunkIndexes the noDictionaryColumnChunkIndexes to set
    */
-  public void setNoDictionaryBlockIndexes(int[] noDictionaryBlockIndexes) {
-    this.noDictionaryBlockIndexes = noDictionaryBlockIndexes;
+  public void setNoDictionaryColumnChunkIndexes(int[] noDictionaryColumnChunkIndexes) {
+    this.noDictionaryColumnChunkIndexes = noDictionaryColumnChunkIndexes;
   }
 
   /**
@@ -519,20 +465,20 @@ public class BlockExecutionInfo {
     this.complexColumnParentBlockIndexes = complexColumnParentBlockIndexes;
   }
 
-  public QueryDimension[] getQueryDimensions() {
-    return queryDimensions;
+  public ProjectionDimension[] getProjectionDimensions() {
+    return projectionDimensions;
   }
 
-  public void setQueryDimensions(QueryDimension[] queryDimensions) {
-    this.queryDimensions = queryDimensions;
+  public void setProjectionDimensions(ProjectionDimension[] projectionDimensions) {
+    this.projectionDimensions = projectionDimensions;
   }
 
-  public QueryMeasure[] getQueryMeasures() {
-    return queryMeasures;
+  public ProjectionMeasure[] getProjectionMeasures() {
+    return projectionMeasures;
   }
 
-  public void setQueryMeasures(QueryMeasure[] queryMeasures) {
-    this.queryMeasures = queryMeasures;
+  public void setProjectionMeasures(ProjectionMeasure[] projectionMeasures) {
+    this.projectionMeasures = projectionMeasures;
   }
 
   /**
@@ -579,7 +525,8 @@ public class BlockExecutionInfo {
     this.vectorBatchCollector = vectorBatchCollector;
   }
 
-  public String getBlockId() {
+  // Return file name and path, like Part0/Segment_0/part-0-0_batchno0-0-1517155583332.carbondata
+  public String getBlockIdString() {
     return blockId;
   }
 
@@ -603,19 +550,19 @@ public class BlockExecutionInfo {
     this.dimensionInfo = dimensionInfo;
   }
 
-  public QueryDimension[] getActualQueryDimensions() {
+  public ProjectionDimension[] getActualQueryDimensions() {
     return actualQueryDimensions;
   }
 
-  public void setActualQueryDimensions(QueryDimension[] actualQueryDimensions) {
+  public void setActualQueryDimensions(ProjectionDimension[] actualQueryDimensions) {
     this.actualQueryDimensions = actualQueryDimensions;
   }
 
-  public QueryMeasure[] getActualQueryMeasures() {
+  public ProjectionMeasure[] getActualQueryMeasures() {
     return actualQueryMeasures;
   }
 
-  public void setActualQueryMeasures(QueryMeasure[] actualQueryMeasures) {
+  public void setActualQueryMeasures(ProjectionMeasure[] actualQueryMeasures) {
     this.actualQueryMeasures = actualQueryMeasures;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index 8eb0a0d..00cdfe7 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -60,9 +60,8 @@ import org.apache.carbondata.core.scan.filter.GenericQueryType;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -83,7 +82,7 @@ public class QueryUtil {
    * @param keyGenerator    key generator
    * @return masked key
    */
-  public static int[] getMaskedByteRange(List<QueryDimension> queryDimensions,
+  public static int[] getMaskedByteRange(List<ProjectionDimension> queryDimensions,
       KeyGenerator keyGenerator) {
     Set<Integer> byteRangeSet = new TreeSet<Integer>();
     int[] byteRange = null;
@@ -167,7 +166,7 @@ public class QueryUtil {
    * @return max key for dimension
    * @throws KeyGenException if any problem while generating the key
    */
-  public static byte[] getMaxKeyBasedOnDimensions(List<QueryDimension> queryDimensions,
+  public static byte[] getMaxKeyBasedOnDimensions(List<ProjectionDimension> queryDimensions,
       KeyGenerator generator) throws KeyGenException {
     long[] max = new long[generator.getDimCount()];
     Arrays.fill(max, 0L);
@@ -209,46 +208,46 @@ public class QueryUtil {
    * on query dimension
    *
    * @param queryDimensions                query dimension
-   * @param dimensionOrdinalToBlockMapping mapping of dimension block in file to query dimension
+   * @param dimensionOrdinalToChunkMapping mapping of dimension block in file to query dimension
    * @return block index of file
    */
-  public static int[] getDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
-      Map<Integer, Integer> dimensionOrdinalToBlockMapping,
+  public static int[] getDimensionChunkIndexes(List<ProjectionDimension> queryDimensions,
+      Map<Integer, Integer> dimensionOrdinalToChunkMapping,
       List<CarbonDimension> customAggregationDimension, Set<CarbonDimension> filterDimensions,
       Set<Integer> allProjectionListDimensionIndexes) {
     // using set as in row group columns will point to same block
-    Set<Integer> dimensionBlockIndex = new HashSet<Integer>();
+    Set<Integer> dimensionChunkIndex = new HashSet<Integer>();
     Set<Integer> filterDimensionOrdinal = getFilterDimensionOrdinal(filterDimensions);
-    int blockIndex = 0;
+    int chunkIndex = 0;
     for (int i = 0; i < queryDimensions.size(); i++) {
       if (queryDimensions.get(i).getDimension().hasEncoding(Encoding.IMPLICIT)) {
         continue;
       }
 
       Integer dimensionOrdinal = queryDimensions.get(i).getDimension().getOrdinal();
-      allProjectionListDimensionIndexes.add(dimensionOrdinalToBlockMapping.get(dimensionOrdinal));
+      allProjectionListDimensionIndexes.add(dimensionOrdinalToChunkMapping.get(dimensionOrdinal));
       if (queryDimensions.get(i).getDimension().getNumberOfChild() > 0) {
         addChildrenBlockIndex(allProjectionListDimensionIndexes,
             queryDimensions.get(i).getDimension());
       }
 
       if (!filterDimensionOrdinal.contains(dimensionOrdinal)) {
-        blockIndex = dimensionOrdinalToBlockMapping.get(dimensionOrdinal);
-        dimensionBlockIndex.add(blockIndex);
+        chunkIndex = dimensionOrdinalToChunkMapping.get(dimensionOrdinal);
+        dimensionChunkIndex.add(chunkIndex);
         if (queryDimensions.get(i).getDimension().getNumberOfChild() > 0) {
-          addChildrenBlockIndex(dimensionBlockIndex, queryDimensions.get(i).getDimension());
+          addChildrenBlockIndex(dimensionChunkIndex, queryDimensions.get(i).getDimension());
         }
       }
     }
     for (int i = 0; i < customAggregationDimension.size(); i++) {
-      blockIndex =
-          dimensionOrdinalToBlockMapping.get(customAggregationDimension.get(i).getOrdinal());
+      chunkIndex =
+          dimensionOrdinalToChunkMapping.get(customAggregationDimension.get(i).getOrdinal());
       // not adding the children dimension as dimension aggregation
       // is not push down in case of complex dimension
-      dimensionBlockIndex.add(blockIndex);
+      dimensionChunkIndex.add(chunkIndex);
     }
     int[] dimensionIndex = ArrayUtils
-        .toPrimitive(dimensionBlockIndex.toArray(new Integer[dimensionBlockIndex.size()]));
+        .toPrimitive(dimensionChunkIndex.toArray(new Integer[dimensionChunkIndex.size()]));
     Arrays.sort(dimensionIndex);
     return dimensionIndex;
   }
@@ -278,7 +277,7 @@ public class QueryUtil {
    * @throws IOException
    */
   public static Map<String, Dictionary> getDimensionDictionaryDetail(
-      List<QueryDimension> queryDimensions, Set<CarbonDimension> filterComplexDimensions,
+      List<ProjectionDimension> queryDimensions, Set<CarbonDimension> filterComplexDimensions,
       AbsoluteTableIdentifier absoluteTableIdentifier, TableProvider tableProvider)
       throws IOException {
     // to store complex dimension and its child id unique column id list, this is required as
@@ -433,23 +432,23 @@ public class QueryUtil {
    * @param ordinalToBlockIndexMapping measure ordinal to block mapping
    * @return block indexes
    */
-  public static int[] getMeasureBlockIndexes(List<QueryMeasure> queryMeasures,
+  public static int[] getMeasureChunkIndexes(List<ProjectionMeasure> queryMeasures,
       List<CarbonMeasure> expressionMeasure, Map<Integer, Integer> ordinalToBlockIndexMapping,
       Set<CarbonMeasure> filterMeasures, List<Integer> allProjectionListMeasureIdexes) {
-    Set<Integer> measureBlockIndex = new HashSet<Integer>();
+    Set<Integer> measureChunkIndex = new HashSet<Integer>();
     Set<Integer> filterMeasureOrdinal = getFilterMeasureOrdinal(filterMeasures);
     for (int i = 0; i < queryMeasures.size(); i++) {
       Integer measureOrdinal = queryMeasures.get(i).getMeasure().getOrdinal();
       allProjectionListMeasureIdexes.add(measureOrdinal);
       if (!filterMeasureOrdinal.contains(measureOrdinal)) {
-        measureBlockIndex.add(ordinalToBlockIndexMapping.get(measureOrdinal));
+        measureChunkIndex.add(ordinalToBlockIndexMapping.get(measureOrdinal));
       }
     }
     for (int i = 0; i < expressionMeasure.size(); i++) {
-      measureBlockIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal()));
+      measureChunkIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal()));
     }
     int[] measureIndexes =
-        ArrayUtils.toPrimitive(measureBlockIndex.toArray(new Integer[measureBlockIndex.size()]));
+        ArrayUtils.toPrimitive(measureChunkIndex.toArray(new Integer[measureChunkIndex.size()]));
     Arrays.sort(measureIndexes);
     return measureIndexes;
   }
@@ -462,8 +461,8 @@ public class QueryUtil {
    * @param queryDimensions  query dimension
    * @return sort dimension indexes
    */
-  public static byte[] getSortDimensionIndexes(List<QueryDimension> sortedDimensions,
-      List<QueryDimension> queryDimensions) {
+  public static byte[] getSortDimensionIndexes(List<ProjectionDimension> sortedDimensions,
+      List<ProjectionDimension> queryDimensions) {
     byte[] sortedDims = new byte[queryDimensions.size()];
     int indexOf = 0;
     for (int i = 0; i < sortedDims.length; i++) {
@@ -485,7 +484,7 @@ public class QueryUtil {
    * @throws KeyGenException if problem while key generation
    */
   public static Map<Integer, KeyStructureInfo> getColumnGroupKeyStructureInfo(
-      List<QueryDimension> queryDimensions, SegmentProperties segmentProperties)
+      List<ProjectionDimension> queryDimensions, SegmentProperties segmentProperties)
       throws KeyGenException {
     Map<Integer, KeyStructureInfo> rowGroupToItsRSInfo = new HashMap<Integer, KeyStructureInfo>();
     // get column group id and its ordinal mapping of column group
@@ -521,7 +520,7 @@ public class QueryUtil {
       restructureInfos.setMdkeyQueryDimensionOrdinal(ArrayUtils
           .toPrimitive(mdKeyOrdinalForQuery.toArray(new Integer[mdKeyOrdinalForQuery.size()])));
       rowGroupToItsRSInfo
-          .put(segmentProperties.getDimensionOrdinalToBlockMapping().get(ordinal.get(0)),
+          .put(segmentProperties.getDimensionOrdinalToChunkMapping().get(ordinal.get(0)),
               restructureInfos);
     }
     return rowGroupToItsRSInfo;
@@ -552,16 +551,16 @@ public class QueryUtil {
    * stored in bit level
    */
   private static Map<Integer, List<Integer>> getColumnGroupAndItsOrdinalMapping(
-      List<QueryDimension> origdimensions) {
+      List<ProjectionDimension> origDimensions) {
 
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>(origdimensions.size());
-    dimensions.addAll(origdimensions);
+    List<ProjectionDimension> dimensions = new ArrayList<>(origDimensions.size());
+    dimensions.addAll(origDimensions);
     /*
      * sort based on column group id
      */
-    Collections.sort(dimensions, new Comparator<QueryDimension>() {
+    Collections.sort(dimensions, new Comparator<ProjectionDimension>() {
 
-      @Override public int compare(QueryDimension o1, QueryDimension o2) {
+      @Override public int compare(ProjectionDimension o1, ProjectionDimension o2) {
         return Integer
             .compare(o1.getDimension().columnGroupId(), o2.getDimension().columnGroupId());
       }
@@ -636,70 +635,26 @@ public class QueryUtil {
    * dictionary column. This is done for specific purpose so that in one
    * iteration we will be able to fill both type dimension block indexes
    *
-   * @param queryDimensions                  dimension present in the query
-   * @param columnOrdinalToBlockIndexMapping column ordinal to block index mapping
-   * @param dictionaryDimensionBlockIndex    list to store dictionary column block indexes
-   * @param noDictionaryDimensionBlockIndex  list to store no dictionary block indexes
+   * @param projectDimensions                  dimension present in the query
+   * @param columnOrdinalToChunkIndexMapping column ordinal to block index mapping
+   * @param dictionaryDimensionChunkIndex    list to store dictionary column block indexes
+   * @param noDictionaryDimensionChunkIndex  list to store no dictionary block indexes
    */
-  public static void fillQueryDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
-      Map<Integer, Integer> columnOrdinalToBlockIndexMapping,
-      Set<Integer> dictionaryDimensionBlockIndex, List<Integer> noDictionaryDimensionBlockIndex) {
-    for (QueryDimension queryDimension : queryDimensions) {
+  public static void fillQueryDimensionChunkIndexes(
+      List<ProjectionDimension> projectDimensions,
+      Map<Integer, Integer> columnOrdinalToChunkIndexMapping,
+      Set<Integer> dictionaryDimensionChunkIndex,
+      List<Integer> noDictionaryDimensionChunkIndex) {
+    for (ProjectionDimension queryDimension : projectDimensions) {
       if (CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)
           && queryDimension.getDimension().getNumberOfChild() == 0) {
-        dictionaryDimensionBlockIndex
-            .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal()));
+        dictionaryDimensionChunkIndex
+            .add(columnOrdinalToChunkIndexMapping.get(queryDimension.getDimension().getOrdinal()));
       } else if (
           !CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.IMPLICIT)
               && queryDimension.getDimension().getNumberOfChild() == 0) {
-        noDictionaryDimensionBlockIndex
-            .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal()));
-      }
-    }
-  }
-
-  /**
-   * Below method will be used to resolve the query model
-   * resolve will be setting the actual dimension and measure object
-   * as from driver only column name will be passes to avoid the heavy object
-   * serialization
-   *
-   * @param queryModel query model
-   */
-  public static void resolveQueryModel(QueryModel queryModel) {
-    CarbonMetadata.getInstance().addCarbonTable(queryModel.getTable());
-    // TODO need to load the table from table identifier
-    CarbonTable carbonTable = queryModel.getTable();
-    String tableName =
-        queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName();
-    // resolve query dimension
-    for (QueryDimension queryDimension : queryModel.getQueryDimension()) {
-      queryDimension
-          .setDimension(carbonTable.getDimensionByName(tableName, queryDimension.getColumnName()));
-    }
-    // resolve query measure
-    for (QueryMeasure queryMeasure : queryModel.getQueryMeasures()) {
-      // in case of count start column name will  be count * so
-      // first need to check any measure is present or not and as if measure
-      // if measure is present and if first measure is not a default
-      // measure than add measure otherwise
-      // than add first dimension as a measure
-      //as currently if measure is not present then
-      //we are adding default measure so first condition will
-      //never come false but if in future we can remove so not removing first if check
-      if (queryMeasure.getColumnName().equals("count(*)")) {
-        if (carbonTable.getMeasureByTableName(tableName).size() > 0 && !carbonTable
-            .getMeasureByTableName(tableName).get(0).getColName()
-            .equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)) {
-          queryMeasure.setMeasure(carbonTable.getMeasureByTableName(tableName).get(0));
-        } else {
-          CarbonMeasure dummyMeasure = new CarbonMeasure(
-              carbonTable.getDimensionByTableName(tableName).get(0).getColumnSchema(), 0);
-          queryMeasure.setMeasure(dummyMeasure);
-        }
-      } else {
-        queryMeasure
-            .setMeasure(carbonTable.getMeasureByName(tableName, queryMeasure.getColumnName()));
+        noDictionaryDimensionChunkIndex
+            .add(columnOrdinalToChunkIndexMapping.get(queryDimension.getDimension().getOrdinal()));
       }
     }
   }
@@ -759,11 +714,11 @@ public class QueryUtil {
    * @return complex dimension and query type
    */
   public static Map<Integer, GenericQueryType> getComplexDimensionsMap(
-      List<QueryDimension> queryDimensions, Map<Integer, Integer> dimensionToBlockIndexMap,
+      List<ProjectionDimension> queryDimensions, Map<Integer, Integer> dimensionToBlockIndexMap,
       int[] eachComplexColumnValueSize, Map<String, Dictionary> columnIdToDictionaryMap,
       Set<CarbonDimension> filterDimensions) {
     Map<Integer, GenericQueryType> complexTypeMap = new HashMap<Integer, GenericQueryType>();
-    for (QueryDimension dimension : queryDimensions) {
+    for (ProjectionDimension dimension : queryDimensions) {
       CarbonDimension actualDimension = dimension.getDimension();
       if (actualDimension.getNumberOfChild() == 0) {
         continue;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
index a029986..2712cbc 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
@@ -34,8 +34,8 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.infos.DimensionInfo;
 import org.apache.carbondata.core.scan.executor.infos.MeasureInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
@@ -61,12 +61,12 @@ public class RestructureUtil {
    * @param tableComplexDimension
    * @return list of query dimension which is present in the table block
    */
-  public static List<QueryDimension> createDimensionInfoAndGetCurrentBlockQueryDimension(
-      BlockExecutionInfo blockExecutionInfo, List<QueryDimension> queryDimensions,
+  public static List<ProjectionDimension> createDimensionInfoAndGetCurrentBlockQueryDimension(
+      BlockExecutionInfo blockExecutionInfo, List<ProjectionDimension> queryDimensions,
       List<CarbonDimension> tableBlockDimensions, List<CarbonDimension> tableComplexDimension,
       int measureCount) {
-    List<QueryDimension> presentDimension =
-        new ArrayList<QueryDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    List<ProjectionDimension> presentDimension =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     boolean[] isDimensionExists = new boolean[queryDimensions.size()];
     Object[] defaultValues = new Object[queryDimensions.size()];
     // create dimension information instance
@@ -76,7 +76,7 @@ public class RestructureUtil {
     int newNoDictionaryColumnCount = 0;
     // selecting only those dimension which is present in the query
     int dimIndex = 0;
-    for (QueryDimension queryDimension : queryDimensions) {
+    for (ProjectionDimension queryDimension : queryDimensions) {
       if (queryDimension.getDimension().hasEncoding(Encoding.IMPLICIT)) {
         presentDimension.add(queryDimension);
         isDimensionExists[dimIndex] = true;
@@ -85,15 +85,14 @@ public class RestructureUtil {
       } else {
         for (CarbonDimension tableDimension : tableBlockDimensions) {
           if (tableDimension.getColumnId().equals(queryDimension.getDimension().getColumnId())) {
-            QueryDimension currentBlockDimension = new QueryDimension(tableDimension.getColName());
+            ProjectionDimension currentBlockDimension = new ProjectionDimension(tableDimension);
             tableDimension.getColumnSchema()
                 .setPrecision(queryDimension.getDimension().getColumnSchema().getPrecision());
             tableDimension.getColumnSchema()
                 .setScale(queryDimension.getDimension().getColumnSchema().getScale());
             tableDimension.getColumnSchema()
                 .setDefaultValue(queryDimension.getDimension().getDefaultValue());
-            currentBlockDimension.setDimension(tableDimension);
-            currentBlockDimension.setQueryOrder(queryDimension.getQueryOrder());
+            currentBlockDimension.setOrdinal(queryDimension.getOrdinal());
             presentDimension.add(currentBlockDimension);
             isDimensionExists[dimIndex] = true;
             dimensionInfo.dataType[currentBlockDimension.getQueryOrder()] =
@@ -108,11 +107,10 @@ public class RestructureUtil {
         }
         for (CarbonDimension tableDimension : tableComplexDimension) {
           if (tableDimension.getColumnId().equals(queryDimension.getDimension().getColumnId())) {
-            QueryDimension currentBlockDimension = new QueryDimension(tableDimension.getColName());
+            ProjectionDimension currentBlockDimension = new ProjectionDimension(tableDimension);
             // TODO: for complex dimension set scale and precision by traversing
             // the child dimensions
-            currentBlockDimension.setDimension(tableDimension);
-            currentBlockDimension.setQueryOrder(queryDimension.getQueryOrder());
+            currentBlockDimension.setOrdinal(queryDimension.getOrdinal());
             presentDimension.add(currentBlockDimension);
             isDimensionExists[dimIndex] = true;
             dimensionInfo.dataType[currentBlockDimension.getQueryOrder()] =
@@ -250,39 +248,6 @@ public class RestructureUtil {
   }
 
   /**
-   * Below method is to add dimension children for complex type dimension as
-   * internally we are creating dimension column for each each complex
-   * dimension so when complex query dimension request will come in the query,
-   * we need to add its children as it is hidden from the user For example if
-   * complex dimension is of Array of String[2] so we are storing 3 dimension
-   * and when user will query for complex type i.e. array type we need to add
-   * its children and then we will read respective block and create a tuple
-   * based on all three dimension
-   *
-   * @param queryDimensions      current query dimensions
-   * @param tableBlockDimensions dimensions which is present in the table block
-   * @return updated dimension(after adding complex type children)
-   */
-  public static List<CarbonDimension> addChildrenForComplexTypeDimension(
-      List<CarbonDimension> queryDimensions, List<CarbonDimension> tableBlockDimensions) {
-    List<CarbonDimension> updatedQueryDimension = new ArrayList<CarbonDimension>();
-    int numberOfChildren = 0;
-    for (CarbonDimension queryDimension : queryDimensions) {
-      // if number of child is zero, then it is not a complex dimension
-      // so directly add it query dimension
-      if (queryDimension.getNumberOfChild() == 0) {
-        updatedQueryDimension.add(queryDimension);
-      }
-      // if number of child is more than 1 then add all its children
-      numberOfChildren = queryDimension.getOrdinal() + queryDimension.getNumberOfChild();
-      for (int j = queryDimension.getOrdinal(); j < numberOfChildren; j++) {
-        updatedQueryDimension.add(tableBlockDimensions.get(j));
-      }
-    }
-    return updatedQueryDimension;
-  }
-
-  /**
    * Method for computing measure default value based on the data type
    *
    * @param columnSchema
@@ -375,30 +340,29 @@ public class RestructureUtil {
    * @param currentBlockMeasures current block measures
    * @return measures present in the block
    */
-  public static List<QueryMeasure> createMeasureInfoAndGetCurrentBlockQueryMeasures(
-      BlockExecutionInfo blockExecutionInfo, List<QueryMeasure> queryMeasures,
+  public static List<ProjectionMeasure> createMeasureInfoAndGetCurrentBlockQueryMeasures(
+      BlockExecutionInfo blockExecutionInfo, List<ProjectionMeasure> queryMeasures,
       List<CarbonMeasure> currentBlockMeasures) {
     MeasureInfo measureInfo = new MeasureInfo();
-    List<QueryMeasure> presentMeasure = new ArrayList<>(queryMeasures.size());
+    List<ProjectionMeasure> presentMeasure = new ArrayList<>(queryMeasures.size());
     int numberOfMeasureInQuery = queryMeasures.size();
     List<Integer> measureOrdinalList = new ArrayList<>(numberOfMeasureInQuery);
     Object[] defaultValues = new Object[numberOfMeasureInQuery];
     boolean[] measureExistsInCurrentBlock = new boolean[numberOfMeasureInQuery];
     int index = 0;
-    for (QueryMeasure queryMeasure : queryMeasures) {
+    for (ProjectionMeasure queryMeasure : queryMeasures) {
       // if query measure exists in current dimension measures
       // then setting measure exists is true
       // otherwise adding a default value of a measure
       for (CarbonMeasure carbonMeasure : currentBlockMeasures) {
         if (carbonMeasure.getColumnId().equals(queryMeasure.getMeasure().getColumnId())) {
-          QueryMeasure currentBlockMeasure = new QueryMeasure(carbonMeasure.getColName());
+          ProjectionMeasure currentBlockMeasure = new ProjectionMeasure(carbonMeasure);
           carbonMeasure.getColumnSchema().setDataType(queryMeasure.getMeasure().getDataType());
           carbonMeasure.getColumnSchema().setPrecision(queryMeasure.getMeasure().getPrecision());
           carbonMeasure.getColumnSchema().setScale(queryMeasure.getMeasure().getScale());
           carbonMeasure.getColumnSchema()
               .setDefaultValue(queryMeasure.getMeasure().getDefaultValue());
-          currentBlockMeasure.setMeasure(carbonMeasure);
-          currentBlockMeasure.setQueryOrder(queryMeasure.getQueryOrder());
+          currentBlockMeasure.setOrdinal(queryMeasure.getOrdinal());
           presentMeasure.add(currentBlockMeasure);
           measureOrdinalList.add(carbonMeasure.getOrdinal());
           measureExistsInCurrentBlock[index] = true;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
index 981efb5..5cd2d34 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
@@ -70,10 +70,6 @@ public class ColumnExpression extends LeafExpression {
     return columnName;
   }
 
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
-
   public boolean isDimension() {
     return isDimension;
   }
@@ -102,10 +98,6 @@ public class ColumnExpression extends LeafExpression {
     return dataType;
   }
 
-  public void setDataType(DataType dataType) {
-    this.dataType = dataType;
-  }
-
   @Override public ExpressionResult evaluate(RowIntf value) {
     return new ExpressionResult(dataType, (null == value ? null : value.getVal(colIndex)));
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
index dad3975..8b060e1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
@@ -17,62 +17,29 @@
 
 package org.apache.carbondata.core.scan.expression;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
 
 public class FilterModificationNode {
   private Expression currentExp;
   private Expression parentExp;
   private ExpressionType expType;
-  private ExpressionResult literalValue;
-  private String columnName;
 
-  public FilterModificationNode(Expression currentNode, Expression parentNode,
-      ExpressionType expType, DataType dataType, Object literalVal, String colName) {
+  FilterModificationNode(Expression currentNode, Expression parentNode, ExpressionType expType) {
     this.currentExp = currentNode;
     this.parentExp = parentNode;
     this.expType = expType;
-    this.columnName = colName;
-    this.literalValue = new ExpressionResult(dataType, literalVal);
   }
 
   public Expression getCurrentExp() {
     return currentExp;
   }
 
-  public void setCurrentExp(Expression currentExp) {
-    this.currentExp = currentExp;
-  }
-
   public Expression getParentExp() {
     return parentExp;
   }
 
-  public void setParentExp(Expression parentExp) {
-    this.parentExp = parentExp;
-  }
-
   public ExpressionType getExpType() {
     return expType;
   }
 
-  public void setExpType(ExpressionType expType) {
-    this.expType = expType;
-  }
-
-  public ExpressionResult getLiteralValue() {
-    return literalValue;
-  }
-
-  public void setLiteralValue(ExpressionResult literalValue) {
-    this.literalValue = literalValue;
-  }
-
-  public String getColumnName() {
-    return columnName;
-  }
-
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
index e7cc1c3..585bf60 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
@@ -48,7 +48,6 @@ public class RangeExpressionEvaluator {
       LogServiceFactory.getLogService(RangeExpressionEvaluator.class.getName());
   private Expression expr;
   private Expression srcNode;
-  private Expression srcParentNode;
   private Expression tarNode;
   private Expression tarParentNode;
 
@@ -64,15 +63,15 @@ public class RangeExpressionEvaluator {
     this.expr = expr;
   }
 
-  public Expression getSrcNode() {
+  private Expression getSrcNode() {
     return srcNode;
   }
 
-  public void setTarNode(Expression expr) {
+  private void setTarNode(Expression expr) {
     this.tarNode = expr;
   }
 
-  public void setTarParentNode(Expression expr) {
+  private void setTarParentNode(Expression expr) {
     this.tarParentNode = expr;
   }
 
@@ -237,12 +236,10 @@ public class RangeExpressionEvaluator {
   private void addFilterExpressionMap(Map<String, List<FilterModificationNode>> filterExpressionMap,
       Expression currentNode, Expression parentNode) {
     String colName = getColumnName(currentNode);
-    DataType dataType = getLiteralDataType(currentNode);
-    Object literalVal = getLiteralValue(currentNode);
     ExpressionType expType = getExpressionType(currentNode);
 
     FilterModificationNode filterExpression =
-        new FilterModificationNode(currentNode, parentNode, expType, dataType, literalVal, colName);
+        new FilterModificationNode(currentNode, parentNode, expType);
 
     if (null == filterExpressionMap.get(colName)) {
       filterExpressionMap.put(colName, new ArrayList<FilterModificationNode>());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index 3e23aa3..b882b51 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -105,8 +105,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
    *
    */
   public List<DataRefNode> getFilterredBlocks(DataRefNode btreeNode,
-      FilterResolverIntf filterResolver, AbstractIndex tableSegment,
-      AbsoluteTableIdentifier tableIdentifier) {
+      FilterResolverIntf filterResolver, AbstractIndex tableSegment) {
     // Need to get the current dimension tables
     List<DataRefNode> listOfDataBlocksToScan = new ArrayList<DataRefNode>();
     // getting the start and end index key based on filter for hitting the
@@ -401,7 +400,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
       case FALSE:
         return new FalseConditionalResolverImpl(expression, false, false, tableIdentifier);
       case TRUE:
-        return new TrueConditionalResolverImpl(expression, false, false, tableIdentifier);
+        return new TrueConditionalResolverImpl(expression, false, false);
       case EQUALS:
         currentCondExpression = (BinaryConditionalExpression) expression;
         // check for implicit column in the expression
@@ -410,7 +409,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
               currentCondExpression.getColumnList().get(0).getCarbonColumn();
           if (carbonColumn.hasEncoding(Encoding.IMPLICIT)) {
             return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-                tableIdentifier,
                 currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
           }
         }
@@ -436,7 +434,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
                   tableIdentifier);
             }
             return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-                tableIdentifier,
                 currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
           }
           // getting new dim index.
@@ -462,14 +459,12 @@ public class FilterExpressionProcessor implements FilterProcessor {
             }
           }
           return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-              tableIdentifier,
               currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
 
         }
         break;
       case RANGE:
-        return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-            tableIdentifier, false);
+        return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true, false);
       case NOT_EQUALS:
         currentCondExpression = (BinaryConditionalExpression) expression;
         column = currentCondExpression.getColumnList().get(0).getCarbonColumn();
@@ -492,8 +487,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
               return new RowLevelRangeFilterResolverImpl(expression, isExpressionResolve, false,
                   tableIdentifier);
             }
-            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false,
-                tableIdentifier, true);
+            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, true);
           }
 
           if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
@@ -516,11 +510,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
                   tableIdentifier);
             }
 
-            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false,
-                tableIdentifier, false);
+            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, false);
           }
-          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false,
-              tableIdentifier, false);
+          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, false);
         }
         break;
 
@@ -534,7 +526,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
                 .hasEncoding(Encoding.DICTIONARY) && !condExpression.getColumnList().get(0)
                 .getCarbonColumn().hasEncoding(Encoding.DIRECT_DICTIONARY))
                 || (condExpression.getColumnList().get(0).getCarbonColumn().isMeasure())) {
-              return new ConditionalFilterResolverImpl(expression, true, true, tableIdentifier,
+              return new ConditionalFilterResolverImpl(expression, true, true,
                   condExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
             }
           }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
index dc392ba..dbb6eb7 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
@@ -53,7 +53,7 @@ public interface FilterProcessor {
    * @return list of DataRefNode.
    */
   List<DataRefNode> getFilterredBlocks(DataRefNode dataRefNode, FilterResolverIntf filterResolver,
-      AbstractIndex segmentIndexBuilder, AbsoluteTableIdentifier tableIdentifier);
+      AbstractIndex segmentIndexBuilder);
 
   /**
    * This API will get the map of required partitions.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index 8dcac30..0b72e77 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -49,7 +49,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -172,10 +172,8 @@ public final class FilterUtil {
                   segmentProperties);
         case RANGE:
           return new RangeValueFilterExecuterImpl(
-              ((ConditionalFilterResolverImpl) filterExpressionResolverTree)
-                  .getDimColResolvedFilterInfo(),
-              null, filterExpressionResolverTree.getFilterExpression(),
-              ((ConditionalFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+              filterExpressionResolverTree.getDimColResolvedFilterInfo(),
+              filterExpressionResolverTree.getFilterExpression(),
               ((ConditionalFilterResolverImpl) filterExpressionResolverTree)
                   .getFilterRangeValues(segmentProperties), segmentProperties);
         case TRUE:
@@ -551,10 +549,9 @@ public final class FilterUtil {
       sortFilterModelMembers(columnExpression, evaluateResultList);
       getDictionaryValue(evaluateResultList, forwardDictionary, surrogates);
       filterInfo =
-          getFilterValues(columnExpression, forwardDictionary, isIncludeFilter, null, surrogates);
+          getFilterValues(forwardDictionary, isIncludeFilter, surrogates);
       if (filterInfo.isOptimized()) {
-        return getDimColumnFilterInfoAfterApplyingCBO(columnExpression,
-            forwardDictionary, filterInfo);
+        return getDimColumnFilterInfoAfterApplyingCBO(forwardDictionary, filterInfo);
       }
     } finally {
       CarbonUtil.clearDictionaryCache(forwardDictionary);
@@ -566,22 +563,20 @@ public final class FilterUtil {
    * Method will prepare the  dimfilterinfo instance by resolving the filter
    * expression value to its respective surrogates.
    *
-   * @param columnExpression
    * @param forwardDictionary
    * @param isIncludeFilter
    * @param filterInfo
    * @param surrogates
    * @return
    */
-  private static ColumnFilterInfo getFilterValues(ColumnExpression columnExpression,
-      Dictionary forwardDictionary, boolean isIncludeFilter, ColumnFilterInfo filterInfo,
-      List<Integer> surrogates) throws QueryExecutionException {
+  private static ColumnFilterInfo getFilterValues(Dictionary forwardDictionary,
+      boolean isIncludeFilter, List<Integer> surrogates) {
     // Default value has to be added
     if (surrogates.isEmpty()) {
       surrogates.add(0);
     }
     boolean isExcludeFilterNeedsToApply = false;
-    if (null == filterInfo && isIncludeFilter) {
+    if (isIncludeFilter) {
       isExcludeFilterNeedsToApply =
           isExcludeFilterNeedsToApply(forwardDictionary, surrogates.size());
     }
@@ -593,17 +588,10 @@ public final class FilterUtil {
         columnFilterInfo.setOptimized(true);
       }
       columnFilterInfo.setIncludeFilter(isIncludeFilter);
-      if (null != filterInfo) {
-        filterInfo.setIncludeFilter(isIncludeFilter);
-        filterInfo.setOptimized(true);
-        filterInfo.setExcludeFilterList(surrogates);
-        return filterInfo;
+      if (!isIncludeFilter) {
+        columnFilterInfo.setExcludeFilterList(surrogates);
       } else {
-        if (!isIncludeFilter) {
-          columnFilterInfo.setExcludeFilterList(surrogates);
-        } else {
-          columnFilterInfo.setFilterList(surrogates);
-        }
+        columnFilterInfo.setFilterList(surrogates);
       }
     }
     return columnFilterInfo;
@@ -619,8 +607,7 @@ public final class FilterUtil {
   }
 
   private static ColumnFilterInfo getDimColumnFilterInfoAfterApplyingCBO(
-      ColumnExpression columnExpression, Dictionary forwardDictionary,
-      ColumnFilterInfo filterInfo) throws FilterUnsupportedException, QueryExecutionException {
+      Dictionary forwardDictionary, ColumnFilterInfo filterInfo) throws FilterUnsupportedException {
     List<Integer> excludeMemberSurrogates =
         prepareExcludeFilterMembers(forwardDictionary, filterInfo.getFilterList());
     filterInfo.setExcludeFilterList(excludeMemberSurrogates);
@@ -730,14 +717,12 @@ public final class FilterUtil {
       prepareIncludeFilterMembers(expression, columnExpression, isIncludeFilter, forwardDictionary,
           surrogates);
       ColumnFilterInfo filterInfo =
-          getFilterValues(columnExpression, forwardDictionary, isIncludeFilter, null, surrogates);
+          getFilterValues(forwardDictionary, isIncludeFilter, surrogates);
       if (filterInfo.isOptimized()) {
-        return getDimColumnFilterInfoAfterApplyingCBO(columnExpression, forwardDictionary,
+        return getDimColumnFilterInfoAfterApplyingCBO(forwardDictionary,
             filterInfo);
       }
       return filterInfo;
-    } catch (QueryExecutionException e) {
-      throw new FilterUnsupportedException(e.getMessage());
     } finally {
       CarbonUtil.clearDictionaryCache(forwardDictionary);
     }
@@ -761,21 +746,16 @@ public final class FilterUtil {
    * expression value to its respective surrogates in the scenario of restructure.
    *
    * @param expression
-   * @param columnExpression
    * @param defaultValues
    * @param defaultSurrogate
    * @return
    * @throws FilterUnsupportedException
    */
-  public static ColumnFilterInfo getFilterListForRS(Expression expression,
-      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate)
-      throws FilterUnsupportedException {
+  public static ColumnFilterInfo getFilterListForRS(Expression expression, String defaultValues,
+      int defaultSurrogate) throws FilterUnsupportedException {
     List<Integer> filterValuesList = new ArrayList<Integer>(20);
     ColumnFilterInfo columnFilterInfo = null;
-    // List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
     List<String> evaluateResultListFinal = new ArrayList<String>(20);
-    // KeyGenerator keyGenerator =
-    // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate });
     try {
       List<ExpressionResult> evaluateResultList = expression.evaluate(null).getList();
       for (ExpressionResult result : evaluateResultList) {
@@ -821,8 +801,6 @@ public final class FilterUtil {
     List<String> evaluateResultListFinal = new ArrayList<String>(20);
     ColumnFilterInfo columnFilterInfo = null;
 
-    // KeyGenerator keyGenerator =
-    // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate });
     try {
       RowIntf row = new RowImpl();
       if (defaultValues.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
@@ -1775,14 +1753,14 @@ public final class FilterUtil {
    * This method will compare the selected data against null values and
    * flip the bitSet if any null value is found
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param bitSet
    */
-  public static void removeNullValues(DimensionColumnDataChunk dimensionColumnDataChunk,
+  public static void removeNullValues(DimensionColumnPage dimensionColumnPage,
       BitSet bitSet, byte[] defaultValue) {
     if (!bitSet.isEmpty()) {
       for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-        if (dimensionColumnDataChunk.compareTo(i, defaultValue) == 0) {
+        if (dimensionColumnPage.compareTo(i, defaultValue) == 0) {
           bitSet.flip(i);
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
index 3742e7e..214bd9d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 
 import org.apache.spark.sql.types.DataType;
 
@@ -45,7 +45,7 @@ public interface GenericQueryType {
 
   DataType getSchemaType();
 
-  void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder) throws IOException;
+  void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder) throws IOException;
 
   Object getDataBasedOnDataTypeFromSurrogates(ByteBuffer surrogateData);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
index f882162..e309451 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
@@ -21,7 +21,7 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilterExecutor {
@@ -35,18 +35,18 @@ public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilt
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    BitSetGroup leftFilters = leftExecuter.applyFilter(blockChunkHolder, useBitsetPipeLine);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    BitSetGroup leftFilters = leftExecuter.applyFilter(rawBlockletColumnChunks, useBitsetPipeLine);
     if (leftFilters.isEmpty()) {
       return leftFilters;
     }
-    BitSetGroup rightFilter = rightExecuter.applyFilter(blockChunkHolder, useBitsetPipeLine);
+    BitSetGroup rightFilter = rightExecuter.applyFilter(rawBlockletColumnChunks, useBitsetPipeLine);
     if (rightFilter.isEmpty()) {
       return rightFilter;
     }
     leftFilters.and(rightFilter);
-    blockChunkHolder.setBitSetGroup(leftFilters);
+    rawBlockletColumnChunks.setBitSetGroup(leftFilters);
     return leftFilters;
   }
 
@@ -69,9 +69,10 @@ public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilt
     return leftFilters;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    leftExecuter.readBlocks(blocksChunkHolder);
-    rightExecuter.readBlocks(blocksChunkHolder);
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    leftExecuter.readColumnChunks(rawBlockletColumnChunks);
+    rightExecuter.readColumnChunks(rawBlockletColumnChunks);
   }
 
   @Override
@@ -93,8 +94,7 @@ public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilt
       rightFilter = ((ImplicitColumnFilterExecutor) rightExecuter)
           .isFilterValuesPresentInBlockOrBlocklet(maxValue, minValue, uniqueBlockPath);
     } else {
-      rightFilter = rightExecuter
-          .isScanRequired(maxValue, minValue);
+      rightFilter = rightExecuter.isScanRequired(maxValue, minValue);
     }
     if (rightFilter.isEmpty()) {
       return rightFilter;