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/01 02:27:51 UTC

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

Repository: carbondata
Updated Branches:
  refs/heads/carbonstore 71c2d8ca4 -> 15b4e192e


http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index 5d927df..73da878 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -35,8 +35,8 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-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.scan.result.iterator.AbstractDetailQueryResultIterator;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
@@ -100,7 +100,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
   /**
    * Implementation of RecordReader API.
    */
-  @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+  @Override
+  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException, UnsupportedOperationException {
     // The input split can contain single HDFS block or multiple blocks, so firstly get all the
     // blocks and then set them in the query model.
@@ -145,7 +146,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     }
   }
 
-  @Override public void close() throws IOException {
+  @Override
+  public void close() throws IOException {
     logStatistics(rowCount, queryModel.getStatisticsRecorder());
     if (columnarBatch != null) {
       columnarBatch.close();
@@ -165,10 +167,13 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     }
   }
 
-  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
     resultBatch();
 
-    if (returnColumnarBatch) return nextBatch();
+    if (returnColumnarBatch) {
+      return nextBatch();
+    }
 
     if (batchIdx >= numBatched) {
       if (!nextBatch()) return false;
@@ -177,7 +182,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     return true;
   }
 
-  @Override public Object getCurrentValue() throws IOException, InterruptedException {
+  @Override
+  public Object getCurrentValue() throws IOException, InterruptedException {
     if (returnColumnarBatch) {
       int value = columnarBatch.numValidRows();
       rowCount += value;
@@ -190,11 +196,13 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     return columnarBatch.getRow(batchIdx - 1);
   }
 
-  @Override public Void getCurrentKey() throws IOException, InterruptedException {
+  @Override
+  public Void getCurrentKey() throws IOException, InterruptedException {
     return null;
   }
 
-  @Override public float getProgress() throws IOException, InterruptedException {
+  @Override
+  public float getProgress() throws IOException, InterruptedException {
     // TODO : Implement it based on total number of rows it is going to retrive.
     return 0;
   }
@@ -206,44 +214,44 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
    */
 
   private void initBatch(MemoryMode memMode) {
-    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
-    List<QueryMeasure> queryMeasures = queryModel.getQueryMeasures();
+    List<ProjectionDimension> queryDimension = queryModel.getProjectionDimensions();
+    List<ProjectionMeasure> queryMeasures = queryModel.getProjectionMeasures();
     StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()];
     for (int i = 0; i < queryDimension.size(); i++) {
-      QueryDimension dim = queryDimension.get(i);
+      ProjectionDimension dim = queryDimension.get(i);
       if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
             .getDirectDictionaryGenerator(dim.getDimension().getDataType());
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(generator.getReturnType()), true, null);
       } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(dim.getDimension().getDataType()), true,
             null);
       } else if (dim.getDimension().isComplex()) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(dim.getDimension().getDataType()), true,
             null);
       } else {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.INT), true, null);
       }
     }
 
     for (int i = 0; i < queryMeasures.size(); i++) {
-      QueryMeasure msr = queryMeasures.get(i);
+      ProjectionMeasure msr = queryMeasures.get(i);
       DataType dataType = msr.getMeasure().getDataType();
       if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT ||
           dataType == DataTypes.INT || dataType == DataTypes.LONG) {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
             null);
       } else if (DataTypes.isDecimal(dataType)) {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(),
             new DecimalType(msr.getMeasure().getPrecision(), msr.getMeasure().getScale()), true,
             null);
       } else {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.DOUBLE), true, null);
       }
     }
@@ -261,9 +269,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     initBatch(DEFAULT_MEMORY_MODE);
   }
 
-  private ColumnarBatch resultBatch() {
+  private void resultBatch() {
     if (columnarBatch == null) initBatch();
-    return columnarBatch;
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
index f51ced3..6a401d8 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
@@ -34,20 +34,16 @@ import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-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.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
 
 /**
  * Executor class for executing the query on the selected segments to be merged.
@@ -70,6 +66,9 @@ public class CarbonCompactionExecutor {
    */
   private boolean restructuredBlockExists;
 
+  // converter for UTF8String and decimal conversion
+  private DataTypeConverter dataTypeConverter;
+
   /**
    * Constructor
    *
@@ -82,13 +81,14 @@ public class CarbonCompactionExecutor {
   public CarbonCompactionExecutor(Map<String, TaskBlockInfo> segmentMapping,
       SegmentProperties segmentProperties, CarbonTable carbonTable,
       Map<String, List<DataFileFooter>> dataFileMetadataSegMapping,
-      boolean restructuredBlockExists) {
+      boolean restructuredBlockExists, DataTypeConverter dataTypeConverter) {
     this.segmentMapping = segmentMapping;
     this.destinationSegProperties = segmentProperties;
     this.carbonTable = carbonTable;
     this.dataFileMetadataSegMapping = dataFileMetadataSegMapping;
     this.restructuredBlockExists = restructuredBlockExists;
-    queryExecutorList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.queryExecutorList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.dataTypeConverter = dataTypeConverter;
   }
 
   /**
@@ -100,7 +100,9 @@ public class CarbonCompactionExecutor {
     List<RawResultIterator> resultList =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     List<TableBlockInfo> list = null;
-    queryModel = prepareQueryModel(list);
+    queryModel = carbonTable.createQueryModelWithProjectAllColumns(dataTypeConverter);
+    queryModel.setReadPageByPage(enablePageLevelReaderForCompaction());
+    queryModel.setForcedDetailRawQuery(true);
     // iterate each seg ID
     for (Map.Entry<String, TaskBlockInfo> taskMap : segmentMapping.entrySet()) {
       String segmentId = taskMap.getKey();
@@ -156,7 +158,7 @@ public class CarbonCompactionExecutor {
    * @param blockList
    * @return
    */
-  private CarbonIterator<BatchResult> executeBlockList(List<TableBlockInfo> blockList)
+  private CarbonIterator<RowBatch> executeBlockList(List<TableBlockInfo> blockList)
       throws QueryExecutionException, IOException {
     queryModel.setTableBlockInfos(blockList);
     QueryExecutor queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
@@ -195,48 +197,6 @@ public class CarbonCompactionExecutor {
   }
 
   /**
-   * Preparing of the query model.
-   *
-   * @param blockList
-   * @return
-   */
-  private QueryModel prepareQueryModel(List<TableBlockInfo> blockList) {
-    QueryModel model = new QueryModel();
-    model.setTableBlockInfos(blockList);
-    model.setForcedDetailRawQuery(true);
-    model.setFilterExpressionResolverTree(null);
-    model.setConverter(DataTypeUtil.getDataTypeConverter());
-    model.setReadPageByPage(enablePageLevelReaderForCompaction());
-
-    List<QueryDimension> dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    List<CarbonDimension> dimensions =
-        carbonTable.getDimensionByTableName(carbonTable.getTableName());
-    for (CarbonDimension dim : dimensions) {
-      // check if dimension is deleted
-      QueryDimension queryDimension = new QueryDimension(dim.getColName());
-      queryDimension.setDimension(dim);
-      dims.add(queryDimension);
-    }
-    model.setQueryDimension(dims);
-
-    List<QueryMeasure> msrs = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<CarbonMeasure> measures =
-        carbonTable.getMeasureByTableName(carbonTable.getTableName());
-    for (CarbonMeasure carbonMeasure : measures) {
-      // check if measure is deleted
-      QueryMeasure queryMeasure = new QueryMeasure(carbonMeasure.getColName());
-      queryMeasure.setMeasure(carbonMeasure);
-      msrs.add(queryMeasure);
-    }
-    model.setQueryMeasures(msrs);
-    model.setQueryId(System.nanoTime() + "");
-    model.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-    model.setTable(carbonTable);
-    return model;
-  }
-
-  /**
    * Whether to enable page level reader for compaction or not.
    */
   private boolean enablePageLevelReaderForCompaction() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
index 79e9e5a..b6f12a5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
@@ -23,7 +23,6 @@ import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
 import org.apache.carbondata.processing.partition.DataPartitioner;
 import org.apache.carbondata.processing.partition.Partition;
 
@@ -46,9 +45,8 @@ public final class QueryPartitionHelper {
   /**
    * Get partitions applicable for query based on filters applied in query
    */
-  public List<Partition> getPartitionsForQuery(CarbonQueryPlan queryPlan) {
-    String tableUniqueName =
-        CarbonTable.buildUniqueName(queryPlan.getDatabaseName(), queryPlan.getTableName());
+  public List<Partition> getPartitionsForQuery(String databaseName, String tableName) {
+    String tableUniqueName = CarbonTable.buildUniqueName(databaseName, tableName);
 
     DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
index 36e022b..01db4f6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
@@ -18,7 +18,6 @@
 package org.apache.carbondata.processing.partition.spliter;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -26,19 +25,14 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-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.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 public abstract class AbstractCarbonQueryExecutor {
@@ -47,8 +41,8 @@ public abstract class AbstractCarbonQueryExecutor {
       LogServiceFactory.getLogService(AbstractCarbonQueryExecutor.class.getName());
   protected CarbonTable carbonTable;
   protected QueryModel queryModel;
-  protected QueryExecutor queryExecutor;
-  protected Map<String, TaskBlockInfo> segmentMapping;
+  private QueryExecutor queryExecutor;
+  Map<String, TaskBlockInfo> segmentMapping;
 
   /**
    * get executor and execute the query model.
@@ -56,7 +50,7 @@ public abstract class AbstractCarbonQueryExecutor {
    * @param blockList
    * @return
    */
-  protected CarbonIterator<BatchResult> executeBlockList(List<TableBlockInfo> blockList)
+  CarbonIterator<RowBatch> executeBlockList(List<TableBlockInfo> blockList)
       throws QueryExecutionException, IOException {
     queryModel.setTableBlockInfos(blockList);
     this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
@@ -64,46 +58,6 @@ public abstract class AbstractCarbonQueryExecutor {
   }
 
   /**
-   * Preparing of the query model.
-   *
-   * @param blockList
-   * @return
-   */
-  protected QueryModel prepareQueryModel(List<TableBlockInfo> blockList) {
-    QueryModel model = new QueryModel();
-    model.setTableBlockInfos(blockList);
-    model.setForcedDetailRawQuery(true);
-    model.setFilterExpressionResolverTree(null);
-
-    List<QueryDimension> dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    List<CarbonDimension> dimensions =
-        carbonTable.getDimensionByTableName(carbonTable.getTableName());
-    for (CarbonDimension dim : dimensions) {
-      // check if dimension is deleted
-      QueryDimension queryDimension = new QueryDimension(dim.getColName());
-      queryDimension.setDimension(dim);
-      dims.add(queryDimension);
-    }
-    model.setQueryDimension(dims);
-
-    List<QueryMeasure> msrs = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<CarbonMeasure> measures =
-        carbonTable.getMeasureByTableName(carbonTable.getTableName());
-    for (CarbonMeasure carbonMeasure : measures) {
-      // check if measure is deleted
-      QueryMeasure queryMeasure = new QueryMeasure(carbonMeasure.getColName());
-      queryMeasure.setMeasure(carbonMeasure);
-      msrs.add(queryMeasure);
-    }
-    model.setQueryMeasures(msrs);
-    model.setQueryId(System.nanoTime() + "");
-    model.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-    model.setTable(carbonTable);
-    return model;
-  }
-
-  /**
    * Below method will be used
    * for cleanup
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
index 6afec0b..b18207d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator;
+import org.apache.carbondata.core.util.DataTypeConverterImpl;
 
 /**
  * Used to read carbon blocks when add/split partition
@@ -48,7 +49,8 @@ public class CarbonSplitExecutor extends AbstractCarbonQueryExecutor {
   public List<PartitionSpliterRawResultIterator> processDataBlocks(String segmentId)
       throws QueryExecutionException, IOException {
     List<TableBlockInfo> list = null;
-    queryModel = prepareQueryModel(list);
+    queryModel = carbonTable.createQueryModelWithProjectAllColumns(new DataTypeConverterImpl());
+    queryModel.setForcedDetailRawQuery(true);
     List<PartitionSpliterRawResultIterator> resultList
         = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     TaskBlockInfo taskBlockInfo = segmentMapping.get(segmentId);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
index ec91472..4abdf3c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
+import org.apache.carbondata.core.scan.model.QueryProjection;
 import org.apache.carbondata.processing.partition.Partition;
 import org.apache.carbondata.processing.partition.impl.DefaultLoadBalancer;
 import org.apache.carbondata.processing.partition.impl.PartitionMultiFileImpl;
@@ -46,7 +46,7 @@ public class CarbonQueryUtil {
    * It creates the one split for each region server.
    */
   public static synchronized TableSplit[] getTableSplits(String databaseName, String tableName,
-      CarbonQueryPlan queryPlan) {
+      QueryProjection queryPlan) {
 
     //Just create splits depends on locations of region servers
     List<Partition> allPartitions = null;
@@ -55,7 +55,7 @@ public class CarbonQueryUtil {
           QueryPartitionHelper.getInstance().getAllPartitions(databaseName, tableName);
     } else {
       allPartitions =
-          QueryPartitionHelper.getInstance().getPartitionsForQuery(queryPlan);
+          QueryPartitionHelper.getInstance().getPartitionsForQuery(databaseName, tableName);
     }
     TableSplit[] splits = new TableSplit[allPartitions.size()];
     for (int i = 0; i < splits.length; i++) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index 36a5a15..197cb14 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -150,7 +150,7 @@ class StreamHandoffRDD[K, V](
     CarbonTableInputFormat.setTableInfo(hadoopConf, carbonTable.getTableInfo)
     val attemptContext = new TaskAttemptContextImpl(hadoopConf, attemptId)
     val format = new CarbonTableInputFormat[Array[Object]]()
-    val model = format.getQueryModel(inputSplit, attemptContext)
+    val model = format.createQueryModel(inputSplit, attemptContext)
     val inputFormat = new CarbonStreamInputFormat
     val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
       .asInstanceOf[CarbonStreamRecordReader]


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
index 9391ebd..44f7c07 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
@@ -16,20 +16,10 @@
  */
 package org.apache.carbondata.core.scan.filter.executer;
 
-import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.List;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.util.ByteUtil;
 
 /**
  * It checks if filter is required on given block and if required, it does
@@ -38,12 +28,6 @@ import org.apache.carbondata.core.util.ByteUtil;
 public class ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl {
 
   /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ExcludeColGroupFilterExecuterImpl.class.getName());
-
-  /**
    * @param dimColResolvedFilterInfo
    * @param segmentProperties
    */
@@ -53,54 +37,6 @@ public class ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl
   }
 
   /**
-   * It fills BitSet with row index which matches filter key
-   */
-  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
-      int numerOfRows) {
-    BitSet bitSet = new BitSet(numerOfRows);
-    bitSet.flip(0, numerOfRows);
-    try {
-      KeyStructureInfo keyStructureInfo = getKeyStructureInfo();
-      byte[][] filterValues = dimColumnExecuterInfo.getExcludeFilterKeys();
-      for (int i = 0; i < filterValues.length; i++) {
-        byte[] filterVal = filterValues[i];
-        for (int rowId = 0; rowId < numerOfRows; rowId++) {
-          byte[] colData = new byte[keyStructureInfo.getMaskByteRanges().length];
-          dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, keyStructureInfo);
-          if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) == 0) {
-            bitSet.flip(rowId);
-          }
-        }
-      }
-
-    } catch (Exception e) {
-      LOGGER.error(e);
-    }
-
-    return bitSet;
-  }
-
-  /**
-   * It is required for extracting column data from columngroup chunk
-   *
-   * @return
-   * @throws KeyGenException
-   */
-  private KeyStructureInfo getKeyStructureInfo() throws KeyGenException {
-    int colGrpId = getColumnGroupId(dimColEvaluatorInfo.getColumnIndex());
-    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
-    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
-    mdKeyOrdinal.add(getMdkeyOrdinal(dimColEvaluatorInfo.getColumnIndex(), colGrpId));
-    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
-    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
-    KeyStructureInfo restructureInfos = new KeyStructureInfo();
-    restructureInfos.setKeyGenerator(keyGenerator);
-    restructureInfos.setMaskByteRanges(maskByteRanges);
-    restructureInfos.setMaxKey(maxKey);
-    return restructureInfos;
-  }
-
-  /**
    * Check if scan is required on given block based on min and max value
    */
   public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
@@ -109,25 +45,4 @@ public class ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl
     return bitSet;
   }
 
-  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
-    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
-  }
-
-  private int getColumnGroupId(int ordinal) {
-    int[][] columnGroups = segmentProperties.getColumnGroups();
-    int colGrpId = -1;
-    for (int i = 0; i < columnGroups.length; i++) {
-      if (columnGroups[i].length > 1) {
-        colGrpId++;
-        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
-          break;
-        }
-      }
-    }
-    return colGrpId;
-  }
-
-  public KeyGenerator getKeyGenerator(int colGrpId) {
-    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
index 465bee6..5974666 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -30,7 +30,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -40,13 +40,13 @@ import org.apache.carbondata.core.util.comparator.SerializableComparator;
 
 public class ExcludeFilterExecuterImpl implements FilterExecuter {
 
-  protected DimColumnResolvedFilterInfo dimColEvaluatorInfo;
-  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-  protected MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
-  protected MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
+  private DimColumnResolvedFilterInfo dimColEvaluatorInfo;
+  private DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+  private MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
+  private MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
   protected SegmentProperties segmentProperties;
-  protected boolean isDimensionPresentInCurrentBlock = false;
-  protected boolean isMeasurePresentInCurrentBlock = false;
+  private boolean isDimensionPresentInCurrentBlock = false;
+  private boolean isMeasurePresentInCurrentBlock = false;
   private SerializableComparator comparator;
   /**
    * is dimension column data is natural sorted
@@ -82,48 +82,50 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     if (isDimensionPresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk dimensionRawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
-      DimensionColumnDataChunk[] dimensionColumnDataChunks =
-          dimensionRawColumnChunk.convertToDimColDataChunks();
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
+      DimensionColumnPage[] dimensionColumnPages =
+          dimensionRawColumnChunk.decodeAllColumnPages();
       BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount());
-      for (int i = 0; i < dimensionColumnDataChunks.length; i++) {
-        BitSet bitSet = getFilteredIndexes(dimensionColumnDataChunks[i],
+      for (int i = 0; i < dimensionColumnPages.length; i++) {
+        BitSet bitSet = getFilteredIndexes(dimensionColumnPages[i],
             dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-            blockChunkHolder.getBitSetGroup(), i);
+            rawBlockletColumnChunks.getBitSetGroup(), i);
         bitSetGroup.setBitSet(bitSet, i);
       }
 
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       MeasureRawColumnChunk measureRawColumnChunk =
-          blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       ColumnPage[] ColumnPages =
-          measureRawColumnChunk.convertToColumnPage();
+          measureRawColumnChunk.decodeAllColumnPages();
       BitSetGroup bitSetGroup = new BitSetGroup(measureRawColumnChunk.getPagesCount());
       DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo);
       for (int i = 0; i < ColumnPages.length; i++) {
         BitSet bitSet =
             getFilteredIndexesForMeasure(
-                measureRawColumnChunk.convertToColumnPage(i),
+                measureRawColumnChunk.decodeColumnPage(i),
                 measureRawColumnChunk.getRowCount()[i],
                 useBitsetPipeLine,
-                blockChunkHolder.getBitSetGroup(),
+                rawBlockletColumnChunks.getBitSetGroup(),
                 i,
                 msrType);
         bitSetGroup.setBitSet(bitSet, i);
@@ -133,7 +135,8 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     return null;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
     if (isDimensionPresentInCurrentBlock) {
       byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
       byte[] col = (byte[])value.getVal(dimColEvaluatorInfo.getDimension().getOrdinal());
@@ -275,43 +278,41 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
 
   /**
    * Below method will be used to apply filter on dimension column
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numberOfRows
    * @param useBitsetPipeLine
    * @param prvBitSetGroup
    * @param pageNumber
    * @return filtered indexes bitset
    */
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows, boolean useBitsetPipeLine, BitSetGroup prvBitSetGroup, int pageNumber) {
     // check whether applying filtered based on previous bitset will be optimal
     if (CarbonUtil.usePreviousFilterBitsetGroup(useBitsetPipeLine, prvBitSetGroup, pageNumber,
         dimColumnExecuterInfo.getExcludeFilterKeys().length)) {
-      return getFilteredIndexesUisngPrvBitset(dimensionColumnDataChunk, prvBitSetGroup, pageNumber,
-          numberOfRows);
+      return getFilteredIndexesUisngPrvBitset(dimensionColumnPage, prvBitSetGroup, pageNumber);
     } else {
-      return getFilteredIndexes(dimensionColumnDataChunk, numberOfRows);
+      return getFilteredIndexes(dimensionColumnPage, numberOfRows);
     }
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows) {
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numberOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numberOfRows);
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numberOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnPage, numberOfRows);
   }
 
   /**
    * Below method will be used to apply filter based on previous filtered bitset
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param prvBitSetGroup
    * @param pageNumber
-   * @param numberOfRows
    * @return filtered indexes bitset
    */
-  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensionColumnDataChunk,
-      BitSetGroup prvBitSetGroup, int pageNumber, int numberOfRows) {
+  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage,
+      BitSetGroup prvBitSetGroup, int pageNumber) {
     BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber);
     BitSet bitSet = new BitSet();
     bitSet.or(prvPageBitSet);
@@ -319,11 +320,11 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     int compareResult = 0;
     // if dimension data was natural sorted then get the index from previous bitset
     // and use the same in next column data, otherwise use the inverted index reverse
-    if (!dimensionColumnDataChunk.isExplicitSorted()) {
+    if (!dimensionColumnPage.isExplicitSorted()) {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, index);
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, index);
         if (compareResult != 0) {
           bitSet.set(index);
         } else {
@@ -336,8 +337,8 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1,
-                dimensionColumnDataChunk.getInvertedReverseIndex(index));
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1,
+                dimensionColumnPage.getInvertedReverseIndex(index));
         if (compareResult != 0) {
           bitSet.set(index);
         } else {
@@ -351,7 +352,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   }
 
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     bitSet.flip(0, numerOfRows);
     int startIndex = 0;
@@ -361,10 +362,10 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
         break;
       }
       int[] rangeIndex = CarbonUtil
-          .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i]);
       for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
-        bitSet.flip(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.flip(dimensionColumnPage.getInvertedIndex(j));
       }
       if (rangeIndex[1] >= 0) {
         startIndex = rangeIndex[1] + 1;
@@ -373,7 +374,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     bitSet.flip(0, numerOfRows);
@@ -390,7 +391,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
           break;
         }
         int[] rangeIndex = CarbonUtil
-            .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[i]);
         for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
           bitSet.flip(j);
@@ -403,14 +404,14 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       if (filterValues.length > 1) {
         for (int i = 0; i < numerOfRows; i++) {
           int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1,
-              dimensionColumnDataChunk.getChunkData(i));
+              dimensionColumnPage.getChunkData(i));
           if (index >= 0) {
             bitSet.flip(i);
           }
         }
       } else {
         for (int j = 0; j < numerOfRows; j++) {
-          if (dimensionColumnDataChunk.compareTo(j, filterValues[0]) == 0) {
+          if (dimensionColumnPage.compareTo(j, filterValues[0]) == 0) {
             bitSet.flip(j);
           }
         }
@@ -419,26 +420,30 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     bitSet.flip(0, 1);
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
index 85891dc..aac5e63 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.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 interface FilterExecuter {
@@ -32,8 +32,8 @@ public interface FilterExecuter {
    * @return
    * @throws FilterUnsupportedException
    */
-  BitSetGroup applyFilter(BlocksChunkHolder blocksChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException;
+  BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException;
 
   boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException;
@@ -50,7 +50,7 @@ public interface FilterExecuter {
 
   /**
    * It just reads necessary block for filter executor, it does not uncompress the data.
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    */
-  void readBlocks(BlocksChunkHolder blockChunkHolder)throws IOException;
+  void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks)throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
index 9c4c7ba..b683fd6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
@@ -23,7 +23,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.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
@@ -41,26 +41,31 @@ public class ImplicitIncludeFilterExecutorImpl
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeline)
-      throws FilterUnsupportedException {
-    BitSetGroup bitSetGroup = new BitSetGroup(blockChunkHolder.getDataBlock().numberOfPages());
-    for (int i = 0; i < blockChunkHolder.getDataBlock().numberOfPages(); i++) {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeline) {
+    BitSetGroup bitSetGroup = new BitSetGroup(
+        rawBlockletColumnChunks.getDataBlock().numberOfPages());
+    for (int i = 0; i < rawBlockletColumnChunks.getDataBlock().numberOfPages(); i++) {
       bitSetGroup.setBitSet(
-          setBitSetForCompleteDimensionData(blockChunkHolder.getDataBlock().getPageRowCount(i)), i);
+          setBitSetForCompleteDimensionData(
+              rawBlockletColumnChunks.getDataBlock().getPageRowCount(i)), i);
     }
     return bitSetGroup;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException {
     return false;
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     return null;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
 
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
index 1cbc0bc..e4da26f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
@@ -24,14 +24,14 @@ import java.util.List;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 
@@ -59,7 +59,7 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
   /**
    * It fills BitSet with row index which matches filter key
    */
-  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  protected BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
 
@@ -70,7 +70,7 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
         byte[] filterVal = filterValues[i];
         for (int rowId = 0; rowId < numerOfRows; rowId++) {
           byte[] colData = new byte[keyStructureInfo.getMaskByteRanges().length];
-          dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, keyStructureInfo);
+          dimensionColumnPage.fillRawData(rowId, 0, colData, keyStructureInfo);
           if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) == 0) {
             bitSet.set(rowId);
           }
@@ -85,20 +85,21 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
         .get(dimColumnEvaluatorInfo.getColumnIndex());
-    if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-      blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+      rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+          rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+              rawBlockletColumnChunks.getFileReader(), chunkIndex);
     }
     DimensionRawColumnChunk dimensionRawColumnChunk =
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
     BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount());
     for (int i = 0; i < dimensionRawColumnChunk.getPagesCount(); i++) {
       if (dimensionRawColumnChunk.getMaxValues() != null) {
-        BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i),
+        BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i),
             dimensionRawColumnChunk.getRowCount()[i]);
         bitSetGroup.setBitSet(bitSet, i);
       }
@@ -133,10 +134,10 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
     BitSet bitSet = new BitSet(1);
     byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
     int columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
+    int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping().get(columnIndex);
     int[] cols = getAllColumns(columnIndex);
-    byte[] maxValue = getMinMaxData(cols, blkMaxVal[blockIndex], columnIndex);
-    byte[] minValue = getMinMaxData(cols, blkMinVal[blockIndex], columnIndex);
+    byte[] maxValue = getMinMaxData(cols, blkMaxVal[chunkIndex], columnIndex);
+    byte[] minValue = getMinMaxData(cols, blkMinVal[chunkIndex], columnIndex);
     boolean isScanRequired = false;
     for (int k = 0; k < filterValues.length; k++) {
       // filter value should be in range of max and min value i.e

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
index fe1421c..05328f3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -30,7 +30,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -41,12 +41,12 @@ import org.apache.carbondata.core.util.comparator.SerializableComparator;
 public class IncludeFilterExecuterImpl implements FilterExecuter {
 
   protected DimColumnResolvedFilterInfo dimColumnEvaluatorInfo;
-  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-  protected MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
-  protected MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
+  DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+  private MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
+  private MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
   protected SegmentProperties segmentProperties;
-  protected boolean isDimensionPresentInCurrentBlock = false;
-  protected boolean isMeasurePresentInCurrentBlock = false;
+  private boolean isDimensionPresentInCurrentBlock = false;
+  private boolean isMeasurePresentInCurrentBlock = false;
   protected SerializableComparator comparator;
   /**
    * is dimension column data is natural sorted
@@ -86,44 +86,46 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    if (isDimensionPresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    if (isDimensionPresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk dimensionRawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount());
       for (int i = 0; i < dimensionRawColumnChunk.getPagesCount(); i++) {
         if (dimensionRawColumnChunk.getMaxValues() != null) {
           if (isScanRequired(dimensionRawColumnChunk.getMaxValues()[i],
               dimensionRawColumnChunk.getMinValues()[i], dimColumnExecuterInfo.getFilterKeys())) {
-            BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i),
                 dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-                blockChunkHolder.getBitSetGroup(), i);
+                rawBlockletColumnChunks.getBitSetGroup(), i);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i),
               dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-              blockChunkHolder.getBitSetGroup(), i);
+              rawBlockletColumnChunks.getBitSetGroup(), i);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       MeasureRawColumnChunk measureRawColumnChunk =
-          blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(measureRawColumnChunk.getPagesCount());
       DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo);
       for (int i = 0; i < measureRawColumnChunk.getPagesCount(); i++) {
@@ -132,16 +134,16 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
               measureRawColumnChunk.getMinValues()[i], msrColumnExecutorInfo.getFilterKeys(),
               msrColumnEvaluatorInfo.getType())) {
             BitSet bitSet =
-                getFilteredIndexesForMeasure(measureRawColumnChunk.convertToColumnPage(i),
+                getFilteredIndexesForMeasure(measureRawColumnChunk.decodeColumnPage(i),
                     measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-                    blockChunkHolder.getBitSetGroup(), i, msrType);
+                    rawBlockletColumnChunks.getBitSetGroup(), i, msrType);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasure(measureRawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasure(measureRawColumnChunk.decodeColumnPage(i),
                   measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-                  blockChunkHolder.getBitSetGroup(), i, msrType);
+                  rawBlockletColumnChunks.getBitSetGroup(), i, msrType);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
@@ -293,43 +295,43 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
 
   /**
    * Below method will be used to apply filter on dimension column
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numberOfRows
    * @param useBitsetPipeLine
    * @param prvBitSetGroup
    * @param pageNumber
    * @return filtered indexes bitset
    */
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows, boolean useBitsetPipeLine, BitSetGroup prvBitSetGroup, int pageNumber) {
     // check whether previous indexes can be optimal to apply filter on dimension column
     if (CarbonUtil.usePreviousFilterBitsetGroup(useBitsetPipeLine, prvBitSetGroup, pageNumber,
         dimColumnExecuterInfo.getFilterKeys().length)) {
-      return getFilteredIndexesUisngPrvBitset(dimensionColumnDataChunk, prvBitSetGroup, pageNumber,
+      return getFilteredIndexesUisngPrvBitset(dimensionColumnPage, prvBitSetGroup, pageNumber,
           numberOfRows);
     } else {
-      return getFilteredIndexes(dimensionColumnDataChunk, numberOfRows);
+      return getFilteredIndexes(dimensionColumnPage, numberOfRows);
     }
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows) {
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numberOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numberOfRows);
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numberOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnPage, numberOfRows);
   }
 
   /**
    * Below method will be used to apply filter on dimension
    * column based on previous filtered indexes
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param prvBitSetGroup
    * @param pageNumber
    * @param numberOfRows
    * @return filtered bitset
    */
-  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage,
       BitSetGroup prvBitSetGroup, int pageNumber, int numberOfRows) {
     BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber);
     BitSet bitSet = new BitSet(numberOfRows);
@@ -337,11 +339,11 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     int compareResult = 0;
     // if dimension data was natural sorted then get the index from previous bitset
     // and use the same in next column data, otherwise use the inverted index reverse
-    if (!dimensionColumnDataChunk.isExplicitSorted()) {
+    if (!dimensionColumnPage.isExplicitSorted()) {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, index);
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, index);
         if (compareResult == 0) {
           bitSet.set(index);
         }
@@ -350,8 +352,8 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1,
-                dimensionColumnDataChunk.getInvertedReverseIndex(index));
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1,
+                dimensionColumnPage.getInvertedReverseIndex(index));
         if (compareResult == 0) {
           bitSet.set(index);
         }
@@ -360,7 +362,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int startIndex = 0;
     byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
@@ -369,10 +371,10 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
         break;
       }
       int[] rangeIndex = CarbonUtil
-          .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i]);
       for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
       }
       if (rangeIndex[1] >= 0) {
         startIndex = rangeIndex[1] + 1;
@@ -381,7 +383,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
@@ -394,7 +396,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
           break;
         }
         int[] rangeIndex = CarbonUtil
-            .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[i]);
         for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
           bitSet.set(j);
@@ -407,14 +409,14 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
       if (filterValues.length > 1) {
         for (int i = 0; i < numerOfRows; i++) {
           int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1,
-              dimensionColumnDataChunk.getChunkData(i));
+              dimensionColumnPage.getChunkData(i));
           if (index >= 0) {
             bitSet.set(i);
           }
         }
       } else {
         for (int j = 0; j < numerOfRows; j++) {
-          if (dimensionColumnDataChunk.compareTo(j, filterValues[0]) == 0) {
+          if (dimensionColumnPage.compareTo(j, filterValues[0]) == 0) {
             bitSet.set(j);
           }
         }
@@ -423,26 +425,25 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
+  @Override
   public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
     BitSet bitSet = new BitSet(1);
     byte[][] filterValues = null;
     int columnIndex = 0;
-    int blockIndex = 0;
+    int chunkIndex = 0;
     boolean isScanRequired = false;
 
     if (isDimensionPresentInCurrentBlock) {
       filterValues = dimColumnExecuterInfo.getFilterKeys();
       columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
-      blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
-      isScanRequired =
-          isScanRequired(blkMaxVal[blockIndex], blkMinVal[blockIndex], filterValues);
-
+      chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping().get(columnIndex);
+      isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], filterValues);
     } else if (isMeasurePresentInCurrentBlock) {
       columnIndex = msrColumnEvaluatorInfo.getColumnIndex();
-      blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(columnIndex) + segmentProperties
-              .getLastDimensionColOrdinal();
-      isScanRequired = isScanRequired(blkMaxVal[blockIndex], blkMinVal[blockIndex],
+      chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(columnIndex) +
+              segmentProperties.getLastDimensionColOrdinal();
+      isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex],
           msrColumnExecutorInfo.getFilterKeys(),
           msrColumnEvaluatorInfo.getType());
     }
@@ -492,20 +493,23 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return false;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
-    if (isDimensionPresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    if (isDimensionPresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-    } else if (isMeasurePresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+    } else if (isMeasurePresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
index 87273bb..9f0afb5 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.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 OrFilterExecuterImpl implements FilterExecuter {
@@ -35,30 +35,33 @@ public class OrFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    BitSetGroup leftFilters = leftExecuter.applyFilter(blockChunkHolder, false);
-    BitSetGroup rightFilters = rightExecuter.applyFilter(blockChunkHolder, false);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    BitSetGroup leftFilters = leftExecuter.applyFilter(rawBlockletColumnChunks, false);
+    BitSetGroup rightFilters = rightExecuter.applyFilter(rawBlockletColumnChunks, false);
     leftFilters.or(rightFilters);
-    blockChunkHolder.setBitSetGroup(leftFilters);
+    rawBlockletColumnChunks.setBitSetGroup(leftFilters);
     return leftFilters;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException {
     return leftExecuter.applyFilter(value, dimOrdinalMax) ||
         rightExecuter.applyFilter(value, dimOrdinalMax);
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
     BitSet rightFilters = rightExecuter.isScanRequired(blockMaxValue, blockMinValue);
     leftFilters.or(rightFilters);
     return leftFilters;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
-    leftExecuter.readBlocks(blockChunkHolder);
-    rightExecuter.readBlocks(blockChunkHolder);
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    leftExecuter.readColumnChunks(rawBlockletColumnChunks);
+    rightExecuter.readColumnChunks(rawBlockletColumnChunks);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
index ee373c5..4a9716c 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
@@ -22,11 +22,10 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -38,8 +37,7 @@ import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -59,25 +57,23 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
   /**
    * it has index at which given dimension is stored in file
    */
-  protected int dimensionBlocksIndex;
+  private int dimensionChunkIndex;
 
   /**
    * flag to check whether the filter dimension is present in current block list of dimensions.
    * Applicable for restructure scenarios
    */
-  protected boolean isDimensionPresentInCurrentBlock;
-  boolean lessThanExp;
-  boolean lessThanEqualExp;
-  boolean greaterThanExp;
-  boolean greaterThanEqualExp;
-  boolean startBlockMinIsDefaultStart;
-  boolean endBlockMaxisDefaultEnd;
-  boolean isRangeFullyCoverBlock;
+  private boolean isDimensionPresentInCurrentBlock;
+  private boolean lessThanExp;
+  private boolean lessThanEqualExp;
+  private boolean greaterThanExp;
+  private boolean greaterThanEqualExp;
+  private boolean startBlockMinIsDefaultStart;
+  private boolean endBlockMaxisDefaultEnd;
+  private boolean isRangeFullyCoverBlock;
 
   public RangeValueFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
-      MeasureColumnResolvedFilterInfo msrColEvaluatorInfo, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
-      SegmentProperties segmentProperties) {
+      Expression exp, byte[][] filterRangeValues, SegmentProperties segmentProperties) {
 
     this.dimColEvaluatorInfo = dimColEvaluatorInfo;
     this.exp = exp;
@@ -90,7 +86,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     startBlockMinIsDefaultStart = false;
     endBlockMaxisDefaultEnd = false;
     isRangeFullyCoverBlock = false;
-    initDimensionBlockIndexes();
+    initDimensionChunkIndexes();
     ifDefaultValueMatchesFilter();
 
   }
@@ -99,13 +95,13 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * This method will initialize the dimension info for the current block to be
    * used for filtering the data
    */
-  private void initDimensionBlockIndexes() {
+  private void initDimensionChunkIndexes() {
     // find the dimension in the current block dimensions list
     CarbonDimension dimensionFromCurrentBlock =
         segmentProperties.getDimensionFromCurrentBlock(dimColEvaluatorInfo.getDimension());
     if (null != dimensionFromCurrentBlock) {
       dimColEvaluatorInfo.setColumnIndex(dimensionFromCurrentBlock.getOrdinal());
-      this.dimensionBlocksIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+      this.dimensionChunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimensionFromCurrentBlock.getOrdinal());
       isDimensionPresentInCurrentBlock = true;
     }
@@ -135,14 +131,14 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
 
   /**
    * Method to apply the filter.
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    * @return
    * @throws FilterUnsupportedException
    * @throws IOException
    */
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    return applyNoAndDirectFilter(blockChunkHolder);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    return applyNoAndDirectFilter(rawBlockletColumnChunks);
   }
 
   /**
@@ -243,32 +239,32 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     //         Filter Min <-----------------------------------------------> Filter Max
 
     if (isDimensionPresentInCurrentBlock) {
-      if (((lessThanExp == true) && (
+      if (((lessThanExp) && (
           ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) >= 0)) || (
-          (lessThanEqualExp == true) && (
+          (lessThanEqualExp) && (
               ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) > 0)) || (
-          (greaterThanExp == true) && (
+          (greaterThanExp) && (
               ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) >= 0)) || (
-          (greaterThanEqualExp == true) && (
+          (greaterThanEqualExp) && (
               ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) > 0))) {
         // completely out of block boundary
         isScanRequired = false;
       } else {
-        if (((greaterThanExp == true) && (
+        if (((greaterThanExp) && (
             ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) > 0)) || (
-            (greaterThanEqualExp == true) && (
+            (greaterThanEqualExp) && (
                 ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) >= 0))) {
           startBlockMinIsDefaultStart = true;
         }
 
-        if (((lessThanExp == true) && (
+        if (((lessThanExp) && (
             ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) > 0)) || (
-            (lessThanEqualExp == true) && (
+            (lessThanEqualExp) && (
                 ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) >= 0))) {
           endBlockMaxisDefaultEnd = true;
         }
 
-        if (startBlockMinIsDefaultStart == true && endBlockMaxisDefaultEnd == true) {
+        if (startBlockMinIsDefaultStart && endBlockMaxisDefaultEnd) {
           isRangeFullyCoverBlock = true;
         }
       }
@@ -284,7 +280,8 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * @param blockMinValue
    * @return
    */
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     byte[][] filterValues = this.filterRangesValues;
     int columnIndex = this.dimColEvaluatorInfo.getColumnIndex();
@@ -300,50 +297,49 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * Method to apply the Range Filter.
    * @param blockChunkHolder
    * @return
-   * @throws FilterUnsupportedException
    * @throws IOException
    */
-  public BitSetGroup applyNoAndDirectFilter(BlocksChunkHolder blockChunkHolder)
-      throws FilterUnsupportedException, IOException {
+  private BitSetGroup applyNoAndDirectFilter(RawBlockletColumnChunks blockChunkHolder)
+      throws IOException {
 
     // In case of Alter Table Add and Delete Columns the isDimensionPresentInCurrentBlock can be
     // false, in that scenario the default values of the column should be shown.
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
-      return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
-              numberOfRows, true);
+      int numberOfRows = blockChunkHolder.getDataBlock().numRows();
+      return FilterUtil.createBitSetGroupWithDefaultValue(
+          blockChunkHolder.getDataBlock().numberOfPages(), numberOfRows, true);
     }
 
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+    int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
         .get(dimColEvaluatorInfo.getColumnIndex());
 
-    if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-      blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    if (null == blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex]) {
+      blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex] =
+          blockChunkHolder.getDataBlock().readDimensionChunk(
+              blockChunkHolder.getFileReader(), chunkIndex);
     }
 
     DimensionRawColumnChunk rawColumnChunk =
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+        blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex];
     BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
     for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
       if (rawColumnChunk.getMaxValues() != null) {
         if (isScanRequired(rawColumnChunk.getMinValues()[i], rawColumnChunk.getMaxValues()[i],
             this.filterRangesValues)) {
-          if (isRangeFullyCoverBlock == true) {
+          if (isRangeFullyCoverBlock) {
             // Set all the bits in this case as filter Min Max values cover the whole block.
             BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]);
             bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           } else {
-            BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                 rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         }
       } else {
-        BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+        BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
             rawColumnChunk.getRowCount()[i]);
         bitSetGroup.setBitSet(bitSet, i);
       }
@@ -351,12 +347,12 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     return bitSetGroup;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows);
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
   }
 
   /**
@@ -365,12 +361,12 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
     int startIndex = 0;
@@ -381,13 +377,13 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     // For Range expression we expect two values. The First is the Min Value and Second is the
     // Max value.
     // Get the Min Value
-    if (startBlockMinIsDefaultStart == false) {
+    if (!startBlockMinIsDefaultStart) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[0], greaterThanExp);
-      if (greaterThanExp == true && start >= 0) {
+      if (greaterThanExp && start >= 0) {
         start = CarbonUtil
-            .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
+            .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[0],
                 numerOfRows);
       }
 
@@ -399,8 +395,8 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
         // Method will compare the tentative index value after binary search, this tentative
         // index needs to be compared by the filter member if its >= filter then from that
         // index the bitset will be considered for filtering process.
-        if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk
-            .getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))) > 0) {
+        if ((ByteUtil.compare(filterValues[0], dimensionColumnPage
+            .getChunkData(dimensionColumnPage.getInvertedIndex(start)))) > 0) {
           start = start + 1;
         }
       }
@@ -411,14 +407,14 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     }
 
     // Get the Max value
-    if (endBlockMaxisDefaultEnd == false) {
+    if (!endBlockMaxisDefaultEnd) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[1], lessThanEqualExp);
 
-      if (lessThanExp == true && start >= 0) {
+      if (lessThanExp && start >= 0) {
         start =
-            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
+            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[1]);
       }
 
       if (start < 0) {
@@ -429,7 +425,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
         // In case the start is less than 0, then positive value of start is pointing to the next
         // value of the searched key. So move to the previous one.
         if ((ByteUtil.compare(filterValues[1],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             < 0)) {
           start = start - 1;
         }
@@ -440,18 +436,18 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     }
 
     for (int j = startMin; j <= endMax; j++) {
-      bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+      bitSet.set(dimensionColumnPage.getInvertedIndex(j));
     }
 
     // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on
     // matching row.
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet);
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      updateForNoDictionaryColumn(startMin, endMax, dimensionColumnPage, bitSet);
     }
     return bitSet;
   }
 
-  private void updateForNoDictionaryColumn(int start, int end, DimensionColumnDataChunk dataChunk,
+  private void updateForNoDictionaryColumn(int start, int end, DimensionColumnPage dataChunk,
       BitSet bitset) {
     for (int j = start; j <= end; j++) {
       if (dataChunk.compareTo(j, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY) == 0
@@ -467,31 +463,31 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * be called if the column is sorted default so column index
    * mapping will be present for accesaing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
-    // if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    // if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
     byte[][] filterValues = this.filterRangesValues;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
+    if (dimensionColumnPage.isExplicitSorted()) {
       int start = 0;
       int startMin = 0;
       int endMax = 0;
       int startIndex = 0;
       // For Range expression we expect two values. The First is the Min Value and Second is the
       // Max value.
-      if (startBlockMinIsDefaultStart == false) {
+      if (!startBlockMinIsDefaultStart) {
 
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[0], greaterThanExp);
 
-        if (greaterThanExp == true && start >= 0) {
+        if (greaterThanExp && start >= 0) {
           start = CarbonUtil
-              .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
+              .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[0],
                   numerOfRows);
         }
 
@@ -503,7 +499,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
           // Method will compare the tentative index value after binary search, this tentative
           // index needs to be compared by the filter member if its >= filter then from that
           // index the bitset will be considered for filtering process.
-          if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk.getChunkData(start)))
+          if ((ByteUtil.compare(filterValues[0], dimensionColumnPage.getChunkData(start)))
               > 0) {
             start = start + 1;
           }
@@ -513,14 +509,14 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
         startMin = startIndex;
       }
 
-      if (endBlockMaxisDefaultEnd == false) {
+      if (!endBlockMaxisDefaultEnd) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[1], lessThanEqualExp);
 
-        if (lessThanExp == true && start >= 0) {
+        if (lessThanExp && start >= 0) {
           start =
-              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
+              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[1]);
         }
 
         if (start < 0) {
@@ -530,7 +526,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
           }
           // In case the start is less than 0, then positive value of start is pointing to the next
           // value of the searched key. So move to the previous one.
-          if ((ByteUtil.compare(filterValues[1], dimensionColumnDataChunk.getChunkData(start))
+          if ((ByteUtil.compare(filterValues[1], dimensionColumnPage.getChunkData(start))
               < 0)) {
             start = start - 1;
           }
@@ -546,8 +542,8 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
 
       // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on
       // matching row.
-      if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-        updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet);
+      if (dimensionColumnPage.isNoDicitionaryColumn()) {
+        updateForNoDictionaryColumn(startMin, endMax, dimensionColumnPage, bitSet);
       }
     } else {
       byte[] defaultValue = null;
@@ -556,7 +552,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
             .getDirectDictionaryGenerator(dimColEvaluatorInfo.getDimension().getDataType());
         int key = directDictionaryGenerator.generateDirectSurrogateKey(null);
         CarbonDimension currentBlockDimension =
-            segmentProperties.getDimensions().get(dimensionBlocksIndex);
+            segmentProperties.getDimensions().get(dimensionChunkIndex);
         if (currentBlockDimension.isSortColumn()) {
           defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
               this.segmentProperties.getSortColumnsGenerator());
@@ -572,55 +568,37 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
       }
       // evaluate result for lower range value first and then perform and operation in the
       // upper range value in order to compute the final result
-      bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[0],
+      bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnPage, filterValues[0],
           numerOfRows);
       BitSet upperRangeBitSet =
-          evaluateLessThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[1],
+          evaluateLessThanFilterForUnsortedColumn(dimensionColumnPage, filterValues[1],
               numerOfRows);
       bitSet.and(upperRangeBitSet);
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet, defaultValue);
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet, defaultValue);
     }
     return bitSet;
   }
 
   /**
-   * This method will compare the selected data against null values and
-   * flip the bitSet if any null value is found
-   *
-   * @param dimensionColumnDataChunk
-   * @param bitSet
-   */
-  private void removeNullValues(DimensionColumnDataChunk dimensionColumnDataChunk, BitSet bitSet) {
-    if (!bitSet.isEmpty()) {
-      for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-        if (dimensionColumnDataChunk.compareTo(i, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY)
-            == 0) {
-          bitSet.flip(i);
-        }
-      }
-    }
-  }
-
-  /**
    * This method will evaluate the result for filter column based on the lower range value
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param filterValue
    * @param numberOfRows
    * @return
    */
   private BitSet evaluateGreaterThanFilterForUnsortedColumn(
-      DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) {
+      DimensionColumnPage dimensionColumnPage, byte[] filterValue, int numberOfRows) {
     BitSet bitSet = new BitSet(numberOfRows);
     if (greaterThanExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) > 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) > 0)) {
           bitSet.set(i);
         }
       }
     } else if (greaterThanEqualExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) >= 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) >= 0)) {
           bitSet.set(i);
         }
       }
@@ -631,23 +609,23 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
   /**
    * This method will evaluate the result for filter column based on the upper range value
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param filterValue
    * @param numberOfRows
    * @return
    */
   private BitSet evaluateLessThanFilterForUnsortedColumn(
-      DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) {
+      DimensionColumnPage dimensionColumnPage, byte[] filterValue, int numberOfRows) {
     BitSet bitSet = new BitSet(numberOfRows);
     if (lessThanExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) < 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) < 0)) {
           bitSet.set(i);
         }
       }
     } else if (lessThanEqualExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) <= 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) <= 0)) {
           bitSet.set(i);
         }
       }
@@ -657,16 +635,18 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
 
   /**
    * Method to read the blocks.
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    * @throws IOException
    */
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
-    if (isDimensionPresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    if (isDimensionPresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
index 5707eb4..d7bec7e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
@@ -24,7 +24,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-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 RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorImpl {
@@ -34,7 +34,7 @@ public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorIm
    * flag to check whether filter values contain the default value applied on the dimension column
    * which does not exist in the current block
    */
-  protected boolean isDefaultValuePresentInFilterValues;
+  private boolean isDefaultValuePresentInFilterValues;
 
   public RestructureExcludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
       MeasureColumnResolvedFilterInfo measureColumnResolvedFilterInfo, boolean isMeasure) {
@@ -48,26 +48,29 @@ public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorIm
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
     return FilterUtil
-        .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+        .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
             numberOfRows, !isDefaultValuePresentInFilterValues);
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException {
     throw new FilterUnsupportedException("Unsupported RestructureExcludeFilterExecutorImpl on row");
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     bitSet.flip(0, 1);
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) {
 
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
index 8bcc53f..c874fc5 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
@@ -24,7 +24,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-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 RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorImpl {
@@ -33,7 +33,7 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm
    * flag to check whether filter values contain the default value applied on the dimension column
    * which does not exist in the current block
    */
-  protected boolean isDefaultValuePresentInFilterValues;
+  private boolean isDefaultValuePresentInFilterValues;
 
   public RestructureIncludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColumnEvaluatorInfo,
       MeasureColumnResolvedFilterInfo measureColumnResolvedFilterInfo, boolean isMeasure) {
@@ -47,15 +47,16 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
-    return FilterUtil
-        .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
-            numberOfRows, isDefaultValuePresentInFilterValues);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
+    return FilterUtil.createBitSetGroupWithDefaultValue(
+        rawBlockletColumnChunks.getDataBlock().numberOfPages(),
+        numberOfRows, isDefaultValuePresentInFilterValues);
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException {
     throw new FilterUnsupportedException("Unsupported RestructureIncludeFilterExecutorImpl on row");
   }
@@ -66,7 +67,8 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) {
 
   }
 


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
index 553f85e..773fbd7 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
@@ -17,20 +17,15 @@
 package org.apache.carbondata.core.scan.result.iterator;
 
 import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 
 public class PartitionSpliterRawResultIterator extends CarbonIterator<Object[]> {
 
-  private CarbonIterator<BatchResult> iterator;
-  private BatchResult batch;
+  private CarbonIterator<RowBatch> iterator;
+  private RowBatch batch;
   private int counter;
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName());
-
-  public PartitionSpliterRawResultIterator(CarbonIterator<BatchResult> iterator) {
+  public PartitionSpliterRawResultIterator(CarbonIterator<RowBatch> iterator) {
     this.iterator = iterator;
   }
 
@@ -65,7 +60,7 @@ public class PartitionSpliterRawResultIterator extends CarbonIterator<Object[]>
    * @param batch
    * @return
    */
-  private boolean checkBatchEnd(BatchResult batch) {
+  private boolean checkBatchEnd(RowBatch batch) {
     return !(counter < batch.getSize());
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
index 70d0958..1dd1595 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
@@ -21,7 +21,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 
 /**
@@ -37,7 +37,7 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
   /**
    * Iterator of the Batch raw result.
    */
-  private CarbonIterator<BatchResult> detailRawQueryResultIterator;
+  private CarbonIterator<RowBatch> detailRawQueryResultIterator;
 
   /**
    * Counter to maintain the row counter.
@@ -55,9 +55,9 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
   /**
    * batch of the result.
    */
-  private BatchResult batch;
+  private RowBatch batch;
 
-  public RawResultIterator(CarbonIterator<BatchResult> detailRawQueryResultIterator,
+  public RawResultIterator(CarbonIterator<RowBatch> detailRawQueryResultIterator,
       SegmentProperties sourceSegProperties, SegmentProperties destinationSegProperties) {
     this.detailRawQueryResultIterator = detailRawQueryResultIterator;
     this.sourceSegProperties = sourceSegProperties;
@@ -155,7 +155,7 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
    * @param batch
    * @return
    */
-  private boolean checkIfBatchIsProcessedCompletely(BatchResult batch) {
+  private boolean checkIfBatchIsProcessedCompletely(RowBatch batch) {
     if (counter < batch.getSize()) {
       return false;
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
index cc9710e..c7cb00d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
@@ -35,10 +35,12 @@ public class VectorDetailQueryResultIterator extends AbstractDetailQueryResultIt
     super(infos, queryModel, execService);
   }
 
-  @Override public Object next() {
+  @Override
+  public Object next() {
     throw new UnsupportedOperationException("call processNextBatch instead");
   }
 
+  @Override
   public void processNextBatch(CarbonColumnarBatch columnarBatch) {
     synchronized (lock) {
       updateDataBlockIterator();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
index cfc2f16..973ce0f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
@@ -87,7 +87,4 @@ public class CarbonColumnarBatch {
     }
   }
 
-  public int getRowsFilteredCount() {
-    return rowsFiltered;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
index a5f81b9..59117dd 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
@@ -18,16 +18,16 @@ package org.apache.carbondata.core.scan.result.vector;
 
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-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;
 
 public class ColumnVectorInfo implements Comparable<ColumnVectorInfo> {
   public int offset;
   public int size;
   public CarbonColumnVector vector;
   public int vectorOffset;
-  public QueryDimension dimension;
-  public QueryMeasure measure;
+  public ProjectionDimension dimension;
+  public ProjectionMeasure measure;
   public int ordinal;
   public DirectDictionaryGenerator directDictionaryGenerator;
   public MeasureDataVectorProcessor.MeasureVectorFiller measureVectorFiller;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
index db4c982..8902dfb 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
@@ -29,7 +29,7 @@ public class MeasureDataVectorProcessor {
 
     void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info);
 
-    void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info);
   }
 
@@ -60,7 +60,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -69,13 +69,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putInt(vectorOffset, (int)dataChunk.getLong(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -117,7 +117,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping,
+    public void fillMeasureVector(int[] filteredRowId,
         ColumnPage dataChunk, ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -126,13 +126,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putBoolean(vectorOffset, dataChunk.getBoolean(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -171,7 +171,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -180,13 +180,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putShort(vectorOffset, (short) dataChunk.getLong(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -225,7 +225,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -234,13 +234,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putLong(vectorOffset, dataChunk.getLong(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -279,7 +279,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -288,7 +288,7 @@ public class MeasureDataVectorProcessor {
       int precision = info.measure.getMeasure().getPrecision();
       BitSet nullBitSet = dataChunk.getNullBits();
       for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
+        int currentRow = filteredRowId[i];
         if (nullBitSet.get(currentRow)) {
           vector.putNull(vectorOffset);
         } else {
@@ -330,7 +330,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -339,13 +339,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putDouble(vectorOffset, dataChunk.getDouble(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java
deleted file mode 100644
index bf26ca3..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java
+++ /dev/null
@@ -1,181 +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.scan.scanner;
-
-import java.io.IOException;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
-import org.apache.carbondata.core.scan.result.impl.NonFilterQueryScannedResult;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-
-/**
- * Blocklet scanner class to process the block
- */
-public abstract class AbstractBlockletScanner implements BlockletScanner {
-
-  /**
-   * block execution info
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  public QueryStatisticsModel queryStatisticsModel;
-
-  private AbstractScannedResult emptyResult;
-
-  public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) {
-    this.blockExecutionInfo = tableBlockExecutionInfos;
-  }
-
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws IOException, FilterUnsupportedException {
-    long startTime = System.currentTimeMillis();
-    AbstractScannedResult scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
-        totalBlockletStatistic.getCount() + 1);
-    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
-    validScannedBlockletStatistic
-        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
-            validScannedBlockletStatistic.getCount() + 1);
-    // adding statistics for valid number of pages
-    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
-    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
-        validPages.getCount() + blocksChunkHolder.getDataBlock().numberOfPages());
-    // adding statistics for number of pages
-    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
-    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
-        totalPagesScanned.getCount() + blocksChunkHolder.getDataBlock().numberOfPages());
-    scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR + blocksChunkHolder
-            .getDataBlock().blockletId());
-    if (!blockExecutionInfo.isPrefetchBlocklet()) {
-      readBlocklet(blocksChunkHolder);
-    }
-    DimensionRawColumnChunk[] dimensionRawColumnChunks =
-        blocksChunkHolder.getDimensionRawDataChunk();
-    DimensionColumnDataChunk[][] dimensionColumnDataChunks =
-        new DimensionColumnDataChunk[dimensionRawColumnChunks.length][blocksChunkHolder
-            .getDataBlock().numberOfPages()];
-    MeasureRawColumnChunk[] measureRawColumnChunks = blocksChunkHolder.getMeasureRawDataChunk();
-    ColumnPage[][] columnPages =
-        new ColumnPage[measureRawColumnChunks.length][blocksChunkHolder.getDataBlock()
-                       .numberOfPages()];
-    scannedResult.setDimensionChunks(dimensionColumnDataChunks);
-    scannedResult.setMeasureChunks(columnPages);
-    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
-    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
-    if (blockExecutionInfo.isPrefetchBlocklet()) {
-      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-        if (dimensionRawColumnChunks[i] != null) {
-          dimensionColumnDataChunks[i] = dimensionRawColumnChunks[i].convertToDimColDataChunks();
-        }
-      }
-      for (int i = 0; i < measureRawColumnChunks.length; i++) {
-        if (measureRawColumnChunks[i] != null) {
-          columnPages[i] = measureRawColumnChunks[i].convertToColumnPage();
-        }
-      }
-    }
-    int[] numberOfRows = null;
-    if (blockExecutionInfo.getAllSelectedDimensionBlocksIndexes().length > 0) {
-      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-        if (dimensionRawColumnChunks[i] != null) {
-          numberOfRows = dimensionRawColumnChunks[i].getRowCount();
-          break;
-        }
-      }
-    } else if (blockExecutionInfo.getAllSelectedMeasureBlocksIndexes().length > 0) {
-      for (int i = 0; i < measureRawColumnChunks.length; i++) {
-        if (measureRawColumnChunks[i] != null) {
-          numberOfRows = measureRawColumnChunks[i].getRowCount();
-          break;
-        }
-      }
-    }
-
-    // count(*)  case there would not be any dimensions are measures selected.
-    if (numberOfRows == null) {
-      numberOfRows = new int[blocksChunkHolder.getDataBlock().numberOfPages()];
-      for (int i = 0; i < numberOfRows.length; i++) {
-        numberOfRows[i] =
-            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      }
-      int lastPageSize = blocksChunkHolder.getDataBlock().nodeSize()
-          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      ;
-      if (lastPageSize > 0) {
-        numberOfRows[numberOfRows.length - 1] = lastPageSize;
-      }
-    }
-    scannedResult.setNumberOfRows(numberOfRows);
-    if (!blockExecutionInfo.isPrefetchBlocklet()) {
-      scannedResult.fillDataChunks();
-    }
-    // adding statistics for carbon scan time
-    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
-    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
-        scanTime.getCount() + (System.currentTimeMillis() - startTime));
-    return scannedResult;
-  }
-
-  @Override public void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    long startTime = System.currentTimeMillis();
-    DimensionRawColumnChunk[] dimensionRawColumnChunks = blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedDimensionBlocksIndexes());
-    blocksChunkHolder.setDimensionRawDataChunk(dimensionRawColumnChunks);
-    MeasureRawColumnChunk[] measureRawColumnChunks = blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedMeasureBlocksIndexes());
-    blocksChunkHolder.setMeasureRawDataChunk(measureRawColumnChunks);
-    // adding statistics for carbon read time
-    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
-    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
-        readTime.getCount() + (System.currentTimeMillis() - startTime));
-  }
-
-  @Override public AbstractScannedResult createEmptyResult() {
-    if (emptyResult == null) {
-      emptyResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-      emptyResult.setNumberOfRows(new int[0]);
-      emptyResult.setIndexes(new int[0][]);
-    }
-    return emptyResult;
-  }
-
-  @Override public boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    // For non filter it is always true
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
index 0ed0d43..0a41032 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
@@ -18,9 +18,10 @@ package org.apache.carbondata.core.scan.scanner;
 
 import java.io.IOException;
 
+import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 
 /**
  * Interface for processing the block
@@ -30,31 +31,26 @@ public interface BlockletScanner {
 
   /**
    * Checks whether this blocklet required to scan or not based on min max of each blocklet.
-   * @param blocksChunkHolder
+   * @param dataBlock
    * @return
    * @throws IOException
    */
-  boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException;
+  boolean isScanRequired(DataRefNode dataBlock);
 
   /**
    * Below method will used to process the block data and get the scanned result
    *
-   * @param blocksChunkHolder block chunk which holds the block data
+   * @param rawBlockletColumnChunks block chunk which holds the block data
    * @return scannerResult
    * result after processing
    */
-  AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
+  BlockletScannedResult scanBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks)
       throws IOException, FilterUnsupportedException;
 
   /**
    * Just reads the blocklet from file, does not uncompress it.
-   * @param blocksChunkHolder
+   * @param rawBlockletColumnChunks
    */
-  void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException;
+  void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException;
 
-  /**
-   * In case if there is no filter satisfies.
-   * @return AbstractScannedResult
-   */
-  AbstractScannedResult createEmptyResult();
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
new file mode 100644
index 0000000..1c73d63
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
@@ -0,0 +1,329 @@
+/*
+ * 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.scan.scanner.impl;
+
+import java.io.IOException;
+import java.util.BitSet;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.impl.FilterQueryScannedResult;
+import org.apache.carbondata.core.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+import org.apache.carbondata.core.util.BitSetGroup;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+/**
+ * Below class will be used for filter query processing
+ * this class will be first apply the filter then it will read the column page if
+ * required and return the scanned result
+ */
+public class BlockletFilterScanner extends BlockletFullScanner {
+
+  /**
+   * filter executer to evaluate filter condition
+   */
+  private FilterExecuter filterExecuter;
+  /**
+   * this will be used to apply min max
+   * this will be useful for dimension column which is on the right side
+   * as node finder will always give tentative blocks, if column data stored individually
+   * and data is in sorted order then we can check whether filter is in the range of min max or not
+   * if it present then only we can apply filter on complete data.
+   * this will be very useful in case of sparse data when rows are
+   * repeating.
+   */
+  private boolean isMinMaxEnabled;
+
+  private QueryStatisticsModel queryStatisticsModel;
+
+  private boolean useBitSetPipeLine;
+
+  public BlockletFilterScanner(BlockExecutionInfo blockExecutionInfo,
+      QueryStatisticsModel queryStatisticsModel) {
+    super(blockExecutionInfo, queryStatisticsModel);
+    // to check whether min max is enabled or not
+    String minMaxEnableValue = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
+            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
+    if (null != minMaxEnableValue) {
+      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
+    }
+    // get the filter tree
+    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
+    this.queryStatisticsModel = queryStatisticsModel;
+
+    String useBitSetPipeLine = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.BITSET_PIPE_LINE,
+            CarbonCommonConstants.BITSET_PIPE_LINE_DEFAULT);
+    if (null != useBitSetPipeLine) {
+      this.useBitSetPipeLine = Boolean.parseBoolean(useBitSetPipeLine);
+    }
+  }
+
+  /**
+   * Below method will be used to process the block
+   *
+   * @param rawBlockletColumnChunks block chunk holder which holds the data
+   * @throws FilterUnsupportedException
+   */
+  @Override
+  public BlockletScannedResult scanBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException, FilterUnsupportedException {
+    return executeFilter(rawBlockletColumnChunks);
+  }
+
+  @Override
+  public boolean isScanRequired(DataRefNode dataBlock) {
+    // adding statistics for number of pages
+    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
+    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
+        totalPagesScanned.getCount() + dataBlock.numberOfPages());
+    // apply min max
+    if (isMinMaxEnabled) {
+      BitSet bitSet = null;
+      // check for implicit include filter instance
+      if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
+        String blockletId = blockExecutionInfo.getBlockIdString() +
+            CarbonCommonConstants.FILE_SEPARATOR + dataBlock.blockletIndex();
+        bitSet = ((ImplicitColumnFilterExecutor) filterExecuter)
+            .isFilterValuesPresentInBlockOrBlocklet(
+                dataBlock.getColumnsMaxValue(),
+                dataBlock.getColumnsMinValue(), blockletId);
+      } else {
+        bitSet = this.filterExecuter
+            .isScanRequired(dataBlock.getColumnsMaxValue(),
+                dataBlock.getColumnsMinValue());
+      }
+      return !bitSet.isEmpty();
+    }
+    return true;
+  }
+
+  @Override
+  public void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    long startTime = System.currentTimeMillis();
+    this.filterExecuter.readColumnChunks(rawBlockletColumnChunks);
+    // adding statistics for carbon read time
+    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
+    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
+        readTime.getCount() + (System.currentTimeMillis() - startTime));
+  }
+
+  /**
+   * This method will process the data in below order
+   * 1. first apply min max on the filter tree and check whether any of the filter
+   * is fall on the range of min max, if not then return empty result
+   * 2. If filter falls on min max range then apply filter on actual
+   * data and get the filtered row index
+   * 3. if row index is empty then return the empty result
+   * 4. if row indexes is not empty then read only those blocks(measure or dimension)
+   * which was present in the query but not present in the filter, as while applying filter
+   * some of the blocks where already read and present in chunk holder so not need to
+   * read those blocks again, this is to avoid reading of same blocks which was already read
+   * 5. Set the blocks and filter indexes to result
+   *
+   * @param rawBlockletColumnChunks
+   * @throws FilterUnsupportedException
+   */
+  private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws FilterUnsupportedException, IOException {
+    long startTime = System.currentTimeMillis();
+    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
+    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
+        totalBlockletStatistic.getCount() + 1);
+    // apply filter on actual data, for each page
+    BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks,
+        useBitSetPipeLine);
+    // if filter result is empty then return with empty result
+    if (bitSetGroup.isEmpty()) {
+      CarbonUtil.freeMemory(rawBlockletColumnChunks.getDimensionRawColumnChunks(),
+          rawBlockletColumnChunks.getMeasureRawColumnChunks());
+
+      QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+          .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
+      scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
+          scanTime.getCount() + (System.currentTimeMillis() - startTime));
+
+      QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+          .get(QueryStatisticsConstants.PAGE_SCANNED);
+      scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
+          scannedPages.getCount() + bitSetGroup.getScannedPages());
+      return createEmptyResult();
+    }
+
+    BlockletScannedResult scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
+    scannedResult.setBlockletId(
+        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR +
+            rawBlockletColumnChunks.getDataBlock().blockletIndex());
+    // valid scanned blocklet
+    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
+    validScannedBlockletStatistic
+        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
+            validScannedBlockletStatistic.getCount() + 1);
+    // adding statistics for valid number of pages
+    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
+    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
+        validPages.getCount() + bitSetGroup.getValidPages());
+    QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.PAGE_SCANNED);
+    scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
+        scannedPages.getCount() + bitSetGroup.getScannedPages());
+    int[] pageFilteredRowCount = new int[bitSetGroup.getNumberOfPages()];
+    // get the row indexes from bit set for each page
+    int[][] pageFilteredRowId = new int[bitSetGroup.getNumberOfPages()][];
+    int numPages = pageFilteredRowId.length;
+    for (int pageId = 0; pageId < numPages; pageId++) {
+      BitSet bitSet = bitSetGroup.getBitSet(pageId);
+      if (bitSet != null && !bitSet.isEmpty()) {
+        int[] matchedRowId = new int[bitSet.cardinality()];
+        int index = 0;
+        for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
+          matchedRowId[index++] = i;
+        }
+        pageFilteredRowCount[pageId] = matchedRowId.length;
+        pageFilteredRowId[pageId] = matchedRowId;
+      }
+    }
+
+    long dimensionReadTime = System.currentTimeMillis();
+    dimensionReadTime = System.currentTimeMillis() - dimensionReadTime;
+
+    FileReader fileReader = rawBlockletColumnChunks.getFileReader();
+
+
+    DimensionRawColumnChunk[] dimensionRawColumnChunks =
+        new DimensionRawColumnChunk[blockExecutionInfo.getTotalNumberDimensionToRead()];
+    int numDimensionChunks = dimensionRawColumnChunks.length;
+    // read dimension chunk blocks from file which is not present
+    for (int chunkIndex = 0; chunkIndex < numDimensionChunks; chunkIndex++) {
+      dimensionRawColumnChunks[chunkIndex] =
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
+    }
+    int[][] allSelectedDimensionColumnIndexRange =
+        blockExecutionInfo.getAllSelectedDimensionColumnIndexRange();
+    DimensionRawColumnChunk[] projectionListDimensionChunk = rawBlockletColumnChunks.getDataBlock()
+        .readDimensionChunks(fileReader, allSelectedDimensionColumnIndexRange);
+    for (int[] columnIndexRange : allSelectedDimensionColumnIndexRange) {
+      System.arraycopy(projectionListDimensionChunk, columnIndexRange[0],
+          dimensionRawColumnChunks, columnIndexRange[0],
+          columnIndexRange[1] + 1 - columnIndexRange[0]);
+    }
+
+    /*
+     * in case projection if the projected dimension are not loaded in the dimensionColumnDataChunk
+     * then loading them
+     */
+    int[] projectionListDimensionIndexes = blockExecutionInfo.getProjectionListDimensionIndexes();
+    for (int projectionListDimensionIndex : projectionListDimensionIndexes) {
+      if (null == dimensionRawColumnChunks[projectionListDimensionIndex]) {
+        dimensionRawColumnChunks[projectionListDimensionIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                fileReader, projectionListDimensionIndex);
+      }
+    }
+
+    DimensionColumnPage[][] dimensionColumnPages =
+        new DimensionColumnPage[numDimensionChunks][numPages];
+    for (int chunkIndex = 0; chunkIndex < numDimensionChunks; chunkIndex++) {
+      if (dimensionRawColumnChunks[chunkIndex] != null) {
+        for (int pageId = 0; pageId < numPages; pageId++) {
+          dimensionColumnPages[chunkIndex][pageId] =
+              dimensionRawColumnChunks[chunkIndex].decodeColumnPage(pageId);
+        }
+      }
+    }
+
+
+    MeasureRawColumnChunk[] measureRawColumnChunks =
+        new MeasureRawColumnChunk[blockExecutionInfo.getTotalNumberOfMeasureToRead()];
+    int numMeasureChunks = measureRawColumnChunks.length;
+
+    // read the measure chunk blocks which is not present
+    for (int chunkIndex = 0; chunkIndex < numMeasureChunks; chunkIndex++) {
+      if (null != rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        measureRawColumnChunks[chunkIndex] =
+            rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
+      }
+    }
+
+    int[][] allSelectedMeasureColumnIndexRange =
+        blockExecutionInfo.getAllSelectedMeasureIndexRange();
+    MeasureRawColumnChunk[] projectionListMeasureChunk = rawBlockletColumnChunks.getDataBlock()
+        .readMeasureChunks(fileReader, allSelectedMeasureColumnIndexRange);
+    for (int[] columnIndexRange : allSelectedMeasureColumnIndexRange) {
+      System.arraycopy(projectionListMeasureChunk, columnIndexRange[0], measureRawColumnChunks,
+          columnIndexRange[0], columnIndexRange[1] + 1 - columnIndexRange[0]);
+    }
+    /*
+     * in case projection if the projected measure are not loaded in the ColumnPage
+     * then loading them
+     */
+    int[] projectionListMeasureIndexes = blockExecutionInfo.getProjectionListMeasureIndexes();
+    for (int projectionListMeasureIndex : projectionListMeasureIndexes) {
+      if (null == measureRawColumnChunks[projectionListMeasureIndex]) {
+        measureRawColumnChunks[projectionListMeasureIndex] = rawBlockletColumnChunks.getDataBlock()
+            .readMeasureChunk(fileReader, projectionListMeasureIndex);
+      }
+    }
+    ColumnPage[][] measureColumnPages = new ColumnPage[numMeasureChunks][numPages];
+    for (int chunkIndex = 0; chunkIndex < numMeasureChunks; chunkIndex++) {
+      if (measureRawColumnChunks[chunkIndex] != null) {
+        for (int pageId = 0; pageId < numPages; pageId++) {
+          measureColumnPages[chunkIndex][pageId] =
+              measureRawColumnChunks[chunkIndex].decodeColumnPage(pageId);
+        }
+      }
+    }
+
+    scannedResult.setDimensionColumnPages(dimensionColumnPages);
+    scannedResult.setPageFilteredRowId(pageFilteredRowId);
+    scannedResult.setMeasureColumnPages(measureColumnPages);
+    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
+    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
+    scannedResult.setPageFilteredRowCount(pageFilteredRowCount);
+    // adding statistics for carbon scan time
+    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
+    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
+        scanTime.getCount() + (System.currentTimeMillis() - startTime - dimensionReadTime));
+    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
+    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
+        readTime.getCount() + dimensionReadTime);
+    return scannedResult;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
new file mode 100644
index 0000000..f0211dc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
@@ -0,0 +1,191 @@
+/*
+ * 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.scan.scanner.impl;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.impl.NonFilterQueryScannedResult;
+import org.apache.carbondata.core.scan.scanner.BlockletScanner;
+import org.apache.carbondata.core.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+
+/**
+ * Blocklet scanner to do full scan of a blocklet,
+ * returning all projection and filter column chunks
+ */
+public class BlockletFullScanner implements BlockletScanner {
+
+  /**
+   * block execution info
+   */
+  protected BlockExecutionInfo blockExecutionInfo;
+
+  private QueryStatisticsModel queryStatisticsModel;
+
+  private BlockletScannedResult emptyResult;
+
+  public BlockletFullScanner(BlockExecutionInfo tableBlockExecutionInfos,
+      QueryStatisticsModel queryStatisticsModel) {
+    this.blockExecutionInfo = tableBlockExecutionInfos;
+    this.queryStatisticsModel = queryStatisticsModel;
+  }
+
+  @Override
+  public BlockletScannedResult scanBlocklet(
+      RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException, FilterUnsupportedException {
+    long startTime = System.currentTimeMillis();
+    BlockletScannedResult scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
+    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
+    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
+        totalBlockletStatistic.getCount() + 1);
+    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
+    validScannedBlockletStatistic
+        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
+            validScannedBlockletStatistic.getCount() + 1);
+    // adding statistics for valid number of pages
+    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
+    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
+        validPages.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages());
+    // adding statistics for number of pages
+    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
+    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
+        totalPagesScanned.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages());
+    scannedResult.setBlockletId(
+        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR +
+            rawBlockletColumnChunks.getDataBlock().blockletIndex());
+    if (!blockExecutionInfo.isPrefetchBlocklet()) {
+      readBlocklet(rawBlockletColumnChunks);
+    }
+    DimensionRawColumnChunk[] dimensionRawColumnChunks =
+        rawBlockletColumnChunks.getDimensionRawColumnChunks();
+    DimensionColumnPage[][] dimensionColumnDataChunks =
+        new DimensionColumnPage[dimensionRawColumnChunks.length][rawBlockletColumnChunks
+            .getDataBlock().numberOfPages()];
+    MeasureRawColumnChunk[] measureRawColumnChunks =
+        rawBlockletColumnChunks.getMeasureRawColumnChunks();
+    ColumnPage[][] measureColumnPages =
+        new ColumnPage[measureRawColumnChunks.length][rawBlockletColumnChunks.getDataBlock()
+                       .numberOfPages()];
+    scannedResult.setDimensionColumnPages(dimensionColumnDataChunks);
+    scannedResult.setMeasureColumnPages(measureColumnPages);
+    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
+    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
+    if (blockExecutionInfo.isPrefetchBlocklet()) {
+      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
+        if (dimensionRawColumnChunks[i] != null) {
+          dimensionColumnDataChunks[i] = dimensionRawColumnChunks[i].decodeAllColumnPages();
+        }
+      }
+      for (int i = 0; i < measureRawColumnChunks.length; i++) {
+        if (measureRawColumnChunks[i] != null) {
+          measureColumnPages[i] = measureRawColumnChunks[i].decodeAllColumnPages();
+        }
+      }
+    }
+    int[] numberOfRows = null;
+    if (blockExecutionInfo.getAllSelectedDimensionColumnIndexRange().length > 0) {
+      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
+        if (dimensionRawColumnChunks[i] != null) {
+          numberOfRows = dimensionRawColumnChunks[i].getRowCount();
+          break;
+        }
+      }
+    } else if (blockExecutionInfo.getAllSelectedMeasureIndexRange().length > 0) {
+      for (int i = 0; i < measureRawColumnChunks.length; i++) {
+        if (measureRawColumnChunks[i] != null) {
+          numberOfRows = measureRawColumnChunks[i].getRowCount();
+          break;
+        }
+      }
+    }
+
+    // count(*)  case there would not be any dimensions are measures selected.
+    if (numberOfRows == null) {
+      numberOfRows = new int[rawBlockletColumnChunks.getDataBlock().numberOfPages()];
+      for (int i = 0; i < numberOfRows.length; i++) {
+        numberOfRows[i] =
+            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      }
+      int lastPageSize = rawBlockletColumnChunks.getDataBlock().numRows()
+          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      ;
+      if (lastPageSize > 0) {
+        numberOfRows[numberOfRows.length - 1] = lastPageSize;
+      }
+    }
+    scannedResult.setPageFilteredRowCount(numberOfRows);
+    if (!blockExecutionInfo.isPrefetchBlocklet()) {
+      scannedResult.fillDataChunks();
+    }
+    // adding statistics for carbon scan time
+    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
+    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
+        scanTime.getCount() + (System.currentTimeMillis() - startTime));
+    return scannedResult;
+  }
+
+  @Override
+  public void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    DimensionRawColumnChunk[] dimensionRawColumnChunks = rawBlockletColumnChunks.getDataBlock()
+        .readDimensionChunks(rawBlockletColumnChunks.getFileReader(),
+            blockExecutionInfo.getAllSelectedDimensionColumnIndexRange());
+    rawBlockletColumnChunks.setDimensionRawColumnChunks(dimensionRawColumnChunks);
+    MeasureRawColumnChunk[] measureRawColumnChunks = rawBlockletColumnChunks.getDataBlock()
+        .readMeasureChunks(rawBlockletColumnChunks.getFileReader(),
+            blockExecutionInfo.getAllSelectedMeasureIndexRange());
+    rawBlockletColumnChunks.setMeasureRawColumnChunks(measureRawColumnChunks);
+    // adding statistics for carbon read time
+    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
+    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
+        readTime.getCount() + (System.currentTimeMillis() - startTime));
+  }
+
+  BlockletScannedResult createEmptyResult() {
+    if (emptyResult == null) {
+      emptyResult = new NonFilterQueryScannedResult(blockExecutionInfo);
+      emptyResult.setPageFilteredRowCount(new int[0]);
+      emptyResult.setPageFilteredRowId(new int[0][]);
+    }
+    return emptyResult;
+  }
+
+  @Override public boolean isScanRequired(DataRefNode dataBlock) {
+    // For non filter it is always true
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java
deleted file mode 100644
index e77093b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java
+++ /dev/null
@@ -1,326 +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.scan.scanner.impl;
-
-import java.io.IOException;
-import java.util.BitSet;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
-import org.apache.carbondata.core.scan.result.impl.FilterQueryScannedResult;
-import org.apache.carbondata.core.scan.scanner.AbstractBlockletScanner;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-import org.apache.carbondata.core.util.BitSetGroup;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-/**
- * Below class will be used for filter query processing
- * this class will be first apply the filter then it will read the block if
- * required and return the scanned result
- */
-public class FilterScanner extends AbstractBlockletScanner {
-
-  /**
-   * filter tree
-   */
-  private FilterExecuter filterExecuter;
-  /**
-   * this will be used to apply min max
-   * this will be useful for dimension column which is on the right side
-   * as node finder will always give tentative blocks, if column data stored individually
-   * and data is in sorted order then we can check whether filter is in the range of min max or not
-   * if it present then only we can apply filter on complete data.
-   * this will be very useful in case of sparse data when rows are
-   * repeating.
-   */
-  private boolean isMinMaxEnabled;
-
-  private QueryStatisticsModel queryStatisticsModel;
-
-  private boolean useBitSetPipeLine;
-
-  public FilterScanner(BlockExecutionInfo blockExecutionInfo,
-      QueryStatisticsModel queryStatisticsModel) {
-    super(blockExecutionInfo);
-    // to check whether min max is enabled or not
-    String minMaxEnableValue = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
-            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
-    if (null != minMaxEnableValue) {
-      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
-    }
-    // get the filter tree
-    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
-    this.queryStatisticsModel = queryStatisticsModel;
-
-    String useBitSetPipeLine = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BITSET_PIPE_LINE,
-            CarbonCommonConstants.BITSET_PIPE_LINE_DEFAULT);
-    if (null != useBitSetPipeLine) {
-      this.useBitSetPipeLine = Boolean.parseBoolean(useBitSetPipeLine);
-    }
-  }
-
-  /**
-   * Below method will be used to process the block
-   *
-   * @param blocksChunkHolder block chunk holder which holds the data
-   * @throws FilterUnsupportedException
-   */
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws IOException, FilterUnsupportedException {
-    return fillScannedResult(blocksChunkHolder);
-  }
-
-  @Override public boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    // adding statistics for number of pages
-    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
-    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
-        totalPagesScanned.getCount() + blocksChunkHolder.getDataBlock().numberOfPages());
-    // apply min max
-    if (isMinMaxEnabled) {
-      BitSet bitSet = null;
-      // check for implicit include filter instance
-      if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-        String blockletId = blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR
-            + blocksChunkHolder.getDataBlock().blockletId();
-        bitSet = ((ImplicitColumnFilterExecutor) filterExecuter)
-            .isFilterValuesPresentInBlockOrBlocklet(
-                blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
-                blocksChunkHolder.getDataBlock().getColumnsMinValue(), blockletId);
-      } else {
-        bitSet = this.filterExecuter
-            .isScanRequired(blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
-                blocksChunkHolder.getDataBlock().getColumnsMinValue());
-      }
-      if (bitSet.isEmpty()) {
-        CarbonUtil.freeMemory(blocksChunkHolder.getDimensionRawDataChunk(),
-            blocksChunkHolder.getMeasureRawDataChunk());
-        return false;
-      }
-    }
-    return true;
-  }
-
-  @Override public void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    long startTime = System.currentTimeMillis();
-    this.filterExecuter.readBlocks(blocksChunkHolder);
-    // adding statistics for carbon read time
-    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
-    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
-        readTime.getCount() + (System.currentTimeMillis() - startTime));
-  }
-
-  /**
-   * This method will process the data in below order
-   * 1. first apply min max on the filter tree and check whether any of the filter
-   * is fall on the range of min max, if not then return empty result
-   * 2. If filter falls on min max range then apply filter on actual
-   * data and get the filtered row index
-   * 3. if row index is empty then return the empty result
-   * 4. if row indexes is not empty then read only those blocks(measure or dimension)
-   * which was present in the query but not present in the filter, as while applying filter
-   * some of the blocks where already read and present in chunk holder so not need to
-   * read those blocks again, this is to avoid reading of same blocks which was already read
-   * 5. Set the blocks and filter indexes to result
-   *
-   * @param blocksChunkHolder
-   * @throws FilterUnsupportedException
-   */
-  private AbstractScannedResult fillScannedResult(BlocksChunkHolder blocksChunkHolder)
-      throws FilterUnsupportedException, IOException {
-    long startTime = System.currentTimeMillis();
-    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
-        totalBlockletStatistic.getCount() + 1);
-    // apply filter on actual data
-    BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(blocksChunkHolder, useBitSetPipeLine);
-    // if indexes is empty then return with empty result
-    if (bitSetGroup.isEmpty()) {
-      CarbonUtil.freeMemory(blocksChunkHolder.getDimensionRawDataChunk(),
-          blocksChunkHolder.getMeasureRawDataChunk());
-
-      QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-          .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
-      scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
-          scanTime.getCount() + (System.currentTimeMillis() - startTime));
-
-      QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-          .get(QueryStatisticsConstants.PAGE_SCANNED);
-      scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
-          scannedPages.getCount() + bitSetGroup.getScannedPages());
-      return createEmptyResult();
-    }
-
-    AbstractScannedResult scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
-    scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR + blocksChunkHolder
-            .getDataBlock().blockletId());
-    // valid scanned blocklet
-    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
-    validScannedBlockletStatistic
-        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
-            validScannedBlockletStatistic.getCount() + 1);
-    // adding statistics for valid number of pages
-    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
-    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
-        validPages.getCount() + bitSetGroup.getValidPages());
-    QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.PAGE_SCANNED);
-    scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
-        scannedPages.getCount() + bitSetGroup.getScannedPages());
-    int[] rowCount = new int[bitSetGroup.getNumberOfPages()];
-    // get the row indexes from bot set
-    int[][] indexesGroup = new int[bitSetGroup.getNumberOfPages()][];
-    for (int k = 0; k < indexesGroup.length; k++) {
-      BitSet bitSet = bitSetGroup.getBitSet(k);
-      if (bitSet != null && !bitSet.isEmpty()) {
-        int[] indexes = new int[bitSet.cardinality()];
-        int index = 0;
-        for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-          indexes[index++] = i;
-        }
-        rowCount[k] = indexes.length;
-        indexesGroup[k] = indexes;
-      }
-    }
-    FileHolder fileReader = blocksChunkHolder.getFileReader();
-    int[][] allSelectedDimensionBlocksIndexes =
-        blockExecutionInfo.getAllSelectedDimensionBlocksIndexes();
-    long dimensionReadTime = System.currentTimeMillis();
-    DimensionRawColumnChunk[] projectionListDimensionChunk = blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(fileReader, allSelectedDimensionBlocksIndexes);
-    dimensionReadTime = System.currentTimeMillis() - dimensionReadTime;
-
-    DimensionRawColumnChunk[] dimensionRawColumnChunks =
-        new DimensionRawColumnChunk[blockExecutionInfo.getTotalNumberDimensionBlock()];
-    // read dimension chunk blocks from file which is not present
-    for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-      if (null != blocksChunkHolder.getDimensionRawDataChunk()[i]) {
-        dimensionRawColumnChunks[i] = blocksChunkHolder.getDimensionRawDataChunk()[i];
-      }
-    }
-    for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) {
-      for (int j = allSelectedDimensionBlocksIndexes[i][0];
-           j <= allSelectedDimensionBlocksIndexes[i][1]; j++) {
-        dimensionRawColumnChunks[j] = projectionListDimensionChunk[j];
-      }
-    }
-    long dimensionReadTime1 = System.currentTimeMillis();
-    /**
-     * in case projection if the projected dimension are not loaded in the dimensionColumnDataChunk
-     * then loading them
-     */
-    int[] projectionListDimensionIndexes = blockExecutionInfo.getProjectionListDimensionIndexes();
-    int projectionListDimensionIndexesLength = projectionListDimensionIndexes.length;
-    for (int i = 0; i < projectionListDimensionIndexesLength; i++) {
-      if (null == dimensionRawColumnChunks[projectionListDimensionIndexes[i]]) {
-        dimensionRawColumnChunks[projectionListDimensionIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getDimensionChunk(fileReader, projectionListDimensionIndexes[i]);
-      }
-    }
-    dimensionReadTime += (System.currentTimeMillis() - dimensionReadTime1);
-    dimensionReadTime1 = System.currentTimeMillis();
-    MeasureRawColumnChunk[] measureRawColumnChunks =
-        new MeasureRawColumnChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()];
-    int[][] allSelectedMeasureBlocksIndexes =
-        blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
-    MeasureRawColumnChunk[] projectionListMeasureChunk = blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(fileReader, allSelectedMeasureBlocksIndexes);
-    dimensionReadTime += System.currentTimeMillis() - dimensionReadTime1;
-    // read the measure chunk blocks which is not present
-    for (int i = 0; i < measureRawColumnChunks.length; i++) {
-      if (null != blocksChunkHolder.getMeasureRawDataChunk()[i]) {
-        measureRawColumnChunks[i] = blocksChunkHolder.getMeasureRawDataChunk()[i];
-      }
-    }
-    for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) {
-      for (int j = allSelectedMeasureBlocksIndexes[i][0];
-           j <= allSelectedMeasureBlocksIndexes[i][1]; j++) {
-        measureRawColumnChunks[j] = projectionListMeasureChunk[j];
-      }
-    }
-    dimensionReadTime1 = System.currentTimeMillis();
-    /**
-     * in case projection if the projected measure are not loaded in the ColumnPage
-     * then loading them
-     */
-    int[] projectionListMeasureIndexes = blockExecutionInfo.getProjectionListMeasureIndexes();
-    int projectionListMeasureIndexesLength = projectionListMeasureIndexes.length;
-    for (int i = 0; i < projectionListMeasureIndexesLength; i++) {
-      if (null == measureRawColumnChunks[projectionListMeasureIndexes[i]]) {
-        measureRawColumnChunks[projectionListMeasureIndexes[i]] = blocksChunkHolder.getDataBlock()
-            .getMeasureChunk(fileReader, projectionListMeasureIndexes[i]);
-      }
-    }
-    dimensionReadTime += System.currentTimeMillis() - dimensionReadTime1;
-    DimensionColumnDataChunk[][] dimensionColumnDataChunks =
-        new DimensionColumnDataChunk[dimensionRawColumnChunks.length][indexesGroup.length];
-    ColumnPage[][] columnPages =
-        new ColumnPage[measureRawColumnChunks.length][indexesGroup.length];
-    for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-      if (dimensionRawColumnChunks[i] != null) {
-        for (int j = 0; j < indexesGroup.length; j++) {
-          dimensionColumnDataChunks[i][j] = dimensionRawColumnChunks[i].convertToDimColDataChunk(j);
-        }
-      }
-    }
-    for (int i = 0; i < measureRawColumnChunks.length; i++) {
-      if (measureRawColumnChunks[i] != null) {
-        for (int j = 0; j < indexesGroup.length; j++) {
-          columnPages[i][j] = measureRawColumnChunks[i].convertToColumnPage(j);
-        }
-      }
-    }
-    scannedResult.setDimensionChunks(dimensionColumnDataChunks);
-    scannedResult.setIndexes(indexesGroup);
-    scannedResult.setMeasureChunks(columnPages);
-    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
-    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
-    scannedResult.setNumberOfRows(rowCount);
-    // adding statistics for carbon scan time
-    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
-    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
-        scanTime.getCount() + (System.currentTimeMillis() - startTime - dimensionReadTime));
-    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
-    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
-        readTime.getCount() + dimensionReadTime);
-    return scannedResult;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java
deleted file mode 100644
index 1373ed5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java
+++ /dev/null
@@ -1,35 +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.scan.scanner.impl;
-
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.scanner.AbstractBlockletScanner;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-
-/**
- * Non filter processor which will be used for non filter query
- * In case of non filter query we just need to read all the blocks requested in the
- * query and pass it to scanned result
- */
-public class NonFilterScanner extends AbstractBlockletScanner {
-
-  public NonFilterScanner(BlockExecutionInfo blockExecutionInfo,
-                          QueryStatisticsModel queryStatisticsModel) {
-    super(blockExecutionInfo);
-    super.queryStatisticsModel = queryStatisticsModel;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
index 2f981b5..6faae03 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
@@ -30,22 +30,17 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper>, Serializa
    * to store key which is generated using
    * key generator
    */
-  protected byte[] dictionaryKey;
+  private byte[] dictionaryKey;
 
   /**
    * to store no dictionary column data
    */
-  protected byte[][] complexTypesKeys;
+  private byte[][] complexTypesKeys;
 
   /**
    * to store no dictionary column data
    */
-  protected byte[][] noDictionaryKeys;
-
-  /**
-   * contains value of implicit columns in byte array format
-   */
-  protected byte[] implicitColumnByteArray;
+  private byte[][] noDictionaryKeys;
 
   public ByteArrayWrapper() {
   }
@@ -91,16 +86,6 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper>, Serializa
   }
 
   /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getComplexTypeByIndex(int index) {
-    return this.complexTypesKeys[index];
-  }
-
-  /**
    * to generate the hash code
    */
   @Override public int hashCode() {
@@ -201,30 +186,10 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper>, Serializa
   }
 
   /**
-   * @return the complexTypesKeys
-   */
-  public byte[][] getComplexTypesKeys() {
-    return complexTypesKeys;
-  }
-
-  /**
    * @param complexTypesKeys the complexTypesKeys to set
    */
   public void setComplexTypesKeys(byte[][] complexTypesKeys) {
     this.complexTypesKeys = complexTypesKeys;
   }
 
-  /**
-   * @return
-   */
-  public byte[] getImplicitColumnByteArray() {
-    return implicitColumnByteArray;
-  }
-
-  /**
-   * @param implicitColumnByteArray
-   */
-  public void setImplicitColumnByteArray(byte[] implicitColumnByteArray) {
-    this.implicitColumnByteArray = implicitColumnByteArray;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/stats/PartitionStatistic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/stats/PartitionStatistic.java b/core/src/main/java/org/apache/carbondata/core/stats/PartitionStatistic.java
deleted file mode 100644
index 8a37d01..0000000
--- a/core/src/main/java/org/apache/carbondata/core/stats/PartitionStatistic.java
+++ /dev/null
@@ -1,23 +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.stats;
-
-import java.io.Serializable;
-
-public class PartitionStatistic implements Serializable {
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
index 55f0882..ed60d37 100644
--- a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
@@ -37,7 +37,4 @@ public class QueryStatisticsModel {
     return statisticsTypeAndObjMap;
   }
 
-  public void setStatisticsTypeAndObjMap(Map<String, QueryStatistic> statisticsTypeAndObjMap) {
-    this.statisticsTypeAndObjMap = statisticsTypeAndObjMap;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
index 73a665d..d0c8e93 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
@@ -341,15 +341,6 @@ public class LoadMetadataDetails implements Serializable {
   }
 
   /**
-   * To get isDeleted property.
-   *
-   * @return isDeleted
-   */
-  public String getIsDeleted() {
-    return isDeleted;
-  }
-
-  /**
    * To set isDeleted property.
    *
    * @param isDeleted

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 6af0304..e02f246 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -652,22 +652,6 @@ public class SegmentStatusManager {
     return "";
   }
 
-  /**
-   * getting the task numbers present in the segment.
-   * @param segmentId
-   * @return
-   */
-  public List<String> getUpdatedTasksDetailsForSegment(String segmentId, SegmentUpdateStatusManager
-          updateStatusManager) {
-    List<String> taskList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<String> list = updateStatusManager.getUpdateDeltaFiles(segmentId);
-    for (String eachFileName : list) {
-      taskList.add(CarbonTablePath.DataFileUtil.getTaskNo(eachFileName));
-    }
-    return taskList;
-  }
-
-
   public static class ValidAndInvalidSegmentsInfo {
     private final List<String> listOfValidSegments;
     private final List<String> listOfValidUpdatedSegments;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index d4ef5c6..66f7a12 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -47,7 +47,6 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
 import org.apache.carbondata.core.mutate.TupleIdEnum;
 import org.apache.carbondata.core.mutate.UpdateVO;
-import org.apache.carbondata.core.reader.CarbonDeleteFilesDataReader;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -140,14 +139,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   *
-   * @param loadMetadataDetails
-   */
-  public void setLoadMetadataDetails(LoadMetadataDetails[] loadMetadataDetails) {
-    this.segmentDetails = loadMetadataDetails;
-  }
-
-  /**
    * Returns the UpdateStatus Details.
    * @return
    */
@@ -174,18 +165,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   * Returns all delete delta files of specified block
-   *
-   * @param tupleId
-   * @return
-   * @throws Exception
-   */
-  public List<String> getDeleteDeltaFiles(String tupleId) throws Exception {
-    return getDeltaFiles(tupleId, CarbonCommonConstants.DELETE_DELTA_FILE_EXT);
-  }
-
-
-  /**
    * Returns all update delta files of specified Segment.
    *
    * @param segmentId
@@ -248,20 +227,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   * Returns all deleted records of specified block
-   *
-   * @param tupleId
-   * @return
-   * @throws Exception
-   */
-  public Map<Integer, Integer[]> getDeleteDeltaDataFromAllFiles(String tupleId) throws Exception {
-    List<String> deltaFiles = getDeltaFiles(tupleId, CarbonCommonConstants.DELETE_DELTA_FILE_EXT);
-    CarbonDeleteFilesDataReader dataReader = new CarbonDeleteFilesDataReader();
-    String blockletId = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.BLOCKLET_ID);
-    return dataReader.getDeleteDataFromAllFiles(deltaFiles, blockletId);
-  }
-
-  /**
    * Below method will be used to get all the delete delta files based on block name
    *
    * @param blockFilePath actual block filePath
@@ -775,41 +740,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   * compares passed time stamp with status file delete timestamp and
-   * returns latest timestamp from status file if both are not equal
-   * returns null otherwise
-   *
-   * @param completeBlockName
-   * @param timestamp
-   * @return
-   */
-  public String getTimestampForRefreshCache(String completeBlockName, String timestamp) {
-    long cacheTimestamp = 0;
-    if (null != timestamp) {
-      cacheTimestamp = CarbonUpdateUtil.getTimeStampAsLong(timestamp);
-    }
-    String blockName = CarbonTablePath.addDataPartPrefix(CarbonUpdateUtil.getBlockName(
-        CarbonUpdateUtil.getRequiredFieldFromTID(completeBlockName, TupleIdEnum.BLOCK_ID)));
-    String segmentId =
-        CarbonUpdateUtil.getRequiredFieldFromTID(completeBlockName, TupleIdEnum.SEGMENT_ID);
-    SegmentUpdateDetails[] listOfSegmentUpdateDetailsArray =
-        readLoadMetadata();
-    for (SegmentUpdateDetails block : listOfSegmentUpdateDetailsArray) {
-      if (segmentId.equalsIgnoreCase(block.getSegmentName()) &&
-          block.getBlockName().equalsIgnoreCase(blockName) &&
-          !CarbonUpdateUtil.isBlockInvalid(block.getSegmentStatus())) {
-        long deleteTimestampFromStatusFile = block.getDeleteDeltaEndTimeAsLong();
-        if (Long.compare(deleteTimestampFromStatusFile, cacheTimestamp) == 0) {
-          return null;
-        } else {
-          return block.getDeleteDeltaEndTimestamp();
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
    * This method closes the streams
    *
    * @param streams - streams to close.
@@ -828,85 +758,7 @@ public class SegmentUpdateStatusManager {
       }
     }
   }
-  /**
-   * Get the invalid tasks in that segment.
-   * @param segmentId
-   * @return
-   */
-  public List<String> getInvalidBlockList(String segmentId) {
-
-    // get the original fact file timestamp from the table status file.
-    List<String> listOfInvalidBlocks = new ArrayList<String>();
-    SegmentStatusManager ssm = new SegmentStatusManager(absoluteTableIdentifier);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    LoadMetadataDetails[] segmentDetails =
-        ssm.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
-    long timestampOfOriginalFacts = 0;
-
-    String startTimestampOfUpdate = "" ;
-    String endTimestampOfUpdate = "";
-
-    for (LoadMetadataDetails segment : segmentDetails) {
-      // find matching segment and return timestamp.
-      if (segment.getLoadName().equalsIgnoreCase(segmentId)) {
-        timestampOfOriginalFacts = segment.getLoadStartTime();
-        startTimestampOfUpdate = segment.getUpdateDeltaStartTimestamp();
-        endTimestampOfUpdate = segment.getUpdateDeltaEndTimestamp();
-      }
-    }
-
-    if (startTimestampOfUpdate.isEmpty()) {
-      return listOfInvalidBlocks;
-
-    }
-
-    // now after getting the original fact timestamp, what ever is remaining
-    // files need to cross check it with table status file.
-
-    // filter out the fact files.
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
-    CarbonFile segDir =
-        FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
-
-    final Long endTimeStampFinal = CarbonUpdateUtil.getTimeStampAsLong(endTimestampOfUpdate);
-    final Long startTimeStampFinal = CarbonUpdateUtil.getTimeStampAsLong(startTimestampOfUpdate);
-    final Long timeStampOriginalFactFinal =
-        timestampOfOriginalFacts;
-
-    CarbonFile[] files = segDir.listFiles(new CarbonFileFilter() {
-
-      @Override public boolean accept(CarbonFile pathName) {
-        String fileName = pathName.getName();
-        if (fileName.endsWith(CarbonCommonConstants.UPDATE_DELTA_FILE_EXT)) {
-          String firstPart = fileName.substring(0, fileName.indexOf('.'));
-
-          long timestamp = Long.parseLong(firstPart
-              .substring(firstPart.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1,
-                  firstPart.length()));
-          if (Long.compare(timestamp, endTimeStampFinal) <= 0
-              && Long.compare(timestamp, startTimeStampFinal) >= 0) {
-            return false;
-          }
-          if (Long.compare(timestamp, timeStampOriginalFactFinal) == 0) {
-            return false;
-          }
-          // take the rest of files as they are invalid.
-          return true;
-        }
-        return false;
-      }
-    });
 
-    // gather the task numbers.
-    for (CarbonFile updateFiles : files) {
-      listOfInvalidBlocks.add(updateFiles.getName());
-    }
-
-    return listOfInvalidBlocks;
-  }
   /**
    * Returns the invalid timestamp range of a segment.
    * @param segmentId
@@ -932,12 +784,11 @@ public class SegmentUpdateStatusManager {
   }
   /**
    *
-   * @param segmentId
    * @param block
    * @param needCompleteList
    * @return
    */
-  public CarbonFile[] getDeleteDeltaInvalidFilesList(final String segmentId,
+  public CarbonFile[] getDeleteDeltaInvalidFilesList(
       final SegmentUpdateDetails block, final boolean needCompleteList,
       CarbonFile[] allSegmentFiles, boolean isAbortedFile) {
 
@@ -983,12 +834,11 @@ public class SegmentUpdateStatusManager {
 
   /**
    *
-   * @param blockName
    * @param allSegmentFiles
    * @return
    */
-  public CarbonFile[] getAllBlockRelatedFiles(String blockName, CarbonFile[] allSegmentFiles,
-                                              String actualBlockName) {
+  public CarbonFile[] getAllBlockRelatedFiles(CarbonFile[] allSegmentFiles,
+      String actualBlockName) {
     List<CarbonFile> files = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
     for (CarbonFile eachFile : allSegmentFiles) {


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
new file mode 100644
index 0000000..fde4e55
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
@@ -0,0 +1,269 @@
+/*
+ * 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.scan.processor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.scan.collector.ResultCollectorFactory;
+import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.scan.scanner.BlockletScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFilterScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFullScanner;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+import org.apache.carbondata.core.util.TaskMetricsMap;
+
+/**
+ * This abstract class provides a skeletal implementation of the
+ * Block iterator.
+ */
+public class DataBlockIterator extends CarbonIterator<List<Object[]>> {
+
+  /**
+   * iterator which will be used to iterate over blocklets
+   */
+  private BlockletIterator blockletIterator;
+
+  /**
+   * result collector which will be used to aggregate the scanned result
+   */
+  private ScannedResultCollector scannerResultAggregator;
+
+  /**
+   * processor which will be used to process the block processing can be
+   * filter processing or non filter processing
+   */
+  private BlockletScanner blockletScanner;
+
+  /**
+   * batch size of result
+   */
+  private int batchSize;
+
+  private ExecutorService executorService;
+
+  private Future<BlockletScannedResult> future;
+
+  private Future<RawBlockletColumnChunks> futureIo;
+
+  private BlockletScannedResult scannedResult;
+
+  private BlockExecutionInfo blockExecutionInfo;
+
+  private FileReader fileReader;
+
+  private AtomicBoolean nextBlock;
+
+  private AtomicBoolean nextRead;
+
+  public DataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileReader fileReader,
+      int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) {
+    this.blockExecutionInfo = blockExecutionInfo;
+    this.fileReader = fileReader;
+    blockletIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
+        blockExecutionInfo.getNumberOfBlockToScan());
+    if (blockExecutionInfo.getFilterExecuterTree() != null) {
+      blockletScanner = new BlockletFilterScanner(blockExecutionInfo, queryStatisticsModel);
+    } else {
+      blockletScanner = new BlockletFullScanner(blockExecutionInfo, queryStatisticsModel);
+    }
+    this.scannerResultAggregator =
+        ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo);
+    this.batchSize = batchSize;
+    this.executorService = executorService;
+    this.nextBlock = new AtomicBoolean(false);
+    this.nextRead = new AtomicBoolean(false);
+  }
+
+  @Override
+  public List<Object[]> next() {
+    List<Object[]> collectedResult = null;
+    if (updateScanner()) {
+      collectedResult = this.scannerResultAggregator.collectResultInRow(scannedResult, batchSize);
+      while (collectedResult.size() < batchSize && updateScanner()) {
+        List<Object[]> data = this.scannerResultAggregator
+            .collectResultInRow(scannedResult, batchSize - collectedResult.size());
+        collectedResult.addAll(data);
+      }
+    } else {
+      collectedResult = new ArrayList<>();
+    }
+    return collectedResult;
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (scannedResult != null && scannedResult.hasNext()) {
+      return true;
+    } else {
+      if (null != scannedResult) {
+        scannedResult.freeMemory();
+      }
+      return blockletIterator.hasNext() || nextBlock.get() || nextRead.get();
+    }
+  }
+
+  /**
+   * Return true if scan result if non-empty
+   */
+  private boolean updateScanner() {
+    try {
+      if (scannedResult != null && scannedResult.hasNext()) {
+        return true;
+      } else {
+        scannedResult = processNextBlocklet();
+        while (scannedResult != null) {
+          if (scannedResult.hasNext()) {
+            return true;
+          }
+          scannedResult = processNextBlocklet();
+        }
+        nextBlock.set(false);
+        nextRead.set(false);
+        return false;
+      }
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  private BlockletScannedResult processNextBlocklet() throws Exception {
+    BlockletScannedResult result = null;
+    if (blockExecutionInfo.isPrefetchBlocklet()) {
+      if (blockletIterator.hasNext() || nextBlock.get() || nextRead.get()) {
+        if (future == null) {
+          future = scanNextBlockletAsync();
+        }
+        result = future.get();
+        nextBlock.set(false);
+        if (blockletIterator.hasNext() || nextRead.get()) {
+          nextBlock.set(true);
+          future = scanNextBlockletAsync();
+        }
+      }
+    } else {
+      if (blockletIterator.hasNext()) {
+        RawBlockletColumnChunks rawChunks = readNextBlockletColumnChunks();
+        if (rawChunks != null) {
+          result = blockletScanner.scanBlocklet(rawChunks);
+        }
+      }
+    }
+    return result;
+  }
+
+  private RawBlockletColumnChunks readNextBlockletColumnChunks() throws IOException {
+    RawBlockletColumnChunks rawBlockletColumnChunks = getNextBlockletColumnChunks();
+    if (rawBlockletColumnChunks != null) {
+      blockletScanner.readBlocklet(rawBlockletColumnChunks);
+      return rawBlockletColumnChunks;
+    }
+    return null;
+  }
+
+  private RawBlockletColumnChunks getNextBlockletColumnChunks() {
+    RawBlockletColumnChunks rawBlockletColumnChunks = null;
+    do {
+      DataRefNode dataBlock = blockletIterator.next();
+      if (dataBlock.getColumnsMaxValue() == null || blockletScanner.isScanRequired(dataBlock)) {
+        rawBlockletColumnChunks =  RawBlockletColumnChunks.newInstance(
+            blockExecutionInfo.getTotalNumberDimensionToRead(),
+            blockExecutionInfo.getTotalNumberOfMeasureToRead(), fileReader, dataBlock);
+      }
+    } while (rawBlockletColumnChunks == null && blockletIterator.hasNext());
+    return rawBlockletColumnChunks;
+  }
+
+  private Future<BlockletScannedResult> scanNextBlockletAsync() {
+    return executorService.submit(new Callable<BlockletScannedResult>() {
+      @Override public BlockletScannedResult call() throws Exception {
+        if (futureIo == null) {
+          futureIo = readNextBlockletAsync();
+        }
+        RawBlockletColumnChunks rawBlockletColumnChunks = futureIo.get();
+        futureIo = null;
+        nextRead.set(false);
+        if (rawBlockletColumnChunks != null) {
+          if (blockletIterator.hasNext()) {
+            nextRead.set(true);
+            futureIo = readNextBlockletAsync();
+          }
+          return blockletScanner.scanBlocklet(rawBlockletColumnChunks);
+        }
+        return null;
+      }
+    });
+  }
+
+  private Future<RawBlockletColumnChunks> readNextBlockletAsync() {
+    return executorService.submit(new Callable<RawBlockletColumnChunks>() {
+      @Override public RawBlockletColumnChunks call() throws Exception {
+        try {
+          TaskMetricsMap.getInstance().registerThreadCallback();
+          if (blockletIterator.hasNext()) {
+            return readNextBlockletColumnChunks();
+          } else {
+            return null;
+          }
+        } finally {
+          // update read bytes metrics for this thread
+          TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId());
+        }
+      }
+    });
+  }
+
+  public void processNextBatch(CarbonColumnarBatch columnarBatch) {
+    if (updateScanner()) {
+      this.scannerResultAggregator.collectResultInColumnarBatch(scannedResult, columnarBatch);
+    }
+  }
+
+
+  /**
+   * Close the resources
+   */
+  public void close() {
+    // free the current scanned result
+    if (null != scannedResult && !scannedResult.hasNext()) {
+      scannedResult.freeMemory();
+    }
+    // free any pre-fetched memory if present
+    if (null != future) {
+      try {
+        BlockletScannedResult blockletScannedResult = future.get();
+        if (blockletScannedResult != null) {
+          blockletScannedResult.freeMemory();
+        }
+      } catch (InterruptedException | ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java
new file mode 100644
index 0000000..6b7e880
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java
@@ -0,0 +1,113 @@
+/*
+ * 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.scan.processor;
+
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.util.BitSetGroup;
+
+/**
+ * Contains dimension and measure raw column chunks of one blocklet
+ */
+public class RawBlockletColumnChunks {
+
+  /**
+   * dimension column data chunk
+   */
+  private DimensionRawColumnChunk[] dimensionRawColumnChunks;
+
+  /**
+   * measure column data chunk
+   */
+  private MeasureRawColumnChunk[] measureRawColumnChunks;
+
+  /**
+   * file reader which will use to read the block from file
+   */
+  private FileReader fileReader;
+
+  /**
+   * data block
+   */
+  private DataRefNode dataBlock;
+
+  private BitSetGroup bitSetGroup;
+
+  private RawBlockletColumnChunks() { }
+
+  public static RawBlockletColumnChunks newInstance(int numberOfDimensionChunk,
+      int numberOfMeasureChunk, FileReader fileReader, DataRefNode dataBlock) {
+    RawBlockletColumnChunks instance = new RawBlockletColumnChunks();
+    instance.dimensionRawColumnChunks = new DimensionRawColumnChunk[numberOfDimensionChunk];
+    instance.measureRawColumnChunks = new MeasureRawColumnChunk[numberOfMeasureChunk];
+    instance.fileReader = fileReader;
+    instance.dataBlock = dataBlock;
+    return instance;
+  }
+
+  /**
+   * @return the dimensionRawColumnChunks
+   */
+  public DimensionRawColumnChunk[] getDimensionRawColumnChunks() {
+    return dimensionRawColumnChunks;
+  }
+
+  /**
+   * @param dimensionRawColumnChunks the dimensionRawColumnChunks to set
+   */
+  public void setDimensionRawColumnChunks(DimensionRawColumnChunk[] dimensionRawColumnChunks) {
+    this.dimensionRawColumnChunks = dimensionRawColumnChunks;
+  }
+
+  /**
+   * @return the measureRawColumnChunks
+   */
+  public MeasureRawColumnChunk[] getMeasureRawColumnChunks() {
+    return measureRawColumnChunks;
+  }
+
+  /**
+   * @param measureRawColumnChunks the measureRawColumnChunks to set
+   */
+  public void setMeasureRawColumnChunks(MeasureRawColumnChunk[] measureRawColumnChunks) {
+    this.measureRawColumnChunks = measureRawColumnChunks;
+  }
+
+  /**
+   * @return the fileReader
+   */
+  public FileReader getFileReader() {
+    return fileReader;
+  }
+
+  /**
+   * @return the dataBlock
+   */
+  public DataRefNode getDataBlock() {
+    return dataBlock;
+  }
+
+  public BitSetGroup getBitSetGroup() {
+    return bitSetGroup;
+  }
+
+  public void setBitSetGroup(BitSetGroup bitSetGroup) {
+    this.bitSetGroup = bitSetGroup;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java
deleted file mode 100644
index 1c97725..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java
+++ /dev/null
@@ -1,69 +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.scan.processor.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-
-/**
- * Below class will be used to process the block for detail query
- */
-public class DataBlockIteratorImpl extends AbstractDataBlockIterator {
-  /**
-   * DataBlockIteratorImpl Constructor
-   *
-   * @param blockExecutionInfo execution information
-   */
-  public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader,
-      int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) {
-    super(blockExecutionInfo, fileReader, batchSize, queryStatisticsModel, executorService);
-  }
-
-  /**
-   * It scans the block and returns the result with @batchSize
-   *
-   * @return Result of @batchSize
-   */
-  public List<Object[]> next() {
-    List<Object[]> collectedResult = null;
-    if (updateScanner()) {
-      collectedResult = this.scannerResultAggregator.collectData(scannedResult, batchSize);
-      while (collectedResult.size() < batchSize && updateScanner()) {
-        List<Object[]> data = this.scannerResultAggregator
-            .collectData(scannedResult, batchSize - collectedResult.size());
-        collectedResult.addAll(data);
-      }
-    } else {
-      collectedResult = new ArrayList<>();
-    }
-    return collectedResult;
-  }
-
-  public void processNextBatch(CarbonColumnarBatch columnarBatch) {
-    if (updateScanner()) {
-      this.scannerResultAggregator.collectVectorBatch(scannedResult, columnarBatch);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
deleted file mode 100644
index b089fad..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
+++ /dev/null
@@ -1,698 +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.scan.result;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.Map;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
-import org.apache.carbondata.core.mutate.DeleteDeltaVo;
-import org.apache.carbondata.core.mutate.TupleIdEnum;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * Scanned result class which will store and provide the result on request
- */
-public abstract class AbstractScannedResult {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractScannedResult.class.getName());
-  /**
-   * current row number
-   */
-  protected int currentRow = -1;
-
-  protected int pageCounter;
-  /**
-   * row mapping indexes
-   */
-  protected int[][] rowMapping;
-  /**
-   * key size of the fixed length column
-   */
-  private int fixedLengthKeySize;
-  /**
-   * total number of rows per page
-   */
-  private int[] numberOfRows;
-
-  /**
-   * Total number of rows.
-   */
-  private int totalNumberOfRows;
-  /**
-   * to keep track of number of rows process
-   */
-  protected int rowCounter;
-  /**
-   * dimension column data chunk
-   */
-  protected DimensionColumnDataChunk[][] dimensionDataChunks;
-
-  /**
-   * Raw dimension chunks;
-   */
-  protected DimensionRawColumnChunk[] dimRawColumnChunks;
-
-  /**
-   * Raw dimension chunks;
-   */
-  protected MeasureRawColumnChunk[] msrRawColumnChunks;
-  /**
-   * measure column data chunk
-   */
-  protected ColumnPage[][] measureDataChunks;
-  /**
-   * dictionary column block index in file
-   */
-  protected int[] dictionaryColumnBlockIndexes;
-
-  /**
-   * no dictionary column block index in file
-   */
-  protected int[] noDictionaryColumnBlockIndexes;
-
-  /**
-   * column group to is key structure info
-   * which will be used to get the key from the complete
-   * column group key
-   * For example if only one dimension of the column group is selected
-   * then from complete column group key it will be used to mask the key and
-   * get the particular column key
-   */
-  protected Map<Integer, KeyStructureInfo> columnGroupKeyStructureInfo;
-
-  /**
-   *
-   */
-  private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
-
-  private int totalDimensionsSize;
-
-  /**
-   * blockedId which will be blockId + blocklet number in the block
-   */
-  private String blockletId;
-
-  private long rowId;
-
-  /**
-   * parent block indexes
-   */
-  private int[] complexParentBlockIndexes;
-
-  /**
-   * blockletid+pageumber to deleted reocrd map
-   */
-  private Map<String, DeleteDeltaVo> deletedRecordMap;
-
-  /**
-   * current page delete delta vo
-   */
-  private DeleteDeltaVo currentDeleteDeltaVo;
-
-  /**
-   * actual blocklet number
-   */
-  private String blockletNumber;
-
-  public AbstractScannedResult(BlockExecutionInfo blockExecutionInfo) {
-    this.fixedLengthKeySize = blockExecutionInfo.getFixedLengthKeySize();
-    this.noDictionaryColumnBlockIndexes = blockExecutionInfo.getNoDictionaryBlockIndexes();
-    this.dictionaryColumnBlockIndexes = blockExecutionInfo.getDictionaryColumnBlockIndex();
-    this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo();
-    this.complexParentIndexToQueryMap = blockExecutionInfo.getComlexDimensionInfoMap();
-    this.complexParentBlockIndexes = blockExecutionInfo.getComplexColumnParentBlockIndexes();
-    this.totalDimensionsSize = blockExecutionInfo.getQueryDimensions().length;
-    this.deletedRecordMap = blockExecutionInfo.getDeletedRecordsMap();
-  }
-
-  /**
-   * Below method will be used to set the dimension chunks
-   * which will be used to create a row
-   *
-   * @param dataChunks dimension chunks used in query
-   */
-  public void setDimensionChunks(DimensionColumnDataChunk[][] dataChunks) {
-    this.dimensionDataChunks = dataChunks;
-  }
-
-  /**
-   * Below method will be used to set the measure column chunks
-   *
-   * @param measureDataChunks measure data chunks
-   */
-  public void setMeasureChunks(ColumnPage[][] measureDataChunks) {
-    this.measureDataChunks = measureDataChunks;
-  }
-
-  public void setDimRawColumnChunks(DimensionRawColumnChunk[] dimRawColumnChunks) {
-    this.dimRawColumnChunks = dimRawColumnChunks;
-  }
-
-  public void setMsrRawColumnChunks(MeasureRawColumnChunk[] msrRawColumnChunks) {
-    this.msrRawColumnChunks = msrRawColumnChunks;
-  }
-
-  /**
-   * Below method will be used to get the chunk based in measure ordinal
-   *
-   * @param ordinal measure ordinal
-   * @return measure column chunk
-   */
-  public ColumnPage getMeasureChunk(int ordinal) {
-    return measureDataChunks[ordinal][pageCounter];
-  }
-
-  /**
-   * Below method will be used to get the key for all the dictionary dimensions
-   * which is present in the query
-   *
-   * @param rowId row id selected after scanning
-   * @return return the dictionary key
-   */
-  protected byte[] getDictionaryKeyArray(int rowId) {
-    byte[] completeKey = new byte[fixedLengthKeySize];
-    int offset = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      offset += dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillChunkData(completeKey, offset, rowId,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
-    }
-    rowCounter++;
-    return completeKey;
-  }
-
-  /**
-   * Below method will be used to get the key for all the dictionary dimensions
-   * in integer array format which is present in the query
-   *
-   * @param rowId row id selected after scanning
-   * @return return the dictionary key
-   */
-  protected int[] getDictionaryKeyIntegerArray(int rowId) {
-    int[] completeKey = new int[totalDimensionsSize];
-    int column = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(rowId, column, completeKey,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
-    }
-    rowCounter++;
-    return completeKey;
-  }
-
-  /**
-   * Fill the column data of dictionary to vector
-   */
-  public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
-    int column = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(vectorInfo, column,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
-    }
-  }
-
-  /**
-   * Fill the column data to vector
-   */
-  public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
-    int column = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(vectorInfo, column,
-              columnGroupKeyStructureInfo.get(noDictionaryColumnBlockIndexes[i]));
-    }
-  }
-
-  /**
-   * Fill the measure column data to vector
-   */
-  public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) {
-    for (int i = 0; i < measuresOrdinal.length; i++) {
-      vectorInfo[i].measureVectorFiller
-          .fillMeasureVector(measureDataChunks[measuresOrdinal[i]][pageCounter], vectorInfo[i]);
-    }
-  }
-
-  public void fillColumnarComplexBatch(ColumnVectorInfo[] vectorInfos) {
-    for (int i = 0; i < vectorInfos.length; i++) {
-      int offset = vectorInfos[i].offset;
-      int len = offset + vectorInfos[i].size;
-      int vectorOffset = vectorInfos[i].vectorOffset;
-      CarbonColumnVector vector = vectorInfos[i].vector;
-      for (int j = offset; j < len; j++) {
-        ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-        DataOutputStream dataOutput = new DataOutputStream(byteStream);
-        try {
-          vectorInfos[i].genericQueryType
-              .parseBlocksAndReturnComplexColumnByteArray(dimRawColumnChunks,
-                  rowMapping == null ? j : rowMapping[pageCounter][j], pageCounter, dataOutput);
-          Object data = vectorInfos[i].genericQueryType
-              .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray()));
-          vector.putObject(vectorOffset++, data);
-        } catch (IOException e) {
-          LOGGER.error(e);
-        } finally {
-          CarbonUtil.closeStreams(dataOutput);
-          CarbonUtil.closeStreams(byteStream);
-        }
-      }
-    }
-  }
-
-  /**
-   * Fill the column data to vector
-   */
-  public void fillColumnarImplicitBatch(ColumnVectorInfo[] vectorInfo) {
-    for (int i = 0; i < vectorInfo.length; i++) {
-      ColumnVectorInfo columnVectorInfo = vectorInfo[i];
-      CarbonColumnVector vector = columnVectorInfo.vector;
-      int offset = columnVectorInfo.offset;
-      int vectorOffset = columnVectorInfo.vectorOffset;
-      int len = offset + columnVectorInfo.size;
-      for (int j = offset; j < len; j++) {
-        // Considering only String case now as we support only
-        String data = getBlockletId();
-        if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID
-            .equals(columnVectorInfo.dimension.getColumnName())) {
-          data = data + CarbonCommonConstants.FILE_SEPARATOR + pageCounter
-              + CarbonCommonConstants.FILE_SEPARATOR + (rowMapping == null ?
-              j :
-              rowMapping[pageCounter][j]);
-        }
-        vector.putBytes(vectorOffset++,
-            data.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-      }
-    }
-  }
-
-  /**
-   * Just increment the counter incase of query only on measures.
-   */
-  public void incrementCounter() {
-    rowCounter++;
-    currentRow++;
-  }
-
-  /**
-   * Just increment the page counter and reset the remaining counters.
-   */
-  public void incrementPageCounter() {
-    rowCounter = 0;
-    currentRow = -1;
-    pageCounter++;
-    fillDataChunks();
-    if (null != deletedRecordMap) {
-      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
-    }
-  }
-
-  /**
-   * This case is used only in case of compaction, since it does not use filter flow.
-   */
-  public void fillDataChunks() {
-    freeDataChunkMemory();
-    if (pageCounter >= numberOfRows.length) {
-      return;
-    }
-    for (int i = 0; i < dimensionDataChunks.length; i++) {
-      if (dimensionDataChunks[i][pageCounter] == null && dimRawColumnChunks[i] != null) {
-        dimensionDataChunks[i][pageCounter] =
-            dimRawColumnChunks[i].convertToDimColDataChunkWithOutCache(pageCounter);
-      }
-    }
-
-    for (int i = 0; i < measureDataChunks.length; i++) {
-      if (measureDataChunks[i][pageCounter] == null && msrRawColumnChunks[i] != null) {
-        measureDataChunks[i][pageCounter] =
-            msrRawColumnChunks[i].convertToColumnPageWithOutCache(pageCounter);
-      }
-    }
-  }
-
-  // free the memory for the last page chunk
-  private void freeDataChunkMemory() {
-    for (int i = 0; i < dimensionDataChunks.length; i++) {
-      if (pageCounter > 0 && dimensionDataChunks[i][pageCounter - 1] != null) {
-        dimensionDataChunks[i][pageCounter - 1].freeMemory();
-        dimensionDataChunks[i][pageCounter - 1] = null;
-      }
-    }
-    for (int i = 0; i < measureDataChunks.length; i++) {
-      if (pageCounter > 0 && measureDataChunks[i][pageCounter - 1] != null) {
-        measureDataChunks[i][pageCounter - 1].freeMemory();
-        measureDataChunks[i][pageCounter - 1] = null;
-      }
-    }
-  }
-
-  public int numberOfpages() {
-    return numberOfRows.length;
-  }
-
-  /**
-   * Get total rows in the current page
-   *
-   * @return
-   */
-  public int getCurrentPageRowCount() {
-    return numberOfRows[pageCounter];
-  }
-
-  public int getCurrentPageCounter() {
-    return pageCounter;
-  }
-
-  /**
-   * increment the counter.
-   */
-  public void setRowCounter(int rowCounter) {
-    this.rowCounter = rowCounter;
-  }
-
-  /**
-   * Below method will be used to get the dimension data based on dimension
-   * ordinal and index
-   *
-   * @param dimOrdinal dimension ordinal present in the query
-   * @param rowId      row index
-   * @return dimension data based on row id
-   */
-  protected byte[] getDimensionData(int dimOrdinal, int rowId) {
-    return dimensionDataChunks[dimOrdinal][pageCounter].getChunkData(rowId);
-  }
-
-  /**
-   * Below method will be used to get the dimension key array
-   * for all the no dictionary dimension present in the query
-   *
-   * @param rowId row number
-   * @return no dictionary keys for all no dictionary dimension
-   */
-  protected byte[][] getNoDictionaryKeyArray(int rowId) {
-    byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnBlockIndexes.length][];
-    int position = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      noDictionaryColumnsKeys[position++] =
-          dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter].getChunkData(rowId);
-    }
-    return noDictionaryColumnsKeys;
-  }
-
-  /**
-   * Below method will be used to get the dimension key array
-   * for all the no dictionary dimension present in the query
-   *
-   * @param rowId row number
-   * @return no dictionary keys for all no dictionary dimension
-   */
-  protected String[] getNoDictionaryKeyStringArray(int rowId) {
-    String[] noDictionaryColumnsKeys = new String[noDictionaryColumnBlockIndexes.length];
-    int position = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      noDictionaryColumnsKeys[position++] = new String(
-          dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter].getChunkData(rowId),
-          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    }
-    return noDictionaryColumnsKeys;
-  }
-
-  /**
-   * @return blockletId
-   */
-  public String getBlockletId() {
-    return blockletId;
-  }
-
-  /**
-   * @param blockletId
-   */
-  public void setBlockletId(String blockletId) {
-    this.blockletId = CarbonTablePath.getShortBlockId(blockletId);
-    blockletNumber = CarbonUpdateUtil.getRequiredFieldFromTID(blockletId, TupleIdEnum.BLOCKLET_ID);
-    // if deleted recors map is present for this block
-    // then get the first page deleted vo
-    if (null != deletedRecordMap) {
-      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + '_' + pageCounter);
-    }
-  }
-
-  /**
-   * @return blockletId
-   */
-  public long getRowId() {
-    return rowId;
-  }
-
-  /**
-   * @param rowId
-   */
-  public void setRowId(long rowId) {
-    this.rowId = rowId;
-  }
-
-  /**
-   * Below method will be used to get the complex type keys array based
-   * on row id for all the complex type dimension selected in query
-   *
-   * @param rowId row number
-   * @return complex type key array for all the complex dimension selected in query
-   */
-  protected byte[][] getComplexTypeKeyArray(int rowId) {
-    byte[][] complexTypeData = new byte[complexParentBlockIndexes.length][];
-    for (int i = 0; i < complexTypeData.length; i++) {
-      GenericQueryType genericQueryType =
-          complexParentIndexToQueryMap.get(complexParentBlockIndexes[i]);
-      ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-      DataOutputStream dataOutput = new DataOutputStream(byteStream);
-      try {
-        genericQueryType
-            .parseBlocksAndReturnComplexColumnByteArray(dimRawColumnChunks, rowId, pageCounter,
-                dataOutput);
-        complexTypeData[i] = byteStream.toByteArray();
-      } catch (IOException e) {
-        LOGGER.error(e);
-      } finally {
-        CarbonUtil.closeStreams(dataOutput);
-        CarbonUtil.closeStreams(byteStream);
-      }
-    }
-    return complexTypeData;
-  }
-
-  /**
-   * @return return the total number of row after scanning
-   */
-  public int numberOfOutputRows() {
-    return this.totalNumberOfRows;
-  }
-
-  /**
-   * to check whether any more row is present in the result
-   *
-   * @return
-   */
-  public boolean hasNext() {
-    if (pageCounter < numberOfRows.length && rowCounter < this.numberOfRows[pageCounter]) {
-      return true;
-    } else if (pageCounter < numberOfRows.length) {
-      pageCounter++;
-      fillDataChunks();
-      rowCounter = 0;
-      currentRow = -1;
-      if (null != deletedRecordMap) {
-        currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
-      }
-      return hasNext();
-    }
-    return false;
-  }
-
-  /**
-   * Below method will be used to free the occupied memory
-   */
-  public void freeMemory() {
-    // first free the dimension chunks
-    if (null != dimensionDataChunks) {
-      for (int i = 0; i < dimensionDataChunks.length; i++) {
-        if (null != dimensionDataChunks[i]) {
-          for (int j = 0; j < dimensionDataChunks[i].length; j++) {
-            if (null != dimensionDataChunks[i][j]) {
-              dimensionDataChunks[i][j].freeMemory();
-            }
-          }
-        }
-      }
-    }
-    // free the measure data chunks
-    if (null != measureDataChunks) {
-      for (int i = 0; i < measureDataChunks.length; i++) {
-        if (null != measureDataChunks[i]) {
-          for (int j = 0; j < measureDataChunks[i].length; j++) {
-            if (null != measureDataChunks[i][j]) {
-              measureDataChunks[i][j].freeMemory();
-            }
-          }
-        }
-      }
-    }
-    // free the raw chunks
-    if (null != dimRawColumnChunks) {
-      for (int i = 0; i < dimRawColumnChunks.length; i++) {
-        if (null != dimRawColumnChunks[i]) {
-          dimRawColumnChunks[i].freeMemory();
-        }
-      }
-    }
-  }
-
-  /**
-   * As this class will be a flyweight object so
-   * for one block all the blocklet scanning will use same result object
-   * in that case we need to reset the counter to zero so
-   * for new result it will give the result from zero
-   */
-  public void reset() {
-    rowCounter = 0;
-    currentRow = -1;
-    pageCounter = 0;
-  }
-
-  /**
-   * @param numberOfRows set total of number rows valid after scanning
-   */
-  public void setNumberOfRows(int[] numberOfRows) {
-    this.numberOfRows = numberOfRows;
-
-    for (int count : numberOfRows) {
-      totalNumberOfRows += count;
-    }
-  }
-
-  /**
-   * After applying filter it will return the  bit set with the valid row indexes
-   * so below method will be used to set the row indexes
-   *
-   * @param indexes
-   */
-  public void setIndexes(int[][] indexes) {
-    this.rowMapping = indexes;
-  }
-
-  public int getRowCounter() {
-    return rowCounter;
-  }
-
-  /**
-   * will return the current valid row id
-   *
-   * @return valid row id
-   */
-  public abstract int getCurrentRowId();
-
-  /**
-   * @return dictionary key array for all the dictionary dimension
-   * selected in query
-   */
-  public abstract byte[] getDictionaryKeyArray();
-
-  /**
-   * @return dictionary key array for all the dictionary dimension in integer array forat
-   * selected in query
-   */
-  public abstract int[] getDictionaryKeyIntegerArray();
-
-  /**
-   * Below method will be used to get the complex type key array
-   *
-   * @return complex type key array
-   */
-  public abstract byte[][] getComplexTypeKeyArray();
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  public abstract byte[][] getNoDictionaryKeyArray();
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * array in string array format for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  public abstract String[] getNoDictionaryKeyStringArray();
-
-  /**
-   * Mark the filtered rows in columnar batch. These rows will not be added to vector batches later.
-   * @param columnarBatch
-   * @param startRow
-   * @param size
-   * @param vectorOffset
-   */
-  public int markFilteredRows(CarbonColumnarBatch columnarBatch, int startRow, int size,
-      int vectorOffset) {
-    int rowsFiltered = 0;
-    if (currentDeleteDeltaVo != null) {
-      int len = startRow + size;
-      for (int i = startRow; i < len; i++) {
-        int rowId = rowMapping != null ? rowMapping[pageCounter][i] : i;
-        if (currentDeleteDeltaVo.containsRow(rowId)) {
-          columnarBatch.markFiltered(vectorOffset);
-          rowsFiltered++;
-        }
-        vectorOffset++;
-      }
-    }
-    return rowsFiltered;
-  }
-
-  /**
-   * Below method will be used to check row got deleted
-   *
-   * @param rowId
-   * @return is present in deleted row
-   */
-  public boolean containsDeletedRow(int rowId) {
-    if (null != currentDeleteDeltaVo) {
-      return currentDeleteDeltaVo.containsRow(rowId);
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java
deleted file mode 100644
index 56ca2ac..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java
+++ /dev/null
@@ -1,103 +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.scan.result;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import org.apache.carbondata.common.CarbonIterator;
-
-/**
- * Below class holds the query result
- */
-public class BatchResult extends CarbonIterator<Object[]> {
-
-  /**
-   * list of keys
-   */
-  protected List<Object[]> rows;
-
-  /**
-   * counter to check whether all the records are processed or not
-   */
-  protected int counter;
-
-  public BatchResult() {
-    this.rows = new ArrayList<>();
-  }
-
-  /**
-   * Below method will be used to get the rows
-   *
-   * @return
-   */
-  public List<Object[]> getRows() {
-    return rows;
-  }
-
-  /**
-   * Below method will be used to get the set the values
-   *
-   * @param rows
-   */
-  public void setRows(List<Object[]> rows) {
-    this.rows = rows;
-  }
-
-  /**
-   * This method will return one row at a time based on the counter given.
-   * @param counter
-   * @return
-   */
-  public Object[] getRawRow(int counter) {
-    return rows.get(counter);
-  }
-
-  /**
-   * For getting the total size.
-   * @return
-   */
-  public int getSize() {
-    return rows.size();
-  }
-
-
-  /**
-   * Returns {@code true} if the iteration has more elements.
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return counter < rows.size();
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Object[] next() {
-    if (!hasNext()) {
-      throw new NoSuchElementException();
-    }
-    Object[] row = rows.get(counter);
-    counter++;
-    return row;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
new file mode 100644
index 0000000..29404b4
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
@@ -0,0 +1,618 @@
+/*
+ * 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.scan.result;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.Map;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
+import org.apache.carbondata.core.mutate.DeleteDeltaVo;
+import org.apache.carbondata.core.mutate.TupleIdEnum;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+/**
+ * Scanned result class which will store and provide the result on request
+ */
+public abstract class BlockletScannedResult {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(BlockletScannedResult.class.getName());
+  /**
+   * current row number
+   */
+  protected int currentRow = -1;
+
+  protected int pageCounter;
+  /**
+   * matched rowId for each page
+   */
+  protected int[][] pageFilteredRowId;
+  /**
+   * key size of the fixed length column
+   */
+  private int fixedLengthKeySize;
+  /**
+   * total number of filtered rows for each page
+   */
+  private int[] pageFilteredRowCount;
+
+  /**
+   * to keep track of number of rows process
+   */
+  protected int rowCounter;
+  /**
+   * dimension column data chunk
+   */
+  protected DimensionColumnPage[][] dimensionColumnPages;
+
+  /**
+   * Raw dimension chunks;
+   */
+  protected DimensionRawColumnChunk[] dimRawColumnChunks;
+
+  /**
+   * Raw dimension chunks;
+   */
+  protected MeasureRawColumnChunk[] msrRawColumnChunks;
+  /**
+   * measure column data chunk
+   */
+  protected ColumnPage[][] measureColumnPages;
+  /**
+   * dictionary column block index in file
+   */
+  protected int[] dictionaryColumnChunkIndexes;
+
+  /**
+   * no dictionary column chunk index in file
+   */
+  protected int[] noDictionaryColumnChunkIndexes;
+
+  /**
+   * column group to is key structure info
+   * which will be used to get the key from the complete
+   * column group key
+   * For example if only one dimension of the column group is selected
+   * then from complete column group key it will be used to mask the key and
+   * get the particular column key
+   */
+  protected Map<Integer, KeyStructureInfo> columnGroupKeyStructureInfo;
+
+  /**
+   *
+   */
+  private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
+
+  private int totalDimensionsSize;
+
+  /**
+   * blockedId which will be blockId + blocklet number in the block
+   */
+  private String blockletId;
+
+  /**
+   * parent block indexes
+   */
+  private int[] complexParentBlockIndexes;
+
+  /**
+   * blockletid+pageumber to deleted reocrd map
+   */
+  private Map<String, DeleteDeltaVo> deletedRecordMap;
+
+  /**
+   * current page delete delta vo
+   */
+  private DeleteDeltaVo currentDeleteDeltaVo;
+
+  /**
+   * actual blocklet number
+   */
+  private String blockletNumber;
+
+  public BlockletScannedResult(BlockExecutionInfo blockExecutionInfo) {
+    this.fixedLengthKeySize = blockExecutionInfo.getFixedLengthKeySize();
+    this.noDictionaryColumnChunkIndexes = blockExecutionInfo.getNoDictionaryColumnChunkIndexes();
+    this.dictionaryColumnChunkIndexes = blockExecutionInfo.getDictionaryColumnChunkIndex();
+    this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo();
+    this.complexParentIndexToQueryMap = blockExecutionInfo.getComlexDimensionInfoMap();
+    this.complexParentBlockIndexes = blockExecutionInfo.getComplexColumnParentBlockIndexes();
+    this.totalDimensionsSize = blockExecutionInfo.getProjectionDimensions().length;
+    this.deletedRecordMap = blockExecutionInfo.getDeletedRecordsMap();
+  }
+
+  /**
+   * Below method will be used to set the dimension chunks
+   * which will be used to create a row
+   *
+   * @param columnPages dimension chunks used in query
+   */
+  public void setDimensionColumnPages(DimensionColumnPage[][] columnPages) {
+    this.dimensionColumnPages = columnPages;
+  }
+
+  /**
+   * Below method will be used to set the measure column chunks
+   *
+   * @param columnPages measure data chunks
+   */
+  public void setMeasureColumnPages(ColumnPage[][] columnPages) {
+    this.measureColumnPages = columnPages;
+  }
+
+  public void setDimRawColumnChunks(DimensionRawColumnChunk[] dimRawColumnChunks) {
+    this.dimRawColumnChunks = dimRawColumnChunks;
+  }
+
+  public void setMsrRawColumnChunks(MeasureRawColumnChunk[] msrRawColumnChunks) {
+    this.msrRawColumnChunks = msrRawColumnChunks;
+  }
+
+  /**
+   * Below method will be used to get the chunk based in measure ordinal
+   *
+   * @param ordinal measure ordinal
+   * @return measure column chunk
+   */
+  public ColumnPage getMeasureChunk(int ordinal) {
+    return measureColumnPages[ordinal][pageCounter];
+  }
+
+  /**
+   * Below method will be used to get the key for all the dictionary dimensions
+   * which is present in the query
+   *
+   * @param rowId row id selected after scanning
+   * @return return the dictionary key
+   */
+  protected byte[] getDictionaryKeyArray(int rowId) {
+    byte[] completeKey = new byte[fixedLengthKeySize];
+    int offset = 0;
+    for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) {
+      offset += dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter].fillRawData(
+          rowId, offset, completeKey,
+          columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i]));
+    }
+    rowCounter++;
+    return completeKey;
+  }
+
+  /**
+   * Below method will be used to get the key for all the dictionary dimensions
+   * in integer array format which is present in the query
+   *
+   * @param rowId row id selected after scanning
+   * @return return the dictionary key
+   */
+  protected int[] getDictionaryKeyIntegerArray(int rowId) {
+    int[] completeKey = new int[totalDimensionsSize];
+    int column = 0;
+    for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) {
+      column = dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter]
+          .fillSurrogateKey(rowId, column, completeKey,
+              columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i]));
+    }
+    rowCounter++;
+    return completeKey;
+  }
+
+  /**
+   * Fill the column data of dictionary to vector
+   */
+  public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
+    int column = 0;
+    for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) {
+      column = dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter]
+          .fillVector(vectorInfo, column,
+              columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i]));
+    }
+  }
+
+  /**
+   * Fill the column data to vector
+   */
+  public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
+    int column = 0;
+    for (int i = 0; i < this.noDictionaryColumnChunkIndexes.length; i++) {
+      column = dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter]
+          .fillVector(vectorInfo, column,
+              columnGroupKeyStructureInfo.get(noDictionaryColumnChunkIndexes[i]));
+    }
+  }
+
+  /**
+   * Fill the measure column data to vector
+   */
+  public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) {
+    for (int i = 0; i < measuresOrdinal.length; i++) {
+      vectorInfo[i].measureVectorFiller
+          .fillMeasureVector(measureColumnPages[measuresOrdinal[i]][pageCounter], vectorInfo[i]);
+    }
+  }
+
+  public void fillColumnarComplexBatch(ColumnVectorInfo[] vectorInfos) {
+    for (int i = 0; i < vectorInfos.length; i++) {
+      int offset = vectorInfos[i].offset;
+      int len = offset + vectorInfos[i].size;
+      int vectorOffset = vectorInfos[i].vectorOffset;
+      CarbonColumnVector vector = vectorInfos[i].vector;
+      for (int j = offset; j < len; j++) {
+        ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+        DataOutputStream dataOutput = new DataOutputStream(byteStream);
+        try {
+          vectorInfos[i].genericQueryType.parseBlocksAndReturnComplexColumnByteArray(
+              dimRawColumnChunks,
+              pageFilteredRowId == null ? j : pageFilteredRowId[pageCounter][j], pageCounter,
+              dataOutput);
+          Object data = vectorInfos[i].genericQueryType
+              .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray()));
+          vector.putObject(vectorOffset++, data);
+        } catch (IOException e) {
+          LOGGER.error(e);
+        } finally {
+          CarbonUtil.closeStreams(dataOutput);
+          CarbonUtil.closeStreams(byteStream);
+        }
+      }
+    }
+  }
+
+  /**
+   * Fill the column data to vector
+   */
+  public void fillColumnarImplicitBatch(ColumnVectorInfo[] vectorInfo) {
+    for (int i = 0; i < vectorInfo.length; i++) {
+      ColumnVectorInfo columnVectorInfo = vectorInfo[i];
+      CarbonColumnVector vector = columnVectorInfo.vector;
+      int offset = columnVectorInfo.offset;
+      int vectorOffset = columnVectorInfo.vectorOffset;
+      int len = offset + columnVectorInfo.size;
+      for (int j = offset; j < len; j++) {
+        // Considering only String case now as we support only
+        String data = getBlockletId();
+        if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID
+            .equals(columnVectorInfo.dimension.getColumnName())) {
+          data = data + CarbonCommonConstants.FILE_SEPARATOR + pageCounter
+              + CarbonCommonConstants.FILE_SEPARATOR + (pageFilteredRowId == null ?
+              j :
+              pageFilteredRowId[pageCounter][j]);
+        }
+        vector.putBytes(vectorOffset++,
+            data.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+      }
+    }
+  }
+
+  /**
+   * Just increment the counter incase of query only on measures.
+   */
+  public void incrementCounter() {
+    rowCounter++;
+    currentRow++;
+  }
+
+  /**
+   * Just increment the page counter and reset the remaining counters.
+   */
+  public void incrementPageCounter() {
+    rowCounter = 0;
+    currentRow = -1;
+    pageCounter++;
+    fillDataChunks();
+    if (null != deletedRecordMap) {
+      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
+    }
+  }
+
+  /**
+   * This case is used only in case of compaction, since it does not use filter flow.
+   */
+  public void fillDataChunks() {
+    freeDataChunkMemory();
+    if (pageCounter >= pageFilteredRowCount.length) {
+      return;
+    }
+    for (int i = 0; i < dimensionColumnPages.length; i++) {
+      if (dimensionColumnPages[i][pageCounter] == null && dimRawColumnChunks[i] != null) {
+        dimensionColumnPages[i][pageCounter] =
+            dimRawColumnChunks[i].convertToDimColDataChunkWithOutCache(pageCounter);
+      }
+    }
+
+    for (int i = 0; i < measureColumnPages.length; i++) {
+      if (measureColumnPages[i][pageCounter] == null && msrRawColumnChunks[i] != null) {
+        measureColumnPages[i][pageCounter] =
+            msrRawColumnChunks[i].convertToColumnPageWithOutCache(pageCounter);
+      }
+    }
+  }
+
+  // free the memory for the last page chunk
+  private void freeDataChunkMemory() {
+    for (int i = 0; i < dimensionColumnPages.length; i++) {
+      if (pageCounter > 0 && dimensionColumnPages[i][pageCounter - 1] != null) {
+        dimensionColumnPages[i][pageCounter - 1].freeMemory();
+        dimensionColumnPages[i][pageCounter - 1] = null;
+      }
+    }
+    for (int i = 0; i < measureColumnPages.length; i++) {
+      if (pageCounter > 0 && measureColumnPages[i][pageCounter - 1] != null) {
+        measureColumnPages[i][pageCounter - 1].freeMemory();
+        measureColumnPages[i][pageCounter - 1] = null;
+      }
+    }
+  }
+
+  public int numberOfpages() {
+    return pageFilteredRowCount.length;
+  }
+
+  /**
+   * Get total rows in the current page
+   *
+   * @return
+   */
+  public int getCurrentPageRowCount() {
+    return pageFilteredRowCount[pageCounter];
+  }
+
+  public int getCurrentPageCounter() {
+    return pageCounter;
+  }
+
+  /**
+   * increment the counter.
+   */
+  public void setRowCounter(int rowCounter) {
+    this.rowCounter = rowCounter;
+  }
+
+  /**
+   * Below method will be used to get the dimension key array
+   * for all the no dictionary dimension present in the query
+   *
+   * @param rowId row number
+   * @return no dictionary keys for all no dictionary dimension
+   */
+  protected byte[][] getNoDictionaryKeyArray(int rowId) {
+    byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnChunkIndexes.length][];
+    int position = 0;
+    for (int i = 0; i < this.noDictionaryColumnChunkIndexes.length; i++) {
+      noDictionaryColumnsKeys[position++] =
+          dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter].getChunkData(rowId);
+    }
+    return noDictionaryColumnsKeys;
+  }
+
+  /**
+   * @return blockletId
+   */
+  public String getBlockletId() {
+    return blockletId;
+  }
+
+  /**
+   * Set blocklet id, which looks like
+   * "Part0/Segment_0/part-0-0_batchno0-0-1517155583332.carbondata/0"
+   */
+  public void setBlockletId(String blockletId) {
+    this.blockletId = CarbonTablePath.getShortBlockId(blockletId);
+    blockletNumber = CarbonUpdateUtil.getRequiredFieldFromTID(blockletId, TupleIdEnum.BLOCKLET_ID);
+    // if deleted recors map is present for this block
+    // then get the first page deleted vo
+    if (null != deletedRecordMap) {
+      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + '_' + pageCounter);
+    }
+  }
+
+  /**
+   * Below method will be used to get the complex type keys array based
+   * on row id for all the complex type dimension selected in query
+   *
+   * @param rowId row number
+   * @return complex type key array for all the complex dimension selected in query
+   */
+  protected byte[][] getComplexTypeKeyArray(int rowId) {
+    byte[][] complexTypeData = new byte[complexParentBlockIndexes.length][];
+    for (int i = 0; i < complexTypeData.length; i++) {
+      GenericQueryType genericQueryType =
+          complexParentIndexToQueryMap.get(complexParentBlockIndexes[i]);
+      ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+      DataOutputStream dataOutput = new DataOutputStream(byteStream);
+      try {
+        genericQueryType
+            .parseBlocksAndReturnComplexColumnByteArray(dimRawColumnChunks, rowId, pageCounter,
+                dataOutput);
+        complexTypeData[i] = byteStream.toByteArray();
+      } catch (IOException e) {
+        LOGGER.error(e);
+      } finally {
+        CarbonUtil.closeStreams(dataOutput);
+        CarbonUtil.closeStreams(byteStream);
+      }
+    }
+    return complexTypeData;
+  }
+
+  /**
+   * to check whether any more row is present in the result
+   *
+   * @return
+   */
+  public boolean hasNext() {
+    if (pageCounter
+        < pageFilteredRowCount.length && rowCounter < this.pageFilteredRowCount[pageCounter]) {
+      return true;
+    } else if (pageCounter < pageFilteredRowCount.length) {
+      pageCounter++;
+      fillDataChunks();
+      rowCounter = 0;
+      currentRow = -1;
+      if (null != deletedRecordMap) {
+        currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
+      }
+      return hasNext();
+    }
+    return false;
+  }
+
+  /**
+   * Below method will be used to free the occupied memory
+   */
+  public void freeMemory() {
+    // first free the dimension chunks
+    if (null != dimensionColumnPages) {
+      for (int i = 0; i < dimensionColumnPages.length; i++) {
+        if (null != dimensionColumnPages[i]) {
+          for (int j = 0; j < dimensionColumnPages[i].length; j++) {
+            if (null != dimensionColumnPages[i][j]) {
+              dimensionColumnPages[i][j].freeMemory();
+            }
+          }
+        }
+      }
+    }
+    // free the measure data chunks
+    if (null != measureColumnPages) {
+      for (int i = 0; i < measureColumnPages.length; i++) {
+        if (null != measureColumnPages[i]) {
+          for (int j = 0; j < measureColumnPages[i].length; j++) {
+            if (null != measureColumnPages[i][j]) {
+              measureColumnPages[i][j].freeMemory();
+            }
+          }
+        }
+      }
+    }
+    // free the raw chunks
+    if (null != dimRawColumnChunks) {
+      for (int i = 0; i < dimRawColumnChunks.length; i++) {
+        if (null != dimRawColumnChunks[i]) {
+          dimRawColumnChunks[i].freeMemory();
+        }
+      }
+    }
+  }
+
+  /**
+   * @param pageFilteredRowCount set total of number rows valid after scanning
+   */
+  public void setPageFilteredRowCount(int[] pageFilteredRowCount) {
+    this.pageFilteredRowCount = pageFilteredRowCount;
+  }
+
+  /**
+   * After applying filter it will return the  bit set with the valid row indexes
+   * so below method will be used to set the row indexes
+   */
+  public void setPageFilteredRowId(int[][] pageFilteredRowId) {
+    this.pageFilteredRowId = pageFilteredRowId;
+  }
+
+  public int getRowCounter() {
+    return rowCounter;
+  }
+
+  /**
+   * will return the current valid row id
+   *
+   * @return valid row id
+   */
+  public abstract int getCurrentRowId();
+
+  /**
+   * @return dictionary key array for all the dictionary dimension
+   * selected in query
+   */
+  public abstract byte[] getDictionaryKeyArray();
+
+  /**
+   * @return dictionary key array for all the dictionary dimension in integer array forat
+   * selected in query
+   */
+  public abstract int[] getDictionaryKeyIntegerArray();
+
+  /**
+   * Below method will be used to get the complex type key array
+   *
+   * @return complex type key array
+   */
+  public abstract byte[][] getComplexTypeKeyArray();
+
+  /**
+   * Below method will be used to get the no dictionary key
+   * array for all the no dictionary dimension selected in query
+   *
+   * @return no dictionary key array for all the no dictionary dimension
+   */
+  public abstract byte[][] getNoDictionaryKeyArray();
+
+  /**
+   * Mark the filtered rows in columnar batch. These rows will not be added to vector batches later.
+   * @param columnarBatch
+   * @param startRow
+   * @param size
+   * @param vectorOffset
+   */
+  public int markFilteredRows(CarbonColumnarBatch columnarBatch, int startRow, int size,
+      int vectorOffset) {
+    int rowsFiltered = 0;
+    if (currentDeleteDeltaVo != null) {
+      int len = startRow + size;
+      for (int i = startRow; i < len; i++) {
+        int rowId = pageFilteredRowId != null ? pageFilteredRowId[pageCounter][i] : i;
+        if (currentDeleteDeltaVo.containsRow(rowId)) {
+          columnarBatch.markFiltered(vectorOffset);
+          rowsFiltered++;
+        }
+        vectorOffset++;
+      }
+    }
+    return rowsFiltered;
+  }
+
+  /**
+   * Below method will be used to check row got deleted
+   *
+   * @param rowId
+   * @return is present in deleted row
+   */
+  public boolean containsDeletedRow(int rowId) {
+    if (null != currentDeleteDeltaVo) {
+      return currentDeleteDeltaVo.containsRow(rowId);
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java b/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java
new file mode 100644
index 0000000..c129161
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java
@@ -0,0 +1,103 @@
+/*
+ * 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.scan.result;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.carbondata.common.CarbonIterator;
+
+/**
+ * Below class holds the query result
+ */
+public class RowBatch extends CarbonIterator<Object[]> {
+
+  /**
+   * list of keys
+   */
+  protected List<Object[]> rows;
+
+  /**
+   * counter to check whether all the records are processed or not
+   */
+  protected int counter;
+
+  public RowBatch() {
+    this.rows = new ArrayList<>();
+  }
+
+  /**
+   * Below method will be used to get the rows
+   *
+   * @return
+   */
+  public List<Object[]> getRows() {
+    return rows;
+  }
+
+  /**
+   * Below method will be used to get the set the values
+   *
+   * @param rows
+   */
+  public void setRows(List<Object[]> rows) {
+    this.rows = rows;
+  }
+
+  /**
+   * This method will return one row at a time based on the counter given.
+   * @param counter
+   * @return
+   */
+  public Object[] getRawRow(int counter) {
+    return rows.get(counter);
+  }
+
+  /**
+   * For getting the total size.
+   * @return
+   */
+  public int getSize() {
+    return rows.size();
+  }
+
+
+  /**
+   * Returns {@code true} if the iteration has more elements.
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  @Override public boolean hasNext() {
+    return counter < rows.size();
+  }
+
+  /**
+   * Returns the next element in the iteration.
+   *
+   * @return the next element in the iteration
+   */
+  @Override public Object[] next() {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
+    Object[] row = rows.get(counter);
+    counter++;
+    return row;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
index 8120310..bcc5634 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
@@ -17,7 +17,7 @@
 package org.apache.carbondata.core.scan.result.impl;
 
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
 
 /**
@@ -25,7 +25,7 @@ import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
  * In case of filter query data will be send
  * based on filtered row index
  */
-public class FilterQueryScannedResult extends AbstractScannedResult {
+public class FilterQueryScannedResult extends BlockletScannedResult {
 
   public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) {
     super(tableBlockExecutionInfos);
@@ -37,7 +37,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   @Override public byte[] getDictionaryKeyArray() {
     ++currentRow;
-    return getDictionaryKeyArray(rowMapping[pageCounter][currentRow]);
+    return getDictionaryKeyArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -46,7 +46,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   @Override public int[] getDictionaryKeyIntegerArray() {
     ++currentRow;
-    return getDictionaryKeyIntegerArray(rowMapping[pageCounter][currentRow]);
+    return getDictionaryKeyIntegerArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -55,7 +55,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    * @return complex type key array
    */
   @Override public byte[][] getComplexTypeKeyArray() {
-    return getComplexTypeKeyArray(rowMapping[pageCounter][currentRow]);
+    return getComplexTypeKeyArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -65,17 +65,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    * @return no dictionary key array for all the no dictionary dimension
    */
   @Override public byte[][] getNoDictionaryKeyArray() {
-    return getNoDictionaryKeyArray(rowMapping[pageCounter][currentRow]);
-  }
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * string array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  @Override public String[] getNoDictionaryKeyStringArray() {
-    return getNoDictionaryKeyStringArray(rowMapping[pageCounter][currentRow]);
+    return getNoDictionaryKeyArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -84,7 +74,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    * @return valid row id
    */
   @Override public int getCurrentRowId() {
-    return rowMapping[pageCounter][currentRow];
+    return pageFilteredRowId[pageCounter][currentRow];
   }
 
   /**
@@ -92,10 +82,12 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
     int column = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(rowMapping[pageCounter], vectorInfo, column,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
+    for (int chunkIndex : this.dictionaryColumnChunkIndexes) {
+      column = dimensionColumnPages[chunkIndex][pageCounter].fillVector(
+          pageFilteredRowId[pageCounter],
+          vectorInfo,
+          column,
+          columnGroupKeyStructureInfo.get(chunkIndex));
     }
   }
 
@@ -104,10 +96,12 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
     int column = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(rowMapping[pageCounter], vectorInfo, column,
-              columnGroupKeyStructureInfo.get(noDictionaryColumnBlockIndexes[i]));
+    for (int chunkIndex : this.noDictionaryColumnChunkIndexes) {
+      column = dimensionColumnPages[chunkIndex][pageCounter].fillVector(
+          pageFilteredRowId[pageCounter],
+          vectorInfo,
+          column,
+          columnGroupKeyStructureInfo.get(chunkIndex));
     }
   }
 
@@ -116,8 +110,10 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) {
     for (int i = 0; i < measuresOrdinal.length; i++) {
-      vectorInfo[i].measureVectorFiller.fillMeasureVectorForFilter(rowMapping[pageCounter],
-          measureDataChunks[measuresOrdinal[i]][pageCounter], vectorInfo[i]);
+      vectorInfo[i].measureVectorFiller.fillMeasureVector(
+          pageFilteredRowId[pageCounter],
+          measureColumnPages[measuresOrdinal[i]][pageCounter],
+          vectorInfo[i]);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
index 3978f9e..06687c2 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
@@ -17,14 +17,14 @@
 package org.apache.carbondata.core.scan.result.impl;
 
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 
 /**
  * Result provide class for non filter query
  * In case of no filter query we need to return
  * complete data
  */
-public class NonFilterQueryScannedResult extends AbstractScannedResult {
+public class NonFilterQueryScannedResult extends BlockletScannedResult {
 
   public NonFilterQueryScannedResult(BlockExecutionInfo blockExecutionInfo) {
     super(blockExecutionInfo);
@@ -68,16 +68,6 @@ public class NonFilterQueryScannedResult extends AbstractScannedResult {
   }
 
   /**
-   * Below method will be used to get the no dictionary key
-   * string array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  @Override public String[] getNoDictionaryKeyStringArray() {
-    return getNoDictionaryKeyStringArray(currentRow);
-  }
-
-  /**
    * will return the current valid row id
    *
    * @return valid row id

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
index 6172b40..4e628fe 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -28,18 +28,17 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.datastore.DataRefNodeFinder;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.impl.btree.BTreeDataRefNodeFinder;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode;
 import org.apache.carbondata.core.mutate.DeleteDeltaVo;
 import org.apache.carbondata.core.reader.CarbonDeleteFilesDataReader;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.infos.DeleteDeltaInfo;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator;
-import org.apache.carbondata.core.scan.processor.impl.DataBlockIteratorImpl;
+import org.apache.carbondata.core.scan.processor.DataBlockIterator;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -63,23 +62,23 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
   private static final Map<DeleteDeltaInfo, Object> deleteDeltaToLockObjectMap =
       new ConcurrentHashMap<>();
 
-  protected ExecutorService execService;
+  private ExecutorService execService;
   /**
    * execution info of the block
    */
-  protected List<BlockExecutionInfo> blockExecutionInfos;
+  private List<BlockExecutionInfo> blockExecutionInfos;
 
   /**
    * file reader which will be used to execute the query
    */
-  protected FileHolder fileReader;
+  protected FileReader fileReader;
 
-  protected AbstractDataBlockIterator dataBlockIterator;
+  DataBlockIterator dataBlockIterator;
 
   /**
    * QueryStatisticsRecorder
    */
-  protected QueryStatisticsRecorder recorder;
+  private QueryStatisticsRecorder recorder;
   /**
    * number of cores which can be used
    */
@@ -89,7 +88,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
    */
   private QueryStatisticsModel queryStatisticsModel;
 
-  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
+  AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
       ExecutorService execService) {
     String batchSizeString =
         CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE);
@@ -107,7 +106,6 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     this.blockExecutionInfos = infos;
     this.fileReader = FileFactory.getFileHolder(
         FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getTablePath()));
-    this.fileReader.setQueryId(queryModel.getQueryId());
     this.fileReader.setReadPageByPage(queryModel.isReadPageByPage());
     this.execService = execService;
     intialiseInfos();
@@ -130,22 +128,21 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
         blockInfo.setDeletedRecordsMap(deletedRowsMap);
       }
       DataRefNode dataRefNode = blockInfo.getDataBlock().getDataRefNode();
-      if (dataRefNode instanceof BlockletDataRefNodeWrapper) {
-        BlockletDataRefNodeWrapper wrapper = (BlockletDataRefNodeWrapper) dataRefNode;
-        blockInfo.setFirstDataBlock(wrapper);
-        blockInfo.setNumberOfBlockToScan(wrapper.numberOfNodes());
-
+      if (dataRefNode instanceof BlockletDataRefNode) {
+        BlockletDataRefNode node = (BlockletDataRefNode) dataRefNode;
+        blockInfo.setFirstDataBlock(node);
+        blockInfo.setNumberOfBlockToScan(node.numberOfNodes());
       } else {
         DataRefNode startDataBlock =
             finder.findFirstDataBlock(dataRefNode, blockInfo.getStartKey());
-        while (startDataBlock.nodeNumber() < blockInfo.getStartBlockletIndex()) {
+        while (startDataBlock.nodeIndex() < blockInfo.getStartBlockletIndex()) {
           startDataBlock = startDataBlock.getNextDataRefNode();
         }
         long numberOfBlockToScan = blockInfo.getNumberOfBlockletToScan();
         //if number of block is less than 0 then take end block.
         if (numberOfBlockToScan <= 0) {
           DataRefNode endDataBlock = finder.findLastDataBlock(dataRefNode, blockInfo.getEndKey());
-          numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
+          numberOfBlockToScan = endDataBlock.nodeIndex() - startDataBlock.nodeIndex() + 1;
         }
         blockInfo.setFirstDataBlock(startDataBlock);
         blockInfo.setNumberOfBlockToScan(numberOfBlockToScan);
@@ -230,7 +227,8 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     }
   }
 
-  @Override public boolean hasNext() {
+  @Override
+  public boolean hasNext() {
     if ((dataBlockIterator != null && dataBlockIterator.hasNext())) {
       return true;
     } else if (blockExecutionInfos.size() > 0) {
@@ -240,7 +238,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     }
   }
 
-  protected void updateDataBlockIterator() {
+  void updateDataBlockIterator() {
     if (dataBlockIterator == null || !dataBlockIterator.hasNext()) {
       dataBlockIterator = getDataBlockIterator();
       while (dataBlockIterator != null && !dataBlockIterator.hasNext()) {
@@ -249,17 +247,17 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     }
   }
 
-  private DataBlockIteratorImpl getDataBlockIterator() {
+  private DataBlockIterator getDataBlockIterator() {
     if (blockExecutionInfos.size() > 0) {
       BlockExecutionInfo executionInfo = blockExecutionInfos.get(0);
       blockExecutionInfos.remove(executionInfo);
-      return new DataBlockIteratorImpl(executionInfo, fileReader, batchSize, queryStatisticsModel,
+      return new DataBlockIterator(executionInfo, fileReader, batchSize, queryStatisticsModel,
           execService);
     }
     return null;
   }
 
-  protected void initQueryStatiticsModel() {
+  private void initQueryStatiticsModel() {
     this.queryStatisticsModel = new QueryStatisticsModel();
     this.queryStatisticsModel.setRecorder(recorder);
     QueryStatistic queryStatisticTotalBlocklet = new QueryStatistic();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
index 1efac30..1235789 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
@@ -18,7 +18,7 @@
 package org.apache.carbondata.core.scan.result.iterator;
 
 import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 
 /**
  * Iterator over row result
@@ -28,14 +28,14 @@ public class ChunkRowIterator extends CarbonIterator<Object[]> {
   /**
    * iterator over chunk result
    */
-  private CarbonIterator<BatchResult> iterator;
+  private CarbonIterator<RowBatch> iterator;
 
   /**
    * currect chunk
    */
-  private BatchResult currentchunk;
+  private RowBatch currentchunk;
 
-  public ChunkRowIterator(CarbonIterator<BatchResult> iterator) {
+  public ChunkRowIterator(CarbonIterator<RowBatch> iterator) {
     this.iterator = iterator;
     if (iterator.hasNext()) {
       currentchunk = iterator.next();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
index 747f5a9..c073c78 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
@@ -21,14 +21,14 @@ import java.util.concurrent.ExecutorService;
 
 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;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
  * executing that query are returning a iterator over block and every time next
  * call will come it will execute the block and return the result
  */
-public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator<BatchResult> {
+public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator<RowBatch> {
 
   private final Object lock = new Object();
 
@@ -37,18 +37,18 @@ public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator
     super(infos, queryModel, execService);
   }
 
-  @Override public BatchResult next() {
+  @Override public RowBatch next() {
     return getBatchResult();
   }
 
-  private BatchResult getBatchResult() {
-    BatchResult batchResult = new BatchResult();
+  private RowBatch getBatchResult() {
+    RowBatch rowBatch = new RowBatch();
     synchronized (lock) {
       updateDataBlockIterator();
       if (dataBlockIterator != null) {
-        batchResult.setRows(dataBlockIterator.next());
+        rowBatch.setRows(dataBlockIterator.next());
       }
     }
-    return batchResult;
+    return rowBatch;
   }
 }


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index 5ebf4cf..eb834a0 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -373,7 +373,6 @@ public abstract class AbstractDataFileFooterConverter {
       cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
     }
     info.setColumnCardinality(cardinality);
-    info.setNumberOfColumns(segmentInfo.getNum_cols());
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index f04b28d..b16bc5e 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -47,10 +47,10 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
@@ -80,7 +80,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
@@ -243,16 +243,13 @@ public final class CarbonUtil {
   public static ColumnGroupModel getColGroupModel(int[][] columnGroups) {
     int[] columnSplit = new int[columnGroups.length];
     int noOfColumnStore = columnSplit.length;
-    boolean[] columnarStore = new boolean[noOfColumnStore];
 
     for (int i = 0; i < columnGroups.length; i++) {
       columnSplit[i] = columnGroups[i].length;
-      columnarStore[i] = columnGroups[i].length <= 1;
     }
     ColumnGroupModel colGroupModel = new ColumnGroupModel();
     colGroupModel.setNoOfColumnStore(noOfColumnStore);
     colGroupModel.setColumnSplit(columnSplit);
-    colGroupModel.setColumnarStore(columnarStore);
     colGroupModel.setColumnGroup(columnGroups);
     return colGroupModel;
   }
@@ -413,7 +410,7 @@ public final class CarbonUtil {
     }
   }
 
-  public static int getFirstIndexUsingBinarySearch(DimensionColumnDataChunk dimColumnDataChunk,
+  public static int getFirstIndexUsingBinarySearch(DimensionColumnPage dimColumnDataChunk,
       int low, int high, byte[] compareValue, boolean matchUpLimit) {
     int cmpResult = 0;
     while (high >= low) {
@@ -452,7 +449,7 @@ public final class CarbonUtil {
    * @return the compareValue's range index in the dimColumnDataChunk
    */
   public static int[] getRangeIndexUsingBinarySearch(
-      DimensionColumnDataChunk dimColumnDataChunk, int low, int high, byte[] compareValue) {
+      DimensionColumnPage dimColumnDataChunk, int low, int high, byte[] compareValue) {
 
     int[] rangeIndex = new int[2];
     int cmpResult = 0;
@@ -546,7 +543,7 @@ public final class CarbonUtil {
    * @return index value
    */
   public static int nextLesserValueToTarget(int currentIndex,
-      DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue) {
+      DimensionColumnPage dimColumnDataChunk, byte[] compareValue) {
     while (currentIndex - 1 >= 0
         && dimColumnDataChunk.compareTo(currentIndex - 1, compareValue) >= 0) {
       --currentIndex;
@@ -566,7 +563,7 @@ public final class CarbonUtil {
    * @return index value
    */
   public static int nextGreaterValueToTarget(int currentIndex,
-      DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue, int numerOfRows) {
+      DimensionColumnPage dimColumnDataChunk, byte[] compareValue, int numerOfRows) {
     while (currentIndex + 1 < numerOfRows
         && dimColumnDataChunk.compareTo(currentIndex + 1, compareValue) <= 0) {
       ++currentIndex;
@@ -929,7 +926,7 @@ public final class CarbonUtil {
     return false;
   }
 
-  public static boolean[] getDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getDictionaryEncodingArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -938,7 +935,7 @@ public final class CarbonUtil {
     return dictionaryEncodingArray;
   }
 
-  public static boolean[] getDirectDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getDirectDictionaryEncodingArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -947,7 +944,7 @@ public final class CarbonUtil {
     return dictionaryEncodingArray;
   }
 
-  public static boolean[] getImplicitColumnArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getImplicitColumnArray(ProjectionDimension[] queryDimensions) {
     boolean[] implicitColumnArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       implicitColumnArray[i] = queryDimensions[i].getDimension().hasEncoding(Encoding.IMPLICIT);
@@ -955,7 +952,7 @@ public final class CarbonUtil {
     return implicitColumnArray;
   }
 
-  public static boolean[] getComplexDataTypeArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getComplexDataTypeArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -985,7 +982,6 @@ public final class CarbonUtil {
       fileFooter.setColumnInTable(schema);
       SegmentInfo segmentInfo = new SegmentInfo();
       segmentInfo.setColumnCardinality(detailInfo.getDimLens());
-      segmentInfo.setNumberOfColumns(detailInfo.getRowCount());
       fileFooter.setSegmentInfo(segmentInfo);
       return fileFooter;
     }
@@ -1025,7 +1021,7 @@ public final class CarbonUtil {
    * @return
    */
   public static long calculateMetaSize(TableBlockInfo tableBlockInfo) throws IOException {
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();
       long footerPointer = completeBlockLength - 8;
@@ -2156,19 +2152,19 @@ public final class CarbonUtil {
   /**
    * Below method will be used to check filter value is present in the data chunk or not
    * @param filterValues
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param low
    * @param high
    * @param chunkRowIndex
    * @return
    */
   public static int isFilterPresent(byte[][] filterValues,
-      DimensionColumnDataChunk dimensionColumnDataChunk, int low, int high, int chunkRowIndex) {
+      DimensionColumnPage dimensionColumnPage, int low, int high, int chunkRowIndex) {
     int compareResult = 0;
     int mid = 0;
     while (low <= high) {
       mid = (low + high) >>> 1;
-      compareResult = dimensionColumnDataChunk.compareTo(chunkRowIndex, filterValues[mid]);
+      compareResult = dimensionColumnPage.compareTo(chunkRowIndex, filterValues[mid]);
       if (compareResult < 0) {
         high = mid - 1;
       } else if (compareResult > 0) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
index e61b477..d665379 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
@@ -45,7 +45,7 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
   @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
       throws IOException {
     DataFileFooter dataFileFooter = new DataFileFooter();
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();
       long footerPointer = completeBlockLength - 8;
@@ -123,7 +123,7 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
   }
 
   @Override public List<ColumnSchema> getSchema(TableBlockInfo tableBlockInfo) throws IOException {
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
index 74fd09a..afacc0b 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
@@ -36,17 +36,6 @@ public interface CarbonDictionaryWriter extends Closeable {
   void write(String value) throws IOException;
 
   /**
-   * write method that accepts one value at a time
-   * This method can be used when data is huge and memory is les. In that
-   * case data can be stored to a file and an iterator can iterate over it and
-   * pass one value at a time
-   *
-   * @param value unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  void write(byte[] value) throws IOException;
-
-  /**
    * write method that accepts list of byte arrays as value
    * This can be used when data is less, then string can be converted
    * to byte array for each value and added to a list

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index 82baccc..53411e9 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -144,7 +144,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * @param value unique dictionary value
    * @throws IOException if an I/O error occurs
    */
-  @Override public void write(byte[] value) throws IOException {
+  private void write(byte[] value) throws IOException {
     if (isFirstTime) {
       init();
       isFirstTime = false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
index 8bf8a07..08dd791 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
@@ -87,7 +87,7 @@ public class DictionaryCacheLoaderImplTest {
         return 9999;
       }
     };
-    dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true);
+    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
     assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 4);
   }
 
@@ -97,7 +97,7 @@ public class DictionaryCacheLoaderImplTest {
         return 10000;
       }
     };
-    dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true);
+    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
     assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 2);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
deleted file mode 100644
index 5c51c87..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import org.apache.carbondata.core.datastore.impl.DFSFileHolderImpl;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-public class DFSFileHolderImplUnitTest {
-
-  private static DFSFileHolderImpl dfsFileHolder;
-  private static String fileName;
-  private static String fileNameWithEmptyContent;
-  private static File file;
-  private static File fileWithEmptyContent;
-
-  @BeforeClass public static void setup() {
-    dfsFileHolder = new DFSFileHolderImpl();
-    file = new File("Test.carbondata");
-    fileWithEmptyContent = new File("TestEXception.carbondata");
-
-    if (!file.exists()) try {
-      file.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    if (!fileWithEmptyContent.exists()) try {
-      fileWithEmptyContent.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    try {
-      FileOutputStream of = new FileOutputStream(file, true);
-      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
-      br.write("Hello World");
-      br.close();
-    } catch (Exception e) {
-      e.getMessage();
-    }
-    fileName = file.getAbsolutePath();
-    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
-  }
-
-  @AfterClass public static void tearDown() throws IOException  {
-    file.delete();
-    fileWithEmptyContent.delete();
-    dfsFileHolder.finish();
-  }
-
-  @Test public void testReadByteArray() throws IOException  {
-    byte[] result = dfsFileHolder.readByteArray(fileName, 1);
-    byte[] expected_result = new byte[] { 72 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadByteArrayWithFilePath() throws IOException  {
-    byte[] result = dfsFileHolder.readByteArray(fileName, 2L, 2);
-    byte[] expected_result = { 108, 108 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadLong() throws IOException  {
-    long actualResult = dfsFileHolder.readLong(fileName, 1L);
-    long expectedResult = 7308335519855243122L;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadLongForIoException() throws IOException {
-    dfsFileHolder.readLong(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadIntForIoException() throws IOException{
-    dfsFileHolder.readInt(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadInt() throws IOException  {
-    int actualResult = dfsFileHolder.readInt(fileName, 1L);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileName() throws IOException {
-    int actualResult = dfsFileHolder.readInt(fileName);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadIntWithFileNameForIOException() throws IOException {
-    dfsFileHolder.readInt(fileNameWithEmptyContent);
-  }
-
-  @Test public void testDouble() throws IOException  {
-    double actualResult = dfsFileHolder.readDouble(fileName, 1L);
-    double expectedResult = 7.3083355198552433E18;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testDoubleForIoException() throws IOException {
-    dfsFileHolder.readDouble(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test
-  public void testDoubleForIoExceptionwithUpdateCache() throws IOException {
-    new MockUp<FileSystem>() {
-      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
-          throws IOException {
-        throw new IOException();
-      }
-
-    };
-    dfsFileHolder.readDouble(fileName, 1L);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
new file mode 100644
index 0000000..da61a94
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+
+import org.apache.carbondata.core.datastore.impl.DFSFileReaderImpl;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class DFSFileReaderImplUnitTest {
+
+  private static DFSFileReaderImpl dfsFileHolder;
+  private static String fileName;
+  private static String fileNameWithEmptyContent;
+  private static File file;
+  private static File fileWithEmptyContent;
+
+  @BeforeClass public static void setup() {
+    dfsFileHolder = new DFSFileReaderImpl();
+    file = new File("Test.carbondata");
+    fileWithEmptyContent = new File("TestEXception.carbondata");
+
+    if (!file.exists()) try {
+      file.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    if (!fileWithEmptyContent.exists()) try {
+      fileWithEmptyContent.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    try {
+      FileOutputStream of = new FileOutputStream(file, true);
+      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
+      br.write("Hello World");
+      br.close();
+    } catch (Exception e) {
+      e.getMessage();
+    }
+    fileName = file.getAbsolutePath();
+    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
+  }
+
+  @AfterClass public static void tearDown() throws IOException  {
+    file.delete();
+    fileWithEmptyContent.delete();
+    dfsFileHolder.finish();
+  }
+
+  @Test public void testReadByteArray() throws IOException  {
+    byte[] result = dfsFileHolder.readByteArray(fileName, 1);
+    byte[] expected_result = new byte[] { 72 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadByteArrayWithFilePath() throws IOException  {
+    byte[] result = dfsFileHolder.readByteArray(fileName, 2L, 2);
+    byte[] expected_result = { 108, 108 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadLong() throws IOException  {
+    long actualResult = dfsFileHolder.readLong(fileName, 1L);
+    long expectedResult = 7308335519855243122L;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test(expected = IOException.class)
+  public void testReadLongForIoException() throws IOException {
+    dfsFileHolder.readLong(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test(expected = IOException.class)
+  public void testReadIntForIoException() throws IOException{
+    dfsFileHolder.readInt(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadInt() throws IOException  {
+    int actualResult = dfsFileHolder.readInt(fileName, 1L);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileName() throws IOException {
+    int actualResult = dfsFileHolder.readInt(fileName);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test(expected = IOException.class)
+  public void testReadIntWithFileNameForIOException() throws IOException {
+    dfsFileHolder.readInt(fileNameWithEmptyContent);
+  }
+
+  @Test public void testDouble() throws IOException  {
+    double actualResult = dfsFileHolder.readDouble(fileName, 1L);
+    double expectedResult = 7.3083355198552433E18;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test(expected = IOException.class)
+  public void testDoubleForIoException() throws IOException {
+    dfsFileHolder.readDouble(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test
+  public void testDoubleForIoExceptionwithUpdateCache() throws IOException {
+    new MockUp<FileSystem>() {
+      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
+          throws IOException {
+        throw new IOException();
+      }
+
+    };
+    dfsFileHolder.readDouble(fileName, 1L);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
deleted file mode 100644
index ed50d63..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import org.apache.carbondata.core.datastore.impl.FileHolderImpl;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-
-public class FileHolderImplUnitTest {
-
-  private static FileHolderImpl fileHolder;
-  private static FileHolderImpl fileHolderWithCapacity;
-  private static String fileName;
-  private static String fileNameWithEmptyContent;
-  private static File file;
-  private static File fileWithEmptyContent;
-
-  @BeforeClass public static void setup() {
-    fileHolder = new FileHolderImpl();
-    fileHolderWithCapacity = new FileHolderImpl(50);
-    file = new File("Test.carbondata");
-    fileWithEmptyContent = new File("TestEXception.carbondata");
-
-    if (!file.exists()) try {
-      file.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    if (!fileWithEmptyContent.exists()) try {
-      fileWithEmptyContent.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    try {
-      FileOutputStream of = new FileOutputStream(file, true);
-      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
-      br.write("Hello World");
-      br.close();
-    } catch (Exception e) {
-      e.getMessage();
-    }
-    fileName = file.getAbsolutePath();
-    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
-  }
-
-  @AfterClass public static void tearDown() throws IOException {
-    file.delete();
-    fileWithEmptyContent.delete();
-    fileHolder.finish();
-  }
-
-  @Test public void testReadByteArray() throws IOException  {
-    byte[] result = fileHolder.readByteArray(fileName, 1);
-    byte[] expected_result = new byte[] { 72 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadByteArrayWithFilePath() throws IOException  {
-    byte[] result = fileHolder.readByteArray(fileName, 2L, 2);
-    byte[] expected_result = { 108, 108 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadLong() throws IOException  {
-    long actualResult = fileHolder.readLong(fileName, 1L);
-    long expectedResult = 7308335519855243122L;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadLongForIoException() throws IOException {
-    fileHolder.readLong(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadIntForIoException() throws IOException {
-    fileHolder.readInt(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadInt() throws IOException  {
-    int actualResult = fileHolder.readInt(fileName, 1L);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileName() throws IOException  {
-    int actualResult = fileHolder.readInt(fileName);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileNameForIOException() throws IOException  {
-    fileHolder.readInt(fileNameWithEmptyContent);
-
-  }
-
-  @Test public void testDouble() throws IOException  {
-    double actualResult = fileHolder.readDouble(fileName, 1L);
-    double expectedResult = 7.3083355198552433E18;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testDoubleForIoException() throws IOException {
-    fileHolder.readDouble(fileNameWithEmptyContent, 1L);
-
-  }
-
-  @Test public void testDoubleForIoExceptionwithUpdateCache() throws Exception {
-    new MockUp<FileSystem>() {
-      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
-          throws IOException {
-        throw new IOException();
-      }
-
-    };
-    try {
-      fileHolder.readDouble(fileName, 1L);
-    } catch (Exception e) {
-      assertNull(e.getMessage());
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
new file mode 100644
index 0000000..a6d3235
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+
+import org.apache.carbondata.core.datastore.impl.FileReaderImpl;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+public class FileReaderImplUnitTest {
+
+  private static FileReaderImpl fileHolder;
+  private static FileReaderImpl fileHolderWithCapacity;
+  private static String fileName;
+  private static String fileNameWithEmptyContent;
+  private static File file;
+  private static File fileWithEmptyContent;
+
+  @BeforeClass public static void setup() {
+    fileHolder = new FileReaderImpl();
+    fileHolderWithCapacity = new FileReaderImpl(50);
+    file = new File("Test.carbondata");
+    fileWithEmptyContent = new File("TestEXception.carbondata");
+
+    if (!file.exists()) try {
+      file.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    if (!fileWithEmptyContent.exists()) try {
+      fileWithEmptyContent.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    try {
+      FileOutputStream of = new FileOutputStream(file, true);
+      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
+      br.write("Hello World");
+      br.close();
+    } catch (Exception e) {
+      e.getMessage();
+    }
+    fileName = file.getAbsolutePath();
+    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
+  }
+
+  @AfterClass public static void tearDown() throws IOException {
+    file.delete();
+    fileWithEmptyContent.delete();
+    fileHolder.finish();
+  }
+
+  @Test public void testReadByteArray() throws IOException  {
+    byte[] result = fileHolder.readByteArray(fileName, 1);
+    byte[] expected_result = new byte[] { 72 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadByteArrayWithFilePath() throws IOException  {
+    byte[] result = fileHolder.readByteArray(fileName, 2L, 2);
+    byte[] expected_result = { 108, 108 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadLong() throws IOException  {
+    long actualResult = fileHolder.readLong(fileName, 1L);
+    long expectedResult = 7308335519855243122L;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadLongForIoException() throws IOException {
+    fileHolder.readLong(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadIntForIoException() throws IOException {
+    fileHolder.readInt(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadInt() throws IOException  {
+    int actualResult = fileHolder.readInt(fileName, 1L);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileName() throws IOException  {
+    int actualResult = fileHolder.readInt(fileName);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileNameForIOException() throws IOException  {
+    fileHolder.readInt(fileNameWithEmptyContent);
+
+  }
+
+  @Test public void testDouble() throws IOException  {
+    double actualResult = fileHolder.readDouble(fileName, 1L);
+    double expectedResult = 7.3083355198552433E18;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testDoubleForIoException() throws IOException {
+    fileHolder.readDouble(fileNameWithEmptyContent, 1L);
+
+  }
+
+  @Test public void testDoubleForIoExceptionwithUpdateCache() throws Exception {
+    new MockUp<FileSystem>() {
+      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
+          throws IOException {
+        throw new IOException();
+      }
+
+    };
+    try {
+      fileHolder.readDouble(fileName, 1L);
+    } catch (Exception e) {
+      assertNull(e.getMessage());
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
index 9c43553..20036ec 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
@@ -84,7 +84,7 @@ public class SegmentPropertiesTest extends TestCase {
     assertTrue(true);
   }
 
-  @Test public void testBlockMetadataHasProperDimensionBlockMapping() {
+  @Test public void testBlockMetadataHasProperDimensionChunkMapping() {
     Map<Integer, Integer> dimensionOrdinalToBlockMapping = new HashMap<Integer, Integer>();
     dimensionOrdinalToBlockMapping.put(0, 0);
     dimensionOrdinalToBlockMapping.put(1, 1);
@@ -97,7 +97,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimensionOrdinalToBlockMapping.put(8, 5);
     dimensionOrdinalToBlockMapping.put(9, 6);
     Map<Integer, Integer> dimensionOrdinalToBlockMappingActual =
-        blockMetadataInfos.getDimensionOrdinalToBlockMapping();
+        blockMetadataInfos.getDimensionOrdinalToChunkMapping();
     assertEquals(dimensionOrdinalToBlockMapping.size(),
         dimensionOrdinalToBlockMappingActual.size());
     Iterator<Entry<Integer, Integer>> iterator =
@@ -112,12 +112,12 @@ public class SegmentPropertiesTest extends TestCase {
     assertTrue(true);
   }
 
-  @Test public void testBlockMetadataHasProperMeasureBlockMapping() {
+  @Test public void testBlockMetadataHasProperMeasureChunkMapping() {
     Map<Integer, Integer> measureOrdinalToBlockMapping = new HashMap<Integer, Integer>();
     measureOrdinalToBlockMapping.put(0, 0);
     measureOrdinalToBlockMapping.put(1, 1);
     Map<Integer, Integer> measureOrdinalToBlockMappingActual =
-        blockMetadataInfos.getMeasuresOrdinalToBlockMapping();
+        blockMetadataInfos.getMeasuresOrdinalToChunkMapping();
     assertEquals(measureOrdinalToBlockMapping.size(), measureOrdinalToBlockMappingActual.size());
     Iterator<Entry<Integer, Integer>> iterator = measureOrdinalToBlockMapping.entrySet().iterator();
     while (iterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
index bdb83cd..54b66a6 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
@@ -37,7 +37,7 @@ import org.junit.Test;
 
 public class ColumnGroupDimensionDataChunkTest {
 
-  static ColumnGroupDimensionDataChunk columnGroupDimensionDataChunk;
+  static ColumnGroupDimensionColumnPage columnGroupDimensionDataChunk;
   static KeyGenerator keyGenerator;
 
   @BeforeClass public static void setup() {
@@ -56,7 +56,7 @@ public class ColumnGroupDimensionDataChunkTest {
       position += keyGenerator.getKeySizeInBytes();
     }
     columnGroupDimensionDataChunk =
-        new ColumnGroupDimensionDataChunk(data, keyGenerator.getKeySizeInBytes(), 3);
+        new ColumnGroupDimensionColumnPage(data, keyGenerator.getKeySizeInBytes(), 3);
   }
 
   @Test public void fillChunkDataTest() {
@@ -64,7 +64,7 @@ public class ColumnGroupDimensionDataChunkTest {
     ordinals.add(1);
     KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator);
     byte[] buffer = new byte[1];
-    columnGroupDimensionDataChunk.fillChunkData(buffer, 0, 1, keyStructureInfo);
+    columnGroupDimensionDataChunk.fillRawData(1, 0, buffer, keyStructureInfo);
     assertEquals(buffer[0], 2);
   }
 
@@ -81,7 +81,7 @@ public class ColumnGroupDimensionDataChunkTest {
     ordinals.add(2);
     KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator);
     keyStructureInfo.setMdkeyQueryDimensionOrdinal(new int[] { 2 });
-    int res = columnGroupDimensionDataChunk.fillConvertedChunkData(2, 2, row, keyStructureInfo);
+    int res = columnGroupDimensionDataChunk.fillSurrogateKey(2, 2, row, keyStructureInfo);
     Assert.assertTrue(Arrays.equals(row, expected));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
index 04b7a80..f327ef6 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
@@ -30,7 +30,7 @@ import org.junit.Test;
 
 public class FixedLengthDimensionDataChunkTest {
 
-  static FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+  static FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
   static byte[] data;
 
   @BeforeClass public static void setup() {
@@ -40,7 +40,7 @@ public class FixedLengthDimensionDataChunkTest {
 
     int invertedIndexReverse[] = { 1, 0, 5, 7, 8 };
     fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(data, invertedIndex, invertedIndexReverse, 5, 4);
+        new FixedLengthDimensionColumnPage(data, invertedIndex, invertedIndexReverse, 5, 4);
   }
 
   @Test public void fillChunkDataTest() {
@@ -48,7 +48,7 @@ public class FixedLengthDimensionDataChunkTest {
     int[] maskByteRanges = { 1, 2, 4, 6, 5 };
     keyStructureInfo.setMaskByteRanges(maskByteRanges);
     keyStructureInfo.setMaxKey("1234567".getBytes());
-    int res = fixedLengthDimensionDataChunk.fillChunkData(data, 0, 0, keyStructureInfo);
+    int res = fixedLengthDimensionDataChunk.fillRawData(0, 0, data, keyStructureInfo);
     int expectedResult = 4 ;
     assertEquals(res, expectedResult);
   }
@@ -62,7 +62,7 @@ public class FixedLengthDimensionDataChunkTest {
   @Test public void fillConvertedChunkDataTest() {
     int[] row = { 1, 2, 4, 6 };
     KeyStructureInfo keyStructureInfo = new KeyStructureInfo();
-    int res = fixedLengthDimensionDataChunk.fillConvertedChunkData(1, 0, row, keyStructureInfo);
+    int res = fixedLengthDimensionDataChunk.fillSurrogateKey(1, 0, row, keyStructureInfo);
     int expectedResult = 1;
     assertEquals(res, expectedResult);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
index d874037..7f7719c 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
@@ -76,9 +76,9 @@ public class BTreeBlockFinderTest extends TestCase {
     buffer.array();
     IndexKey key = new IndexKey(null, buffer.array());
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(1, findFirstBlock.nodeNumber());
+    assertEquals(1, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(1, findLastBlock.nodeNumber());
+    assertEquals(1, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithNoDictionary() {
@@ -96,9 +96,9 @@ public class BTreeBlockFinderTest extends TestCase {
     buffer.array();
     IndexKey key = new IndexKey(null, buffer.array());
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey1()
@@ -118,9 +118,9 @@ public class BTreeBlockFinderTest extends TestCase {
     IndexKey key =
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 1, 1 }), null);
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey2()
@@ -141,9 +141,9 @@ public class BTreeBlockFinderTest extends TestCase {
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 0, 0 }), null);
 
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   /**
@@ -169,9 +169,9 @@ public class BTreeBlockFinderTest extends TestCase {
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 10001, 10001 }), null);
 
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(99, findFirstBlock.nodeNumber());
+    assertEquals(99, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(99, findLastBlock.nodeNumber());
+    assertEquals(99, findLastBlock.nodeIndex());
   }
 
   private List<DataFileFooter> getDataFileFooterList() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
index 85bdfd8..be91410 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
@@ -50,32 +50,32 @@ public class RawBasedResultCollectorTest {
 //    aggregatorInfo.setDefaultValues(new Object[] { 1, 2, 3, 4 });
 //    aggregatorInfo.setMeasureDataTypes(
 //        new DataType[] { DataTypes.INT, DataTypes.TIMESTAMP, DataTypes.INT, DataTypes.INT });
-//    QueryMeasure queryMeasure1 = new QueryMeasure("QMCol1");
+//    ProjectionMeasure queryMeasure1 = new ProjectionMeasure("QMCol1");
 //    queryMeasure1.setQueryOrder(1);
-//    QueryMeasure queryMeasure2 = new QueryMeasure("QMCol2");
+//    ProjectionMeasure queryMeasure2 = new ProjectionMeasure("QMCol2");
 //    queryMeasure1.setQueryOrder(2);
-//    QueryMeasure queryMeasure3 = new QueryMeasure("QMCol3");
+//    ProjectionMeasure queryMeasure3 = new ProjectionMeasure("QMCol3");
 //    queryMeasure1.setQueryOrder(3);
-//    QueryMeasure queryMeasure4 = new QueryMeasure("QMCol4");
+//    ProjectionMeasure queryMeasure4 = new ProjectionMeasure("QMCol4");
 //    queryMeasure1.setQueryOrder(4);
-//    QueryDimension queryDimension1 = new QueryDimension("QDCol1");
+//    ProjectionDimension queryDimension1 = new ProjectionDimension("QDCol1");
 //    queryDimension1.setQueryOrder(1);
 //    ColumnSchema columnSchema = new ColumnSchema();
 //    queryDimension1.setDimension(new CarbonDimension(columnSchema, 0, 0, 0, 0));
-//    QueryDimension queryDimension2 = new QueryDimension("QDCol2");
+//    ProjectionDimension queryDimension2 = new ProjectionDimension("QDCol2");
 //    queryDimension2.setQueryOrder(2);
 //    queryDimension2.setDimension(new CarbonDimension(columnSchema, 1, 1, 1, 1));
-//    QueryDimension queryDimension3 = new QueryDimension("QDCol3");
+//    ProjectionDimension queryDimension3 = new ProjectionDimension("QDCol3");
 //    queryDimension3.setQueryOrder(3);
 //    queryDimension3.setDimension(new CarbonDimension(columnSchema, 2, 0, 0, 0));
-//    QueryDimension queryDimension4 = new QueryDimension("QDCol4");
+//    ProjectionDimension queryDimension4 = new ProjectionDimension("QDCol4");
 //    queryDimension4.setQueryOrder(4);
 //    queryDimension4.setDimension(new CarbonDimension(columnSchema, 3, 0, 0, 0));
-//    blockExecutionInfo.setQueryDimensions(
-//        new QueryDimension[] { queryDimension1, queryDimension2, queryDimension3,
+//    blockExecutionInfo.setProjectionDimensions(
+//        new ProjectionDimension[] { queryDimension1, queryDimension2, queryDimension3,
 //            queryDimension4 });
-//    blockExecutionInfo.setQueryMeasures(
-//        new QueryMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 });
+//    blockExecutionInfo.setProjectionMeasures(
+//        new ProjectionMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 });
 //    blockExecutionInfo.setFixedKeyUpdateRequired(true);
 //    blockExecutionInfo.setMeasureInfo(aggregatorInfo);
 //    blockExecutionInfo.setMaskedByteForBlock(new int[] { 1, 2 });
@@ -99,7 +99,7 @@ public class RawBasedResultCollectorTest {
 //        return new byte[][] { { 1, 2 }, { 1, 2 } };
 //      }
 //
-//      @SuppressWarnings("unused") @Mock public ColumnPage getMeasureChunk(int ordinal) {
+//      @SuppressWarnings("unused") @Mock public ColumnPage readMeasureChunk(int ordinal) {
 //        ColumnPage ColumnPage = new ColumnPage();
 //        PresenceMeta presenceMeta = new PresenceMeta();
 //        BitSet bitSet = new BitSet();
@@ -120,10 +120,10 @@ public class RawBasedResultCollectorTest {
 //      }
 //    };
 //
-//    AbstractScannedResult abstractScannedResult =
+//    BlockletScannedResult abstractScannedResult =
 //        new NonFilterQueryScannedResult(blockExecutionInfo);
-//    abstractScannedResult.setNumberOfRows(2);
-//    List<Object[]> result = rawBasedResultCollector.collectData(abstractScannedResult, 2);
+//    abstractScannedResult.setPageFilteredRowCount(2);
+//    List<Object[]> result = rawBasedResultCollector.collectResultInRow(abstractScannedResult, 2);
 //    int expectedResult = 2;
 //    assertThat(result.size(), is(equalTo(expectedResult)));
 //  }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
index 45e86f2..e8b75b7 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesTestUtil;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import junit.framework.TestCase;
 import mockit.Mock;
@@ -45,9 +45,8 @@ public class QueryUtilTest extends TestCase {
 
   @Test public void testGetMaskedByteRangeGivingProperMaksedByteRange() {
 
-    QueryDimension dimension =
-        new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
+    ProjectionDimension dimension =
+        new ProjectionDimension(segmentProperties.getDimensions().get(0));
     int[] maskedByteRange = QueryUtil
         .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
     int[] expectedMaskedByteRange = { 0 };
@@ -57,11 +56,10 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaskedByteRangeGivingProperMaksedByteRangeOnlyForDictionaryKey() {
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       dimensions.add(dimension);
     }
     int[] maskedByteRange =
@@ -84,11 +82,10 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaxKeyBasedOnDimensions() {
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       dimensions.add(dimension);
     }
     byte[] maxKeyBasedOnDimensions = null;
@@ -115,10 +112,8 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaksedByte() {
-    QueryDimension dimension =
-        new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
+    ProjectionDimension dimension =
+        new ProjectionDimension(segmentProperties.getDimensions().get(0));
     int[] maskedByteRange = QueryUtil
         .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
     int[] maskedByte = QueryUtil
@@ -203,18 +198,16 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetSortDimensionIndexes() {
-    List<QueryDimension> sortedDimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> sortedDimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       sortedDimensions.add(dimension);
     }
-    List<QueryDimension> queryDimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> queryDimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       queryDimensions.add(dimension);
     }
     byte[] actualValue = QueryUtil.getSortDimensionIndexes(sortedDimensions, queryDimensions);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
index 172a53e..bdf81e7 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
@@ -28,8 +28,8 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 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.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.junit.Test;
 
@@ -79,22 +79,20 @@ public class RestructureUtilTest {
     List<CarbonDimension> tableComplexDimensions =
         Arrays.asList(tableComplexDimension1, tableComplexDimension2);
 
-    QueryDimension queryDimension1 = new QueryDimension("Id");
-    queryDimension1.setDimension(tableBlockDimension1);
-    QueryDimension queryDimension2 = new QueryDimension("Name");
-    queryDimension2.setDimension(tableComplexDimension2);
-    QueryDimension queryDimension3 = new QueryDimension("Address");
-    queryDimension3.setDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3));
+    ProjectionDimension queryDimension1 = new ProjectionDimension(tableBlockDimension1);
+    ProjectionDimension queryDimension2 = new ProjectionDimension(tableComplexDimension2);
+    ProjectionDimension
+        queryDimension3 = new ProjectionDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3));
 
-    List<QueryDimension> queryDimensions =
+    List<ProjectionDimension> queryDimensions =
         Arrays.asList(queryDimension1, queryDimension2, queryDimension3);
 
-    List<QueryDimension> result = null;
+    List<ProjectionDimension> result = null;
     result = RestructureUtil
         .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo, queryDimensions,
             tableBlockDimensions, tableComplexDimensions);
     List<CarbonDimension> resultDimension = new ArrayList<>(result.size());
-    for (QueryDimension queryDimension : result) {
+    for (ProjectionDimension queryDimension : result) {
       resultDimension.add(queryDimension.getDimension());
     }
     assertThat(resultDimension,
@@ -121,13 +119,10 @@ public class RestructureUtilTest {
     carbonMeasure3.getColumnSchema().setDefaultValue("3".getBytes());
     List<CarbonMeasure> currentBlockMeasures = Arrays.asList(carbonMeasure1, carbonMeasure2);
 
-    QueryMeasure queryMeasure1 = new QueryMeasure("Id");
-    queryMeasure1.setMeasure(carbonMeasure1);
-    QueryMeasure queryMeasure2 = new QueryMeasure("Name");
-    queryMeasure2.setMeasure(carbonMeasure2);
-    QueryMeasure queryMeasure3 = new QueryMeasure("Age");
-    queryMeasure3.setMeasure(carbonMeasure3);
-    List<QueryMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3);
+    ProjectionMeasure queryMeasure1 = new ProjectionMeasure(carbonMeasure1);
+    ProjectionMeasure queryMeasure2 = new ProjectionMeasure(carbonMeasure2);
+    ProjectionMeasure queryMeasure3 = new ProjectionMeasure(carbonMeasure3);
+    List<ProjectionMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3);
     BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
     RestructureUtil.createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo, queryMeasures,
         currentBlockMeasures);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index 89b3122..cbb939b 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -325,7 +325,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
         return "test";
       }
     };
-    assertTrue(FilterUtil.getFilterListForRS(expression, columnExpression, defaultValues,
+    assertTrue(FilterUtil.getFilterListForRS(expression, defaultValues,
         defaultSurrogate) instanceof ColumnFilterInfo);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
index e3ae42c..11c4980 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
@@ -18,12 +18,12 @@ package org.apache.carbondata.core.scan.filter.executer;
 
 import java.util.BitSet;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest {
 
- @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimColumnDataChunk,
+ @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimColumnDataChunk,
      int numerOfRows, byte[][] filterValues) {
    BitSet bitSet = new BitSet(numerOfRows);
    bitSet.flip(0, numerOfRows);
@@ -46,7 +46,7 @@ public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest
    return bitSet;
  }
 
- @Override public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimColumnDataChunk,
+ @Override public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     bitSet.flip(0, numerOfRows);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
index 29dda52..1bfa875 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
@@ -18,8 +18,8 @@ package org.apache.carbondata.core.scan.filter.executer;
 
 import java.util.BitSet;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.junit.Assert;
@@ -38,22 +38,22 @@ public class IncludeFilterExecuterImplTest extends TestCase {
 
   }
 
-  public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimensionColumnDataChunk,
+  public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimensionColumnPage,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
-    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
       // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
       if (filterValues.length > 1) {
         for (int i = 0; i < numerOfRows; i++) {
           int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1,
-              dimensionColumnDataChunk.getChunkData(i));
+              dimensionColumnPage.getChunkData(i));
           if (index >= 0) {
             bitSet.set(i);
           }
         }
       } else if (filterValues.length == 1) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (dimensionColumnDataChunk.compareTo(i, filterValues[0]) == 0) {
+          if (dimensionColumnPage.compareTo(i, filterValues[0]) == 0) {
             bitSet.set(i);
           }
         }
@@ -62,14 +62,14 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     return bitSet;
   }
 
-  public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows,
+  public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows,
       byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
-    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
       // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
       for (int k = 0; k < filterValues.length; k++) {
         for (int j = 0; j < numerOfRows; j++) {
-          if (dimensionColumnDataChunk.compareTo(j, filterValues[k]) == 0) {
+          if (dimensionColumnPage.compareTo(j, filterValues[k]) == 0) {
             bitSet.set(j);
           }
         }
@@ -163,7 +163,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     System.out.println("dimColumnSize: " + dimColumnSize);
     
-    FixedLengthDimensionDataChunk dimensionColumnDataChunk;
+    FixedLengthDimensionColumnPage dimensionColumnDataChunk;
     DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo();
 
     byte[] dataChunk = new byte[dataChunkSize * dimColumnSize];
@@ -183,7 +183,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     dim.setFilterKeys(filterKeys);
 
-    dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunkSize, dimColumnSize);
 
     // repeat query and compare 2 result between old code and new optimized code
@@ -194,7 +194,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
       oldTime = oldTime + System.currentTimeMillis() - start;
 
       start = System.currentTimeMillis();
-      BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionDataChunk) dimensionColumnDataChunk, dataChunkSize,
+      BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionColumnPage) dimensionColumnDataChunk, dataChunkSize,
           filterKeys);
       newTime = newTime + System.currentTimeMillis() - start;
 
@@ -212,7 +212,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
   }
 
 
-  private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionColumnPage dimensionColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
@@ -243,7 +243,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     return bitSet;
   }
 
-  private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionColumnPage dimensionColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     int startIndex = 0;
@@ -281,7 +281,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     int filteredValueCnt = 800;
     // column dictionary size
     int dimColumnSize = 2;
-    FixedLengthDimensionDataChunk dimensionColumnDataChunk;
+    FixedLengthDimensionColumnPage dimensionColumnDataChunk;
     DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo();
 
     byte[] dataChunk = new byte[dataChunkSize * dimColumnSize];
@@ -303,7 +303,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     dim.setFilterKeys(filterKeys);
 
-    dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / dimColumnSize, dimColumnSize);
 
     // initial to run

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
deleted file mode 100644
index 541ce48..0000000
--- a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
+++ /dev/null
@@ -1,94 +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.scan.result;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-public class BatchResultTest {
-  private static BatchResult batchResult;
-  private static List<Object[]> rowsList = new ArrayList(2);
-
-  @BeforeClass public static void setUp() {
-    batchResult = new BatchResult();
-    rowsList.add(0, new Integer[] { 1, 2 });
-    rowsList.add(1, new Integer[] { 3 });
-  }
-
-  @Test public void testNext() throws NoSuchElementException {
-    BatchResult rows = new BatchResult();
-    rows.setRows(rowsList);
-    Object[] result = rows.next();
-    Assert.assertTrue(result.equals(rowsList.get(0)));
-  }
-
-  @Test(expected = NoSuchElementException.class) public void testNextWithNoSuchElementException() {
-    BatchResult rows = new BatchResult();
-    List emptyList = new ArrayList(2);
-    rows.setRows(emptyList);
-    rows.next();
-  }
-
-  @Test public void testGetRows() {
-    new MockUp<BatchResult>() {
-      @Mock public void $init() {
-        //to be left blank
-      }
-    };
-    BatchResult batchResult = new BatchResult();
-    List<Object[]> list = batchResult.getRows();
-    assertNull("Number of rows is null", list);
-  }
-
-  @Test public void testHasNext() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    list.add(1, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    boolean result = batchResult.hasNext();
-    Assert.assertTrue(result);
-  }
-
-  @Test public void testGetRawRow() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    Object[] actualValue = batchResult.getRawRow(0);
-    Assert.assertTrue(list.get(0) == actualValue);
-  }
-
-  @Test public void testGetSize() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    list.add(1, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    int actualValue = batchResult.getSize();
-    int expectedValue = 2;
-    assertEquals(expectedValue, actualValue);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
new file mode 100644
index 0000000..123d64e
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.scan.result;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class RowBatchTest {
+  private static RowBatch rowBatch;
+  private static List<Object[]> rowsList = new ArrayList(2);
+
+  @BeforeClass public static void setUp() {
+    rowBatch = new RowBatch();
+    rowsList.add(0, new Integer[] { 1, 2 });
+    rowsList.add(1, new Integer[] { 3 });
+  }
+
+  @Test public void testNext() throws NoSuchElementException {
+    RowBatch rows = new RowBatch();
+    rows.setRows(rowsList);
+    Object[] result = rows.next();
+    Assert.assertTrue(result.equals(rowsList.get(0)));
+  }
+
+  @Test(expected = NoSuchElementException.class) public void testNextWithNoSuchElementException() {
+    RowBatch rows = new RowBatch();
+    List emptyList = new ArrayList(2);
+    rows.setRows(emptyList);
+    rows.next();
+  }
+
+  @Test public void testGetRows() {
+    new MockUp<RowBatch>() {
+      @Mock public void $init() {
+        //to be left blank
+      }
+    };
+    RowBatch rowBatch = new RowBatch();
+    List<Object[]> list = rowBatch.getRows();
+    assertNull("Number of rows is null", list);
+  }
+
+  @Test public void testHasNext() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    list.add(1, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    boolean result = rowBatch.hasNext();
+    Assert.assertTrue(result);
+  }
+
+  @Test public void testGetRawRow() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    Object[] actualValue = rowBatch.getRawRow(0);
+    Assert.assertTrue(list.get(0) == actualValue);
+  }
+
+  @Test public void testGetSize() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    list.add(1, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    int actualValue = rowBatch.getSize();
+    int expectedValue = 2;
+    assertEquals(expectedValue, actualValue);
+  }
+
+}


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 224a69f..c18f090 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -34,8 +34,8 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -58,7 +58,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
@@ -67,20 +67,20 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName());
-  protected List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
-  protected List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
+  List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
+  List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
   protected Expression exp;
   protected AbsoluteTableIdentifier tableIdentifier;
   protected SegmentProperties segmentProperties;
   /**
    * it has index at which given dimension is stored in file
    */
-  protected int[] dimensionBlocksIndex;
+  int[] dimensionChunkIndex;
 
   /**
    * it has index at which given measure is stored in file
    */
-  protected int[] measureBlocksIndex;
+  int[] measureChunkIndex;
 
   private Map<Integer, GenericQueryType> complexDimensionInfoMap;
 
@@ -88,18 +88,18 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * flag to check whether the filter dimension is present in current block list of dimensions.
    * Applicable for restructure scenarios
    */
-  protected boolean[] isDimensionPresentInCurrentBlock;
+  boolean[] isDimensionPresentInCurrentBlock;
 
   /**
    * flag to check whether the filter measure is present in current block list of measures.
    * Applicable for restructure scenarios
    */
-  protected boolean[] isMeasurePresentInCurrentBlock;
+  boolean[] isMeasurePresentInCurrentBlock;
 
   /**
    * is dimension column data is natural sorted
    */
-  protected boolean isNaturalSorted;
+  boolean isNaturalSorted;
 
   /**
    * date direct dictionary generator
@@ -123,10 +123,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     }
     if (this.dimColEvaluatorInfoList.size() > 0) {
       this.isDimensionPresentInCurrentBlock = new boolean[dimColEvaluatorInfoList.size()];
-      this.dimensionBlocksIndex = new int[dimColEvaluatorInfoList.size()];
+      this.dimensionChunkIndex = new int[dimColEvaluatorInfoList.size()];
     } else {
       this.isDimensionPresentInCurrentBlock = new boolean[]{false};
-      this.dimensionBlocksIndex = new int[]{0};
+      this.dimensionChunkIndex = new int[]{0};
     }
     if (null == msrColEvalutorInfoList) {
       this.msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(20);
@@ -135,10 +135,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     }
     if (this.msrColEvalutorInfoList.size() > 0) {
       this.isMeasurePresentInCurrentBlock = new boolean[msrColEvalutorInfoList.size()];
-      this.measureBlocksIndex = new int[msrColEvalutorInfoList.size()];
+      this.measureChunkIndex = new int[msrColEvalutorInfoList.size()];
     } else {
       this.isMeasurePresentInCurrentBlock = new boolean[]{false};
-      this.measureBlocksIndex = new int[] {0};
+      this.measureChunkIndex = new int[] {0};
     }
     this.exp = exp;
     this.tableIdentifier = tableIdentifier;
@@ -147,22 +147,22 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
         DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.DATE);
     this.timestampDictionaryGenerator =
         DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP);
-    initDimensionBlockIndexes();
-    initMeasureBlockIndexes();
+    initDimensionChunkIndexes();
+    initMeasureChunkIndexes();
   }
 
   /**
    * This method will initialize the dimension info for the current block to be
    * used for filtering the data
    */
-  private void initDimensionBlockIndexes() {
+  private void initDimensionChunkIndexes() {
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
       // find the dimension in the current block dimensions list
       CarbonDimension dimensionFromCurrentBlock = segmentProperties
           .getDimensionFromCurrentBlock(dimColEvaluatorInfoList.get(i).getDimension());
       if (null != dimensionFromCurrentBlock) {
         dimColEvaluatorInfoList.get(i).setColumnIndex(dimensionFromCurrentBlock.getOrdinal());
-        this.dimensionBlocksIndex[i] = segmentProperties.getDimensionOrdinalToBlockMapping()
+        this.dimensionChunkIndex[i] = segmentProperties.getDimensionOrdinalToChunkMapping()
             .get(dimensionFromCurrentBlock.getOrdinal());
         isDimensionPresentInCurrentBlock[i] = true;
       }
@@ -173,14 +173,14 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * This method will initialize the measure info for the current block to be
    * used for filtering the data
    */
-  private void initMeasureBlockIndexes() {
+  private void initMeasureChunkIndexes() {
     for (int i = 0; i < msrColEvalutorInfoList.size(); i++) {
       // find the measure in the current block measures list
       CarbonMeasure measureFromCurrentBlock = segmentProperties.getMeasureFromCurrentBlock(
           msrColEvalutorInfoList.get(i).getCarbonColumn().getColumnId());
       if (null != measureFromCurrentBlock) {
         msrColEvalutorInfoList.get(i).setColumnIndex(measureFromCurrentBlock.getOrdinal());
-        this.measureBlocksIndex[i] = segmentProperties.getMeasuresOrdinalToBlockMapping()
+        this.measureChunkIndex[i] = segmentProperties.getMeasuresOrdinalToChunkMapping()
             .get(measureFromCurrentBlock.getOrdinal());
         isMeasurePresentInCurrentBlock[i] = true;
       }
@@ -188,9 +188,9 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    readBlocks(blockChunkHolder);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    readColumnChunks(rawBlockletColumnChunks);
     // CHECKSTYLE:ON
 
     int[] numberOfRows = null;
@@ -198,26 +198,26 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
 
     if (dimColEvaluatorInfoList.size() > 0) {
       if (isDimensionPresentInCurrentBlock[0]) {
-        pageNumbers =
-            blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getPagesCount();
-        numberOfRows =
-            blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getRowCount();
+        pageNumbers = rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[0]]
+            .getPagesCount();
+        numberOfRows = rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[0]]
+            .getRowCount();
       } else {
         // specific for restructure case where default values need to be filled
-        pageNumbers = blockChunkHolder.getDataBlock().numberOfPages();
-        numberOfRows = new int[] { blockChunkHolder.getDataBlock().nodeSize() };
+        pageNumbers = rawBlockletColumnChunks.getDataBlock().numberOfPages();
+        numberOfRows = new int[] { rawBlockletColumnChunks.getDataBlock().numRows() };
       }
     }
     if (msrColEvalutorInfoList.size() > 0) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        pageNumbers =
-            blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getPagesCount();
-        numberOfRows =
-            blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getRowCount();
+        pageNumbers = rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]]
+            .getPagesCount();
+        numberOfRows = rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]]
+            .getRowCount();
       } else {
         // specific for restructure case where default values need to be filled
-        pageNumbers = blockChunkHolder.getDataBlock().numberOfPages();
-        numberOfRows = new int[] { blockChunkHolder.getDataBlock().nodeSize() };
+        pageNumbers = rawBlockletColumnChunks.getDataBlock().numberOfPages();
+        numberOfRows = new int[] { rawBlockletColumnChunks.getDataBlock().numRows() };
       }
     }
     BitSetGroup bitSetGroup = new BitSetGroup(pageNumbers);
@@ -227,10 +227,12 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
       BitSet prvBitset = null;
       // if bitset pipe line is enabled then use rowid from previous bitset
       // otherwise use older flow
-      if (!useBitsetPipeLine || null == blockChunkHolder.getBitSetGroup() || null == bitSetGroup
-          .getBitSet(i) || blockChunkHolder.getBitSetGroup().getBitSet(i).isEmpty()) {
+      if (!useBitsetPipeLine ||
+          null == rawBlockletColumnChunks.getBitSetGroup() ||
+          null == bitSetGroup.getBitSet(i) ||
+          rawBlockletColumnChunks.getBitSetGroup().getBitSet(i).isEmpty()) {
         for (int index = 0; index < numberOfRows[i]; index++) {
-          createRow(blockChunkHolder, row, i, index);
+          createRow(rawBlockletColumnChunks, row, i, index);
           Boolean rslt = false;
           try {
             rslt = exp.evaluate(row).getBoolean();
@@ -246,10 +248,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
           }
         }
       } else {
-        prvBitset = blockChunkHolder.getBitSetGroup().getBitSet(i);
+        prvBitset = rawBlockletColumnChunks.getBitSetGroup().getBitSet(i);
         for (int index = prvBitset.nextSetBit(0);
              index >= 0; index = prvBitset.nextSetBit(index + 1)) {
-          createRow(blockChunkHolder, row, i, index);
+          createRow(rawBlockletColumnChunks, row, i, index);
           Boolean rslt = false;
           try {
             rslt = exp.evaluate(row).getBoolean();
@@ -284,8 +286,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * @param index
    * @throws IOException
    */
-  private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int pageIndex, int index)
-      throws IOException {
+  private void createRow(RawBlockletColumnChunks blockChunkHolder, RowIntf row, int pageIndex,
+      int index) throws IOException {
     Object[] record = new Object[dimColEvaluatorInfoList.size() + msrColEvalutorInfoList.size()];
     String memberString;
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
@@ -302,14 +304,14 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
           record[dimColumnEvaluatorInfo.getRowIndex()] =
               dimColumnEvaluatorInfo.getDimension().getDefaultValue();
         }
-        DimensionColumnDataChunk columnDataChunk =
-            blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]]
-                .convertToDimColDataChunk(pageIndex);
+        DimensionColumnPage columnDataChunk =
+            blockChunkHolder.getDimensionRawColumnChunks()[dimensionChunkIndex[i]]
+                .decodeColumnPage(pageIndex);
         if (!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
-            && columnDataChunk instanceof VariableLengthDimensionDataChunk) {
+            && columnDataChunk instanceof VariableLengthDimensionColumnPage) {
 
-          VariableLengthDimensionDataChunk dimensionColumnDataChunk =
-              (VariableLengthDimensionDataChunk) columnDataChunk;
+          VariableLengthDimensionColumnPage dimensionColumnDataChunk =
+              (VariableLengthDimensionColumnPage) columnDataChunk;
           byte[] memberBytes = dimensionColumnDataChunk.getChunkData(index);
           if (null != memberBytes) {
             if (Arrays.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, memberBytes)) {
@@ -322,8 +324,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
                     dimColumnEvaluatorInfo.getDimension().getDataType());
           }
         } else {
-          int dictionaryValue = readSurrogatesFromColumnBlock(blockChunkHolder, index, pageIndex,
-              dimColumnEvaluatorInfo, dimensionBlocksIndex[i]);
+          int dictionaryValue = readSurrogatesFromColumnChunk(blockChunkHolder, index, pageIndex,
+              dimColumnEvaluatorInfo, dimensionChunkIndex[i]);
           if (dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
               && !dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
             memberString =
@@ -341,11 +343,11 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
         }
       } else {
         try {
-          GenericQueryType complexType = complexDimensionInfoMap.get(dimensionBlocksIndex[i]);
+          GenericQueryType complexType = complexDimensionInfoMap.get(dimensionChunkIndex[i]);
           ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
           DataOutputStream dataOutputStream = new DataOutputStream(byteStream);
           complexType.parseBlocksAndReturnComplexColumnByteArray(
-              blockChunkHolder.getDimensionRawDataChunk(), index, pageIndex, dataOutputStream);
+              blockChunkHolder.getDimensionRawColumnChunks(), index, pageIndex, dataOutputStream);
           record[dimColumnEvaluatorInfo.getRowIndex()] = complexType
               .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray()));
           byteStream.close();
@@ -384,8 +386,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
 
       Object msrValue;
       ColumnPage columnPage =
-          blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]]
-              .convertToColumnPage(pageIndex);
+          blockChunkHolder.getMeasureRawColumnChunks()[measureChunkIndex[0]]
+              .decodeColumnPage(pageIndex);
       if (msrType == DataTypes.BOOLEAN) {
         msrValue = columnPage.getBoolean(index);
       } else if (msrType == DataTypes.SHORT) {
@@ -487,10 +489,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * @param dimColumnEvaluatorInfo
    * @return
    */
-  private int readSurrogatesFromColumnBlock(BlocksChunkHolder blockChunkHolder, int index, int page,
-      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int blockIndex) {
-    DimensionColumnDataChunk dataChunk =
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex].convertToDimColDataChunk(page);
+  private int readSurrogatesFromColumnChunk(RawBlockletColumnChunks blockChunkHolder, int index,
+      int page, DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int chunkIndex) {
+    DimensionColumnPage dataChunk =
+        blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex].decodeColumnPage(page);
     if (dimColumnEvaluatorInfo.getDimension().isColumnar()) {
       byte[] rawData = dataChunk.getChunkData(index);
       ByteBuffer byteBuffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
@@ -506,7 +508,7 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * @param dimColumnEvaluatorInfo
    * @return read surrogate of given row of given column group dimension
    */
-  private int readSurrogatesFromColumnGroupBlock(DimensionColumnDataChunk chunk, int index,
+  private int readSurrogatesFromColumnGroupBlock(DimensionColumnPage chunk, int index,
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo) {
     try {
       KeyStructureInfo keyStructureInfo =
@@ -514,9 +516,9 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
       byte[] colData = chunk.getChunkData(index);
       long[] result = keyStructureInfo.getKeyGenerator().getKeyArray(colData);
       int colGroupId =
-          QueryUtil.getColumnGroupId(segmentProperties, dimensionBlocksIndex[0]);
+          QueryUtil.getColumnGroupId(segmentProperties, dimensionChunkIndex[0]);
       return (int) result[segmentProperties
-          .getColumnGroupMdKeyOrdinal(colGroupId, dimensionBlocksIndex[0])];
+          .getColumnGroupMdKeyOrdinal(colGroupId, dimensionChunkIndex[0])];
     } catch (KeyGenException e) {
       LOGGER.error(e);
     }
@@ -524,33 +526,36 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   }
 
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     bitSet.set(0);
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
       if (!dimColumnEvaluatorInfo.getDimension().getDataType().isComplexType()) {
-        if (null == blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]]) {
-          blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]] =
-              blockChunkHolder.getDataBlock()
-                  .getDimensionChunk(blockChunkHolder.getFileReader(), dimensionBlocksIndex[i]);
+        if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[i]])
+        {
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[i]] =
+              rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                  rawBlockletColumnChunks.getFileReader(), dimensionChunkIndex[i]);
         }
       } else {
-        GenericQueryType complexType = complexDimensionInfoMap.get(dimensionBlocksIndex[i]);
-        complexType.fillRequiredBlockData(blockChunkHolder);
+        GenericQueryType complexType = complexDimensionInfoMap.get(dimensionChunkIndex[i]);
+        complexType.fillRequiredBlockData(rawBlockletColumnChunks);
       }
     }
 
     if (null != msrColEvalutorInfoList) {
       for (MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo : msrColEvalutorInfoList) {
-        if (null == blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]]) {
-          blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]] =
-              blockChunkHolder.getDataBlock()
-                  .getMeasureChunk(blockChunkHolder.getFileReader(), measureBlocksIndex[0]);
+        if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]]) {
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]] =
+              rawBlockletColumnChunks.getDataBlock()
+                  .readMeasureChunk(rawBlockletColumnChunks.getFileReader(), measureChunkIndex[0]);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
index 306f3fa..6baa3e4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -32,11 +32,10 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -55,12 +54,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
    */
   private boolean isDefaultValuePresentInFilter;
   private int lastDimensionColOrdinal = 0;
-  public RowLevelRangeGrtThanFiterExecuterImpl(
-      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
+
+  RowLevelRangeGrtThanFiterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
-      Object[] msrFilterRangeValues,
-      SegmentProperties segmentProperties) {
+      Object[] msrFilterRangeValues, SegmentProperties segmentProperties) {
     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties,
         null);
     this.filterRangeValues = filterRangeValues;
@@ -110,17 +108,18 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     }
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     boolean isScanRequired = false;
     byte[] maxValue = null;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        maxValue = blockMaxValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        maxValue = blockMaxValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        maxValue = blockMaxValue[dimensionBlocksIndex[0]];
+        maxValue = blockMaxValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(maxValue, filterRangeValues);
       }
     } else {
@@ -167,24 +166,25 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -196,26 +196,28 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             } else {
-              BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+              BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             }
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -225,7 +227,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
                 .getMeasureObjectFromDataType(rawColumnChunk.getMinValues()[i],
                     msrColEvalutorInfoList.get(0).getType()));
             ColumnPage columnPage =
-                rawColumnChunk.convertToColumnPage(i);
+                rawColumnChunk.decodeColumnPage(i);
             if (compare < 0 && columnPage.getNullBits().isEmpty()) {
               BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]);
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
@@ -238,7 +240,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
@@ -278,16 +280,16 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     return bitSet;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -299,12 +301,12 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
     int last = 0;
@@ -312,11 +314,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     byte[][] filterValues = this.filterRangeValues;
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], true);
       if (start >= 0) {
         start = CarbonUtil
-            .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[i],
+            .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[i],
                 numerOfRows);
       }
       // Logic will handle the case where the range filter member is not present in block
@@ -332,7 +334,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
         // index needs to be compared by the filter member if its > filter then from that
         // index the bitset will be considered for filtering process.
         if (ByteUtil.compare(filterValues[i],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             > 0) {
           start = start + 1;
         }
@@ -340,7 +342,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
 
       last = start;
       for (int j = start; j < numerOfRows; j++) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last++;
       }
       startIndex = last;
@@ -358,11 +360,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
    * be called if the column is sorted default so column index
    * mapping will be present for accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -373,11 +375,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
       int startIndex = 0;
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], true);
         if (start >= 0) {
           start = CarbonUtil
-              .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[k],
+              .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[k],
                   numerOfRows);
         }
         if (start < 0) {
@@ -388,7 +390,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
           // Method will compare the tentative index value after binary search, this tentative
           // index needs to be compared by the filter member if its > filter then from that
           // index the bitset will be considered for filtering process.
-          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) > 0) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) > 0) {
             start = start + 1;
           }
         }
@@ -405,7 +407,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) > 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) > 0) {
             bitSet.set(i);
           }
         }
@@ -414,21 +416,24 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
index db55e42..3bb24af 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -32,11 +32,10 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -46,7 +45,7 @@ import org.apache.carbondata.core.util.comparator.SerializableComparator;
 
 public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl {
 
-  protected byte[][] filterRangeValues;
+  private byte[][] filterRangeValues;
   private Object[] msrFilterRangeValues;
   private SerializableComparator comparator;
   /**
@@ -55,7 +54,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
   private boolean isDefaultValuePresentInFilter;
   private int lastDimensionColOrdinal = 0;
 
-  public RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
+  RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
@@ -115,11 +114,11 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     byte[] maxValue = null;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        maxValue = blockMaxValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        maxValue = blockMaxValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        maxValue = blockMaxValue[dimensionBlocksIndex[0]];
+        maxValue = blockMaxValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(maxValue, filterRangeValues);
       }
     } else {
@@ -165,25 +164,26 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
 
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -195,26 +195,28 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             } else {
-              BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+              BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             }
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
-    } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+    } else {
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -224,28 +226,27 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
                 .getMeasureObjectFromDataType(rawColumnChunk.getMinValues()[i],
                     msrColEvalutorInfoList.get(0).getType()));
             ColumnPage columnPage =
-                rawColumnChunk.convertToColumnPage(i);
+                rawColumnChunk.decodeColumnPage(i);
             if (compare <= 0 && columnPage.getNullBits().isEmpty()) {
               BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]);
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             } else {
               BitSet bitSet =
-                  getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+                  getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                       rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             }
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     }
-    return null;
   }
 
   private BitSet getFilteredIndexesForMeasures(ColumnPage columnPage,
@@ -279,16 +280,16 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
   }
 
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -300,12 +301,12 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
     int last = 0;
@@ -313,7 +314,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     byte[][] filterValues = this.filterRangeValues;
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], false);
       if (start < 0) {
         start = -(start + 1);
@@ -324,14 +325,14 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
         // index needs to be compared by the filter member if its >= filter then from that
         // index the bitset will be considered for filtering process.
         if (ByteUtil.compare(filterValues[i],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             > 0) {
           start = start + 1;
         }
       }
       last = start;
       for (int j = start; j < numerOfRows; j++) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last++;
       }
       startIndex = last;
@@ -348,11 +349,11 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
    * be called if the column is sorted default so column index
    * mapping will be present for accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -363,7 +364,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
       int startIndex = 0;
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], false);
         if (start < 0) {
           start = -(start + 1);
@@ -373,7 +374,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
           // Method will compare the tentative index value after binary search, this tentative
           // index needs to be compared by the filter member if its >= filter then from that
           // index the bitset will be considered for filtering process.
-          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) > 0) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) > 0) {
             start = start + 1;
           }
         }
@@ -391,7 +392,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) >= 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) >= 0) {
             bitSet.set(i);
           }
         }
@@ -400,21 +401,24 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index 88cf75c..2e9bcb6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -35,11 +35,10 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -117,11 +116,11 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     boolean isScanRequired = false;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        minValue = blockMinValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        minValue = blockMinValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        minValue = blockMinValue[dimensionBlocksIndex[0]];
+        minValue = blockMinValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(minValue, filterRangeValues);
       }
     } else {
@@ -167,60 +166,63 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) {
-            BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                 rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.msrFilterRangeValues,
               msrColEvalutorInfoList.get(0).getType())) {
             BitSet bitSet =
-                getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+                getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                     rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
@@ -260,7 +262,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     return bitSet;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     byte[] defaultValue = null;
     if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
@@ -269,7 +271,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
               dimColEvaluatorInfoList.get(0).getDimension().getDataType());
       int key = directDictionaryGenerator.generateDirectSurrogateKey(null);
       CarbonDimension currentBlockDimension =
-          segmentProperties.getDimensions().get(dimensionBlocksIndex[0]);
+          segmentProperties.getDimensions().get(dimensionChunkIndex[0]);
       if (currentBlockDimension.isSortColumn()) {
         defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
             this.segmentProperties.getSortColumnsGenerator());
@@ -280,14 +282,14 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows,
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows,
           defaultValue);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -299,12 +301,12 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows,
+      DimensionColumnPage dimensionColumnPage, int numerOfRows,
       byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
@@ -315,7 +317,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     //find the number of default values to skip the null value in case of direct dictionary
     if (null != defaultValue) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               defaultValue, true);
       if (start < 0) {
         skip = -(start + 1);
@@ -332,7 +334,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     }
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], true);
       if (start < 0) {
         start = -(start + 1);
@@ -343,14 +345,14 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
         // will be pointing to the next consecutive position. So compare it again and point to the
         // previous value returned from getFirstIndexUsingBinarySearch.
         if (ByteUtil.compare(filterValues[i],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             < 0) {
           start = start - 1;
         }
       }
       last = start;
       for (int j = start; j >= skip; j--) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last--;
       }
       startIndex = last;
@@ -367,12 +369,12 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
    * be called if the column is sorted default so column index
    * mapping will be present for accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @param defaultValue
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows, byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -385,7 +387,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
       //find the number of default values to skip the null value in case of direct dictionary
       if (null != defaultValue) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, defaultValue, true);
         if (start < 0) {
           skip = -(start + 1);
@@ -402,7 +404,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
       }
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], true);
         if (start < 0) {
           start = -(start + 1);
@@ -412,7 +414,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
           // When negative value of start is returned from getFirstIndexUsingBinarySearch the Start
           // will be pointing to the next consecutive position. So compare it again and point to the
           // previous value returned from getFirstIndexUsingBinarySearch.
-          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) < 0) {
             start = start - 1;
           }
         }
@@ -429,7 +431,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) <= 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) <= 0) {
             bitSet.set(i);
           }
         }
@@ -438,21 +440,24 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
deleted file mode 100644
index 6629d31..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
+++ /dev/null
@@ -1,163 +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.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 FixedLengthDimensionDataChunk extends AbstractDimensionDataChunk {
-
-  /**
-   * 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 FixedLengthDimensionDataChunk(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 data             data to filed
-   * @param offset           offset from which data need to be filed
-   * @param index            row id of the chunk
-   * @param keyStructureInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int index,
-      KeyStructureInfo keyStructureInfo) {
-    dataChunkStore.fillRow(index, data, offset);
-    return dataChunkStore.getColumnValueSize();
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo
-   * @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo) {
-    row[columnIndex] = dataChunkStore.getSurrogate(rowId);
-    return columnIndex + 1;
-  }
-
-  /**
-   * 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 = 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 column + 1;
-  }
-
-  /**
-   * 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 = columnVectorInfo.size + offset;
-    CarbonColumnVector vector = columnVectorInfo.vector;
-    for (int j = offset; j < len; j++) {
-      int dict = dataChunkStore.getSurrogate(rowMapping[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 column + 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
index fa0777b..5e8618b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
@@ -19,7 +19,7 @@ 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.reader.MeasureColumnChunkReader;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.memory.MemoryException;
 /**
  * Contains raw measure 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 convertToColumnPage method
+ * 2. The raw data can be converted to processed chunk using decodeColumnPage method
  *  by specifying page number.
  */
 public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
@@ -37,7 +37,7 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
 
   private MeasureColumnChunkReader chunkReader;
 
-  private FileHolder fileReader;
+  private FileReader fileReader;
 
   public MeasureRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, int length,
       MeasureColumnChunkReader chunkReader) {
@@ -48,14 +48,14 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
   /**
    * Convert all raw data with all pages to processed ColumnPage
    */
-  public ColumnPage[] convertToColumnPage() {
+  public ColumnPage[] decodeAllColumnPages() {
     if (columnPages == null) {
       columnPages = new ColumnPage[pagesCount];
     }
     for (int i = 0; i < pagesCount; i++) {
       try {
         if (columnPages[i] == null) {
-          columnPages[i] = chunkReader.convertToColumnPage(this, i);
+          columnPages[i] = chunkReader.decodeColumnPage(this, i);
         }
       } catch (Exception e) {
         throw new RuntimeException(e);
@@ -68,21 +68,21 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
   /**
    * Convert raw data with specified `columnIndex` processed to ColumnPage
    */
-  public ColumnPage convertToColumnPage(int columnIndex) {
-    assert columnIndex < pagesCount;
+  public ColumnPage decodeColumnPage(int pageNumber) {
+    assert pageNumber < pagesCount;
     if (columnPages == null) {
       columnPages = new ColumnPage[pagesCount];
     }
 
     try {
-      if (columnPages[columnIndex] == null) {
-        columnPages[columnIndex] = chunkReader.convertToColumnPage(this, columnIndex);
+      if (columnPages[pageNumber] == null) {
+        columnPages[pageNumber] = chunkReader.decodeColumnPage(this, pageNumber);
       }
     } catch (IOException | MemoryException e) {
       throw new RuntimeException(e);
     }
 
-    return columnPages[columnIndex];
+    return columnPages[pageNumber];
   }
 
   /**
@@ -95,7 +95,7 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
     assert index < pagesCount;
 
     try {
-      return chunkReader.convertToColumnPage(this, index);
+      return chunkReader.decodeColumnPage(this, index);
     } catch (IOException | MemoryException e) {
       throw new RuntimeException(e);
     }
@@ -111,11 +111,11 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
     }
   }
 
-  public void setFileReader(FileHolder fileReader) {
+  public void setFileReader(FileReader fileReader) {
     this.fileReader = fileReader;
   }
 
-  public FileHolder getFileReader() {
+  public FileReader getFileReader() {
     return fileReader;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java
new file mode 100644
index 0000000..d03b2de
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java
@@ -0,0 +1,133 @@
+/*
+ * 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.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 variable length dimension data chunk store
+ */
+public class VariableLengthDimensionColumnPage extends AbstractDimensionColumnPage {
+
+  /**
+   * Constructor for this class
+   * @param dataChunks
+   * @param invertedIndex
+   * @param invertedIndexReverse
+   * @param numberOfRows
+   */
+  public VariableLengthDimensionColumnPage(byte[] dataChunks, int[] invertedIndex,
+      int[] invertedIndexReverse, int numberOfRows) {
+    long totalSize = null != invertedIndex ?
+        (dataChunks.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) + (
+            numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE)) :
+        (dataChunks.length + (numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE));
+    dataChunkStore = DimensionChunkStoreFactory.INSTANCE
+        .getDimensionChunkStore(0, null != invertedIndex, numberOfRows, totalSize,
+            DimensionStoreType.VARIABLELENGTH);
+    dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunks);
+  }
+
+  /**
+   * 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) {
+    // no required in this case because this column chunk is not the part if
+    // mdkey
+    return 0;
+  }
+
+  /**
+   * 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) {
+    return chunkIndex + 1;
+  }
+
+  /**
+   * @return whether column is dictionary column or not
+   */
+  @Override public boolean isNoDicitionaryColumn() {
+    return true;
+  }
+
+  /**
+   * 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];
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    for (int i = offset; i < len; i++) {
+      // Considering only String case now as we support only
+      // string in no dictionary case at present.
+      dataChunkStore.fillRow(i, vector, vectorOffset++);
+    }
+    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];
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    for (int i = offset; i < len; i++) {
+      // Considering only String case now as we support only
+      // string in no dictionary case at present.
+      dataChunkStore.fillRow(filteredRowId[i], vector, vectorOffset++);
+    }
+    return chunkIndex + 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
deleted file mode 100644
index 6c47bf5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
+++ /dev/null
@@ -1,140 +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.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.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 variable length dimension data chunk store
- */
-public class VariableLengthDimensionDataChunk extends AbstractDimensionDataChunk {
-
-  /**
-   * Constructor for this class
-   * @param dataChunks
-   * @param invertedIndex
-   * @param invertedIndexReverse
-   * @param numberOfRows
-   */
-  public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex,
-      int[] invertedIndexReverse, int numberOfRows) {
-    long totalSize = null != invertedIndex ?
-        (dataChunks.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) + (
-            numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE)) :
-        (dataChunks.length + (numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE));
-    dataChunkStore = DimensionChunkStoreFactory.INSTANCE
-        .getDimensionChunkStore(0, null != invertedIndex, numberOfRows, totalSize,
-            DimensionStoreType.VARIABLELENGTH);
-    dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunks);
-  }
-
-  /**
-   * 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 index             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 index,
-      KeyStructureInfo restructuringInfo) {
-    // no required in this case because this column chunk is not the part if
-    // mdkey
-    return 0;
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo
-   * @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo) {
-    return columnIndex + 1;
-  }
-
-  /**
-   * @return whether column is dictionary column or not
-   */
-  @Override public boolean isNoDicitionaryColumn() {
-    return true;
-  }
-
-  /**
-   * @return length of each column
-   */
-  @Override public int getColumnValueSize() {
-    return -1;
-  }
-
-  /**
-   * 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];
-    CarbonColumnVector vector = columnVectorInfo.vector;
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    for (int i = offset; i < len; i++) {
-      // Considering only String case now as we support only
-      // string in no dictionary case at present.
-      dataChunkStore.fillRow(i, vector, vectorOffset++);
-    }
-    return column + 1;
-  }
-
-  /**
-   * 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];
-    CarbonColumnVector vector = columnVectorInfo.vector;
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    for (int i = offset; i < len; i++) {
-      // Considering only String case now as we support only
-      // string in no dictionary case at present.
-      dataChunkStore.fillRow(rowMapping[i], vector, vectorOffset++);
-    }
-    return column + 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
index 7b5b9c8..fd81973 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
@@ -18,8 +18,8 @@ package org.apache.carbondata.core.datastore.chunk.reader;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.memory.MemoryException;
 
@@ -34,20 +34,20 @@ public interface DimensionColumnChunkReader {
    * Below method will be used to read the chunk based on block indexes
    *
    * @param fileReader   file reader to read the blocks from file
-   * @param blockletIndexes blocklets to be read
+   * @param columnIndexRange blocklets to be read
    * @return dimension column chunks
    */
-  DimensionRawColumnChunk[] readRawDimensionChunks(FileHolder fileReader, int[][] blockletIndexes)
+  DimensionRawColumnChunk[] readRawDimensionChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
    * Below method will be used to read the chunk based on block index
    *
    * @param fileReader file reader to read the blocks from file
-   * @param blockletIndex block to be read
+   * @param columnIndex column to be read
    * @return dimension column chunk
    */
-  DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int blockletIndex)
+  DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int columnIndex)
       throws IOException;
 
   /**
@@ -58,6 +58,6 @@ public interface DimensionColumnChunkReader {
    * @return
    * @throws IOException
    */
-  DimensionColumnDataChunk convertToDimensionChunk(DimensionRawColumnChunk dimensionRawColumnChunk,
+  DimensionColumnPage decodeColumnPage(DimensionRawColumnChunk dimensionRawColumnChunk,
       int pageNumber) throws IOException, MemoryException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
index 02dc6a2..bf76025 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.core.datastore.chunk.reader;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -32,20 +32,20 @@ public interface MeasureColumnChunkReader {
    * Method to read the blocks data based on block indexes
    *
    * @param fileReader   file reader to read the blocks
-   * @param blockIndexes blocks to be read
+   * @param columnIndexRange blocks to be read
    * @return measure data chunks
    */
-  MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
+  MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
    * Method to read the blocks data based on block index
    *
    * @param fileReader file reader to read the blocks
-   * @param blockIndex block to be read
+   * @param columnIndex block to be read
    * @return measure data chunk
    */
-  MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex)
+  MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException;
 
   /**
@@ -55,7 +55,7 @@ public interface MeasureColumnChunkReader {
    * @return
    * @throws IOException
    */
-  ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
+  ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
       int pageNumber) throws IOException, MemoryException;
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
index f083612..60950c9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
@@ -19,7 +19,7 @@ package org.apache.carbondata.core.datastore.chunk.reader.dimension;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.format.Encoding;
@@ -55,43 +55,43 @@ public abstract class AbstractChunkReaderV2V3Format extends AbstractChunkReader
    * For last column read is separately and process
    *
    * @param fileReader      file reader to read the blocks from file
-   * @param blockletIndexes blocks range to be read
+   * @param columnIndexRange column index range to be read
    * @return dimension column chunks
    */
-  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(final FileHolder fileReader,
-      final int[][] blockletIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(final FileReader fileReader,
+      final int[][] columnIndexRange) throws IOException {
     // read the column chunk based on block index and add
     DimensionRawColumnChunk[] dataChunks =
         new DimensionRawColumnChunk[dimensionChunksOffset.size()];
     // if blocklet index is empty then return empry data chunk
-    if (blockletIndexes.length == 0) {
+    if (columnIndexRange.length == 0) {
       return dataChunks;
     }
     DimensionRawColumnChunk[] groupChunk = null;
     int index = 0;
     // iterate till block indexes -1 as block index will be in sorted order, so to avoid
     // the last column reading in group
-    for (int i = 0; i < blockletIndexes.length - 1; i++) {
+    for (int i = 0; i < columnIndexRange.length - 1; i++) {
       index = 0;
       groupChunk =
-          readRawDimensionChunksInGroup(fileReader, blockletIndexes[i][0], blockletIndexes[i][1]);
-      for (int j = blockletIndexes[i][0]; j <= blockletIndexes[i][1]; j++) {
+          readRawDimensionChunksInGroup(fileReader, columnIndexRange[i][0], columnIndexRange[i][1]);
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
     // check last index is present in block index, if it is present then read separately
-    if (blockletIndexes[blockletIndexes.length - 1][0] == dimensionChunksOffset.size() - 1) {
-      dataChunks[blockletIndexes[blockletIndexes.length - 1][0]] =
-          readRawDimensionChunk(fileReader, blockletIndexes[blockletIndexes.length - 1][0]);
+    if (columnIndexRange[columnIndexRange.length - 1][0] == dimensionChunksOffset.size() - 1) {
+      dataChunks[columnIndexRange[columnIndexRange.length - 1][0]] =
+          readRawDimensionChunk(fileReader, columnIndexRange[columnIndexRange.length - 1][0]);
     }
     // otherwise read the data in group
     else {
-      groupChunk =
-          readRawDimensionChunksInGroup(fileReader, blockletIndexes[blockletIndexes.length - 1][0],
-              blockletIndexes[blockletIndexes.length - 1][1]);
+      groupChunk = readRawDimensionChunksInGroup(
+          fileReader, columnIndexRange[columnIndexRange.length - 1][0],
+          columnIndexRange[columnIndexRange.length - 1][1]);
       index = 0;
-      for (int j = blockletIndexes[blockletIndexes.length - 1][0];
-           j <= blockletIndexes[blockletIndexes.length - 1][1]; j++) {
+      for (int j = columnIndexRange[columnIndexRange.length - 1][0];
+           j <= columnIndexRange[columnIndexRange.length - 1][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
@@ -109,7 +109,7 @@ public abstract class AbstractChunkReaderV2V3Format extends AbstractChunkReader
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected abstract DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected abstract DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
index 27a4d89..0dc1c1b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
@@ -20,12 +20,12 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReader;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -61,14 +61,14 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
    * Below method will be used to read the raw chunk based on block indexes
    *
    * @param fileReader   file reader to read the blocks from file
-   * @param blockletIndexes blocks to be read
+   * @param columnIndexRange blocks to be read
    * @return dimension column chunks
    */
-  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(FileHolder fileReader,
-      int[][] blockletIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     DimensionRawColumnChunk[] dataChunks = new DimensionRawColumnChunk[dimensionColumnChunk.size()];
-    for (int i = 0; i < blockletIndexes.length; i++) {
-      for (int j = blockletIndexes[i][0]; j <= blockletIndexes[i][1]; j++) {
+    for (int i = 0; i < columnIndexRange.length; i++) {
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         dataChunks[j] = readRawDimensionChunk(fileReader, j);
       }
     }
@@ -82,7 +82,7 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
    * @param columnIndex column to be read
    * @return dimension column chunk
    */
-  @Override public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
+  @Override public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader,
       int columnIndex) throws IOException {
     DataChunk dataChunk = dimensionColumnChunk.get(columnIndex);
     ByteBuffer buffer = null;
@@ -92,20 +92,20 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
     }
     DimensionRawColumnChunk rawColumnChunk = new DimensionRawColumnChunk(columnIndex, buffer, 0,
         dataChunk.getDataPageLength(), this);
-    rawColumnChunk.setFileHolder(fileReader);
+    rawColumnChunk.setFileReader(fileReader);
     rawColumnChunk.setPagesCount(1);
     rawColumnChunk.setRowCount(new int[] { numberOfRows });
     return rawColumnChunk;
   }
 
-  @Override public DimensionColumnDataChunk convertToDimensionChunk(
+  @Override public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
     int blockIndex = dimensionRawColumnChunk.getColumnIndex();
     byte[] dataPage = null;
     int[] invertedIndexes = null;
     int[] invertedIndexesReverse = null;
     int[] rlePage = null;
-    FileHolder fileReader = dimensionRawColumnChunk.getFileReader();
+    FileReader fileReader = dimensionRawColumnChunk.getFileReader();
 
     ByteBuffer rawData = dimensionRawColumnChunk.getRawData();
     dataPage = COMPRESSOR.unCompressByte(rawData.array(), (int) dimensionRawColumnChunk.getOffSet(),
@@ -145,23 +145,23 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
       rlePage = null;
     }
     // fill chunk attributes
-    DimensionColumnDataChunk columnDataChunk = null;
+    DimensionColumnPage columnDataChunk = null;
     if (dataChunk.isRowMajor()) {
       // to store fixed length column chunk values
-      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, eachColumnValueSize[blockIndex],
-          numberOfRows);
+      columnDataChunk = new ColumnGroupDimensionColumnPage(
+          dataPage, eachColumnValueSize[blockIndex], numberOfRows);
     }
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
     else if (!CarbonUtil
         .hasEncoding(dataChunk.getEncodingList(), Encoding.DICTIONARY)) {
       columnDataChunk =
-          new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows);
     } else {
       // to store fixed length column chunk values
       columnDataChunk =
-          new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows, eachColumnValueSize[blockIndex]);
     }
     return columnDataChunk;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
index b43f89c..31fa819 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
@@ -19,12 +19,12 @@ package org.apache.carbondata.core.datastore.chunk.reader.dimension.v2;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -56,7 +56,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
    * @param columnIndex   column to be read
    * @return dimension column chunk
    */
-  public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int columnIndex)
+  public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     int length = 0;
     if (dimensionChunksOffset.size() - 1 == columnIndex) {
@@ -73,7 +73,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
     }
     DimensionRawColumnChunk rawColumnChunk =
         new DimensionRawColumnChunk(columnIndex, buffer, 0, length, this);
-    rawColumnChunk.setFileHolder(fileReader);
+    rawColumnChunk.setFileReader(fileReader);
     rawColumnChunk.setPagesCount(1);
     rawColumnChunk.setRowCount(new int[] { numberOfRows });
     return rawColumnChunk;
@@ -90,7 +90,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
     long currentDimensionOffset = dimensionChunksOffset.get(startColumnBlockletIndex);
     ByteBuffer buffer = null;
@@ -106,7 +106,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
       int currentLength = (int) (dimensionChunksOffset.get(i + 1) - dimensionChunksOffset.get(i));
       dataChunks[index] =
           new DimensionRawColumnChunk(i, buffer, runningLength, currentLength, this);
-      dataChunks[index].setFileHolder(fileReader);
+      dataChunks[index].setFileReader(fileReader);
       dataChunks[index].setPagesCount(1);
       dataChunks[index].setRowCount(new int[] { numberOfRows });
       runningLength += currentLength;
@@ -115,7 +115,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
     return dataChunks;
   }
 
-  public DimensionColumnDataChunk convertToDimensionChunk(
+  public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
     byte[] dataPage = null;
     int[] invertedIndexes = null;
@@ -169,23 +169,23 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
       dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[blockIndex]);
     }
     // fill chunk attributes
-    DimensionColumnDataChunk columnDataChunk = null;
+    DimensionColumnPage columnDataChunk = null;
 
     if (dimensionColumnChunk.isRowMajor()) {
       // to store fixed length column chunk values
-      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, eachColumnValueSize[blockIndex],
-          numberOfRows);
+      columnDataChunk = new ColumnGroupDimensionColumnPage(
+          dataPage, eachColumnValueSize[blockIndex], numberOfRows);
     }
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
     else if (!hasEncoding(dimensionColumnChunk.encoders, Encoding.DICTIONARY)) {
       columnDataChunk =
-          new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows);
     } else {
       // to store fixed length column chunk values
       columnDataChunk =
-          new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows, eachColumnValueSize[blockIndex]);
     }
     return columnDataChunk;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
index 1edfd09..60f0b67 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
@@ -20,8 +20,8 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -70,7 +70,8 @@ public class CompressedDimChunkFileBasedPageLevelReaderV3
    * @param blockletColumnIndex blocklet index of the column in carbon data file
    * @return dimension raw chunk
    */
-  public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
+  @Override
+  public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader,
       int blockletColumnIndex) throws IOException {
     // get the current dimension offset
     long currentDimensionOffset = dimensionChunksOffset.get(blockletColumnIndex);
@@ -116,7 +117,7 @@ public class CompressedDimChunkFileBasedPageLevelReaderV3
    * @param endBlockletColumnIndex   blocklet index of the last dimension column
    * @ DimensionRawColumnChunk array
    */
-  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startBlockletColumnIndex, int endBlockletColumnIndex) throws IOException {
     // create raw chunk for each dimension column
     DimensionRawColumnChunk[] dimensionDataChunks =
@@ -136,7 +137,7 @@ public class CompressedDimChunkFileBasedPageLevelReaderV3
    * @param pageNumber              number
    * @return DimensionColumnDataChunk
    */
-  @Override public DimensionColumnDataChunk convertToDimensionChunk(
+  @Override public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber)
       throws IOException, MemoryException {
     // data chunk of page

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
index 566e9b7..0fdc515 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
@@ -20,11 +20,11 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.chunk.store.ColumnPageWrapper;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
@@ -78,23 +78,23 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
    * 5. Create the raw chunk object and fill the details
    *
    * @param fileReader          reader for reading the column from carbon data file
-   * @param blockletColumnIndex blocklet index of the column in carbon data file
+   * @param columnIndex blocklet index of the column in carbon data file
    * @return dimension raw chunk
    */
-  public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
-      int blockletColumnIndex) throws IOException {
+  public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader,
+      int columnIndex) throws IOException {
     // get the current dimension offset
-    long currentDimensionOffset = dimensionChunksOffset.get(blockletColumnIndex);
+    long currentDimensionOffset = dimensionChunksOffset.get(columnIndex);
     int length = 0;
     // to calculate the length of the data to be read
     // column other than last column we can subtract the offset of current column with
     // next column and get the total length.
     // but for last column we need to use lastDimensionOffset which is the end position
     // of the last dimension, we can subtract current dimension offset from lastDimesionOffset
-    if (dimensionChunksOffset.size() - 1 == blockletColumnIndex) {
+    if (dimensionChunksOffset.size() - 1 == columnIndex) {
       length = (int) (lastDimensionOffsets - currentDimensionOffset);
     } else {
-      length = (int) (dimensionChunksOffset.get(blockletColumnIndex + 1) - currentDimensionOffset);
+      length = (int) (dimensionChunksOffset.get(columnIndex + 1) - currentDimensionOffset);
     }
     ByteBuffer buffer = null;
     // read the data from carbon data file
@@ -103,15 +103,15 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     }
     // get the data chunk which will have all the details about the data pages
     DataChunk3 dataChunk = CarbonUtil.readDataChunk3(buffer, 0, length);
-    return getDimensionRawColumnChunk(fileReader, blockletColumnIndex, 0, length, buffer,
+    return getDimensionRawColumnChunk(fileReader, columnIndex, 0, length, buffer,
         dataChunk);
   }
 
-  protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileHolder fileReader,
-      int blockletColumnIndex, long offset, int length, ByteBuffer buffer, DataChunk3 dataChunk) {
+  protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileReader fileReader,
+      int columnIndex, long offset, int length, ByteBuffer buffer, DataChunk3 dataChunk) {
     // creating a raw chunks instance and filling all the details
     DimensionRawColumnChunk rawColumnChunk =
-        new DimensionRawColumnChunk(blockletColumnIndex, buffer, offset, length, this);
+        new DimensionRawColumnChunk(columnIndex, buffer, offset, length, this);
     int numberOfPages = dataChunk.getPage_length().size();
     byte[][] maxValueOfEachPage = new byte[numberOfPages][];
     byte[][] minValueOfEachPage = new byte[numberOfPages][];
@@ -124,7 +124,7 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
       eachPageLength[i] = dataChunk.getData_chunk_list().get(i).getNumberOfRowsInpage();
     }
     rawColumnChunk.setDataChunkV3(dataChunk);
-    rawColumnChunk.setFileHolder(fileReader);
+    rawColumnChunk.setFileReader(fileReader);
     rawColumnChunk.setPagesCount(dataChunk.getPage_length().size());
     rawColumnChunk.setMaxValues(maxValueOfEachPage);
     rawColumnChunk.setMinValues(minValueOfEachPage);
@@ -153,7 +153,7 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
    *        blocklet index of the last dimension column
    * @ DimensionRawColumnChunk array
    */
-  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startBlockletColumnIndex, int endBlockletColumnIndex) throws IOException {
     // to calculate the length of the data to be read
     // column we can subtract the offset of start column offset with
@@ -188,9 +188,9 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
    *
    * @param rawColumnPage dimension raw chunk
    * @param pageNumber              number
-   * @return DimensionColumnDataChunk
+   * @return DimensionColumnPage
    */
-  @Override public DimensionColumnDataChunk convertToDimensionChunk(
+  @Override public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException {
     // data chunk of blocklet column
     DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
@@ -228,20 +228,19 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     return false;
   }
 
-  protected DimensionColumnDataChunk decodeDimension(DimensionRawColumnChunk rawColumnPage,
+  protected DimensionColumnPage decodeDimension(DimensionRawColumnChunk rawColumnPage,
       ByteBuffer pageData, DataChunk2 pageMetadata, int offset)
       throws IOException, MemoryException {
     if (isEncodedWithMeta(pageMetadata)) {
       ColumnPage decodedPage = decodeDimensionByMeta(pageMetadata, pageData, offset);
-      return new ColumnPageWrapper(decodedPage,
-          eachColumnValueSize[rawColumnPage.getColumnIndex()]);
+      return new ColumnPageWrapper(decodedPage);
     } else {
       // following code is for backward compatibility
       return decodeDimensionLegacy(rawColumnPage, pageData, pageMetadata, offset);
     }
   }
 
-  private DimensionColumnDataChunk decodeDimensionLegacy(DimensionRawColumnChunk rawColumnPage,
+  private DimensionColumnPage decodeDimensionLegacy(DimensionRawColumnChunk rawColumnPage,
       ByteBuffer pageData, DataChunk2 pageMetadata, int offset) {
     byte[] dataPage;
     int[] rlePage;
@@ -267,18 +266,18 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
           eachColumnValueSize[rawColumnPage.getColumnIndex()]);
     }
 
-    DimensionColumnDataChunk columnDataChunk = null;
+    DimensionColumnPage columnDataChunk = null;
 
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
     if (!hasEncoding(pageMetadata.encoders, Encoding.DICTIONARY)) {
       columnDataChunk =
-          new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               pageMetadata.getNumberOfRowsInpage());
     } else {
       // to store fixed length column chunk values
       columnDataChunk =
-          new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               pageMetadata.getNumberOfRowsInpage(),
               eachColumnValueSize[rawColumnPage.getColumnIndex()]);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
index 2239a2b..a3ed339 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.compression.Compressor;
 import org.apache.carbondata.core.datastore.compression.CompressorFactory;
@@ -56,36 +56,40 @@ public abstract class AbstractMeasureChunkReaderV2V3Format extends AbstractMeasu
    * separately and process
    *
    * @param fileReader   file reader to read the blocks from file
-   * @param blockIndexes blocks range to be read
+   * @param columnIndexRange blocks range to be read, columnIndexGroup[i] is one group, inside the
+   *                         group, columnIndexGroup[i][0] is start column index,
+   *                         and columnIndexGroup[i][1] is end column index
    * @return measure column chunks
    * @throws IOException
    */
-  public MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
-      throws IOException {
+  public MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     // read the column chunk based on block index and add
     MeasureRawColumnChunk[] dataChunks =
         new MeasureRawColumnChunk[measureColumnChunkOffsets.size()];
-    if (blockIndexes.length == 0) {
+    if (columnIndexRange.length == 0) {
       return dataChunks;
     }
     MeasureRawColumnChunk[] groupChunk = null;
     int index = 0;
-    for (int i = 0; i < blockIndexes.length - 1; i++) {
+    for (int i = 0; i < columnIndexRange.length - 1; i++) {
       index = 0;
-      groupChunk = readRawMeasureChunksInGroup(fileReader, blockIndexes[i][0], blockIndexes[i][1]);
-      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+      groupChunk = readRawMeasureChunksInGroup(
+          fileReader, columnIndexRange[i][0], columnIndexRange[i][1]);
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
-    if (blockIndexes[blockIndexes.length - 1][0] == measureColumnChunkOffsets.size() - 1) {
-      dataChunks[blockIndexes[blockIndexes.length - 1][0]] =
-          readRawMeasureChunk(fileReader, blockIndexes[blockIndexes.length - 1][0]);
+    if (columnIndexRange[columnIndexRange.length - 1][0] == measureColumnChunkOffsets.size() - 1) {
+      dataChunks[columnIndexRange[columnIndexRange.length - 1][0]] =
+          readRawMeasureChunk(fileReader, columnIndexRange[columnIndexRange.length - 1][0]);
     } else {
-      groupChunk = readRawMeasureChunksInGroup(fileReader, blockIndexes[blockIndexes.length - 1][0],
-          blockIndexes[blockIndexes.length - 1][1]);
+      groupChunk = readRawMeasureChunksInGroup(
+          fileReader, columnIndexRange[columnIndexRange.length - 1][0],
+          columnIndexRange[columnIndexRange.length - 1][1]);
       index = 0;
-      for (int j = blockIndexes[blockIndexes.length - 1][0];
-           j <= blockIndexes[blockIndexes.length - 1][1]; j++) {
+      for (int j = columnIndexRange[columnIndexRange.length - 1][0];
+           j <= columnIndexRange[columnIndexRange.length - 1][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
@@ -112,12 +116,12 @@ public abstract class AbstractMeasureChunkReaderV2V3Format extends AbstractMeasu
    * data from
    *
    * @param fileReader               file reader to read the data
-   * @param startColumnBlockletIndex first column blocklet index to be read
-   * @param endColumnBlockletIndex   end column blocklet index to be read
+   * @param startColumnIndex first column index to be read
+   * @param endColumnIndex   end column index to be read
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected abstract MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
-      int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException;
+  protected abstract MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
+      int startColumnIndex, int endColumnIndex) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
index ae55375..f0c1b75 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReader;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -56,14 +56,14 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
    * Method to read the blocks data based on block indexes
    *
    * @param fileReader   file reader to read the blocks
-   * @param blockIndexes blocks to be read
+   * @param columnIndexRange blocks to be read
    * @return measure data chunks
    */
-  @Override public MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     MeasureRawColumnChunk[] datChunk = new MeasureRawColumnChunk[measureColumnChunks.size()];
-    for (int i = 0; i < blockIndexes.length; i++) {
-      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+    for (int i = 0; i < columnIndexRange.length; i++) {
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         datChunk[j] = readRawMeasureChunk(fileReader, j);
       }
     }
@@ -77,7 +77,7 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
    * @param columnIndex column to be read
    * @return measure data chunk
    */
-  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int columnIndex)
+  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     DataChunk dataChunk = measureColumnChunks.get(columnIndex);
     ByteBuffer buffer = fileReader
@@ -91,7 +91,7 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
   }
 
   @Override
-  public ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
+  public ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
       int pageNumber) throws IOException, MemoryException {
     int blockIndex = measureRawColumnChunk.getColumnIndex();
     DataChunk dataChunk = measureColumnChunks.get(blockIndex);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
index 2ddc202..e990b4b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -48,7 +48,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
   }
 
   @Override
-  public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int columnIndex)
+  public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     int dataLength = 0;
     if (measureColumnChunkOffsets.size() - 1 == columnIndex) {
@@ -76,30 +76,30 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
    * data from
    *
    * @param fileReader               file reader to read the data
-   * @param startColumnBlockletIndex first column blocklet index to be read
-   * @param endColumnBlockletIndex   end column blocklet index to be read
+   * @param startColumnIndex first column blocklet index to be read
+   * @param endColumnIndex   end column blocklet index to be read
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
-      int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
-    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex);
+  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
+      int startColumnIndex, int endColumnIndex) throws IOException {
+    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnIndex);
     ByteBuffer buffer = null;
     synchronized (fileReader) {
       buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset,
-          (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
+          (int) (measureColumnChunkOffsets.get(endColumnIndex + 1) - currentMeasureOffset));
     }
     MeasureRawColumnChunk[] dataChunks =
-        new MeasureRawColumnChunk[endColumnBlockletIndex - startColumnBlockletIndex + 1];
+        new MeasureRawColumnChunk[endColumnIndex - startColumnIndex + 1];
     int runningLength = 0;
     int index = 0;
-    for (int i = startColumnBlockletIndex; i <= endColumnBlockletIndex; i++) {
+    for (int i = startColumnIndex; i <= endColumnIndex; i++) {
       int currentLength =
           (int) (measureColumnChunkOffsets.get(i + 1) - measureColumnChunkOffsets.get(i));
       MeasureRawColumnChunk measureRawColumnChunk =
           new MeasureRawColumnChunk(i, buffer, runningLength, currentLength, this);
-      measureRawColumnChunk.setFileReader(fileReader);
       measureRawColumnChunk.setRowCount(new int[] { numberOfRows });
+      measureRawColumnChunk.setFileReader(fileReader);
       measureRawColumnChunk.setPagesCount(1);
       dataChunks[index] = measureRawColumnChunk;
       runningLength += currentLength;
@@ -108,7 +108,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
     return dataChunks;
   }
 
-  public ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
+  public ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
       int pageNumber) throws IOException, MemoryException {
     int copyPoint = (int) measureRawColumnChunk.getOffSet();
     int blockIndex = measureRawColumnChunk.getColumnIndex();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
index 4f6987b..6dc02a3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -70,7 +70,7 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
    * @param columnIndex         column to be read
    * @return measure raw chunk
    */
-  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader,
+  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader,
       int columnIndex) throws IOException {
     int dataLength = 0;
     // to calculate the length of the data to be read
@@ -99,9 +99,8 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
         dataChunk);
   }
 
-  protected MeasureRawColumnChunk getMeasureRawColumnChunk(FileHolder fileReader,
-      int columnIndex, long offset, int dataLength, ByteBuffer buffer,
-      DataChunk3 dataChunk) {
+  MeasureRawColumnChunk getMeasureRawColumnChunk(FileReader fileReader, int columnIndex,
+      long offset, int dataLength, ByteBuffer buffer, DataChunk3 dataChunk) {
     // creating a raw chunks instance and filling all the details
     MeasureRawColumnChunk rawColumnChunk =
         new MeasureRawColumnChunk(columnIndex, buffer, offset, dataLength, this);
@@ -140,30 +139,30 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
    *
    * @param fileReader
    *        reader which will be used to read the measure columns data from file
-   * @param startColumnBlockletIndex
-   *        blocklet index of the first measure column
-   * @param endColumnBlockletIndex
-   *        blocklet index of the last measure column
+   * @param startColumnIndex
+   *        column index of the first measure column
+   * @param endColumnIndex
+   *        column index of the last measure column
    * @return MeasureRawColumnChunk array
    */
-  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
-      int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
+  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
+      int startColumnIndex, int endColumnIndex) throws IOException {
     // to calculate the length of the data to be read
     // column we can subtract the offset of start column offset with
     // end column+1 offset and get the total length.
-    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex);
+    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnIndex);
     ByteBuffer buffer = null;
     // read the data from carbon data file
     synchronized (fileReader) {
       buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset,
-          (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
+          (int) (measureColumnChunkOffsets.get(endColumnIndex + 1) - currentMeasureOffset));
     }
     // create raw chunk for each measure column
     MeasureRawColumnChunk[] measureDataChunk =
-        new MeasureRawColumnChunk[endColumnBlockletIndex - startColumnBlockletIndex + 1];
+        new MeasureRawColumnChunk[endColumnIndex - startColumnIndex + 1];
     int runningLength = 0;
     int index = 0;
-    for (int i = startColumnBlockletIndex; i <= endColumnBlockletIndex; i++) {
+    for (int i = startColumnIndex; i <= endColumnIndex; i++) {
       int currentLength =
           (int) (measureColumnChunkOffsets.get(i + 1) - measureColumnChunkOffsets.get(i));
       DataChunk3 dataChunk =
@@ -180,25 +179,25 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
   /**
    * Below method will be used to convert the compressed measure chunk raw data to actual data
    *
-   * @param rawColumnPage measure raw chunk
+   * @param rawColumnChunk measure raw chunk
    * @param pageNumber            number
-   * @return DimensionColumnDataChunk
+   * @return DimensionColumnPage
    */
   @Override
-  public ColumnPage convertToColumnPage(
-      MeasureRawColumnChunk rawColumnPage, int pageNumber)
+  public ColumnPage decodeColumnPage(
+      MeasureRawColumnChunk rawColumnChunk, int pageNumber)
       throws IOException, MemoryException {
     // data chunk of blocklet column
-    DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
+    DataChunk3 dataChunk3 = rawColumnChunk.getDataChunkV3();
     // data chunk of page
     DataChunk2 pageMetadata = dataChunk3.getData_chunk_list().get(pageNumber);
     // calculating the start point of data
     // as buffer can contain multiple column data, start point will be datachunkoffset +
     // data chunk length + page offset
-    int offset = (int) rawColumnPage.getOffSet() +
-        measureColumnChunkLength.get(rawColumnPage.getColumnIndex()) +
+    int offset = (int) rawColumnChunk.getOffSet() +
+        measureColumnChunkLength.get(rawColumnChunk.getColumnIndex()) +
         dataChunk3.getPage_offset().get(pageNumber);
-    ColumnPage decodedPage = decodeMeasure(pageMetadata, rawColumnPage.getRawData(), offset);
+    ColumnPage decodedPage = decodeMeasure(pageMetadata, rawColumnChunk.getRawData(), offset);
     decodedPage.setNullBits(getNullBitSet(pageMetadata.presence));
     return decodedPage;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
index 31ff4c0..6b37575 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
@@ -20,7 +20,7 @@ import java.io.ByteArrayInputStream;
 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.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -66,7 +66,7 @@ public class CompressedMsrChunkFileBasedPageLevelReaderV3
    * @param blockletColumnIndex blocklet index of the column in carbon data file
    * @return measure raw chunk
    */
-  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader,
+  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader,
       int blockletColumnIndex) throws IOException {
     int dataLength = 0;
     // to calculate the length of the data to be read
@@ -110,7 +110,7 @@ public class CompressedMsrChunkFileBasedPageLevelReaderV3
    * @param endColumnBlockletIndex   blocklet index of the last measure column
    * @return MeasureRawColumnChunk array
    */
-  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
+  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
     // create raw chunk for each measure column
     MeasureRawColumnChunk[] measureDataChunk =
@@ -130,7 +130,7 @@ public class CompressedMsrChunkFileBasedPageLevelReaderV3
    * @param pageNumber            number
    * @return DimensionColumnDataChunk
    */
-  @Override public ColumnPage convertToColumnPage(
+  @Override public ColumnPage decodeColumnPage(
       MeasureRawColumnChunk rawColumnPage, int pageNumber)
       throws IOException, MemoryException {
     // data chunk of blocklet column

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
index fbdb499..c89ecc3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
@@ -17,48 +17,45 @@
 
 package org.apache.carbondata.core.datastore.chunk.store;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
 
-public class ColumnPageWrapper implements DimensionColumnDataChunk {
+public class ColumnPageWrapper implements DimensionColumnPage {
 
   private ColumnPage columnPage;
-  private int columnValueSize;
 
-  public ColumnPageWrapper(ColumnPage columnPage, int columnValueSize) {
+  public ColumnPageWrapper(ColumnPage columnPage) {
     this.columnPage = columnPage;
-    this.columnValueSize = columnValueSize;
   }
 
   @Override
-  public int fillChunkData(byte[] data, int offset, int columnIndex,
-      KeyStructureInfo restructuringInfo) {
+  public int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
+  public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
       KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
+  public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
       KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column,
+  public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int chunkIndex,
       KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public byte[] getChunkData(int columnIndex) {
-    return columnPage.getBytes(columnIndex);
+  public byte[] getChunkData(int rowId) {
+    return columnPage.getBytes(rowId);
   }
 
   @Override
@@ -66,7 +63,7 @@ public class ColumnPageWrapper implements DimensionColumnDataChunk {
     throw new UnsupportedOperationException("internal error");
   }
 
-  @Override public int getInvertedReverseIndex(int invertedIndex) {
+  @Override public int getInvertedReverseIndex(int rowId) {
     throw new UnsupportedOperationException("internal error");
   }
 
@@ -76,17 +73,12 @@ public class ColumnPageWrapper implements DimensionColumnDataChunk {
   }
 
   @Override
-  public int getColumnValueSize() {
-    return columnValueSize;
-  }
-
-  @Override
   public boolean isExplicitSorted() {
     return false;
   }
 
   @Override
-  public int compareTo(int index, byte[] compareValue) {
+  public int compareTo(int rowId, byte[] compareValue) {
     throw new UnsupportedOperationException("internal error");
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
index 5072c75..28aed5b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
@@ -69,10 +69,10 @@ public interface DimensionDataChunkStore {
 
   /**
    * Below method will be used to get the reverse Inverted Index
-   * @param invertedIndex
+   * @param rowId
    * @return reverse Inverted Index
    */
-  int getInvertedReverseIndex(int invertedIndex);
+  int getInvertedReverseIndex(int rowId);
 
   /**
    * Below method will be used to get the surrogate key of the
@@ -102,9 +102,9 @@ public interface DimensionDataChunkStore {
   /**
    * to compare the two byte array
    *
-   * @param index        index of first byte array
+   * @param rowId        index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  int compareTo(int index, byte[] compareValue);
+  int compareTo(int rowId, byte[] compareValue);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
index 8656878..41218d0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
@@ -97,13 +97,13 @@ public class SafeFixedLengthDimensionDataChunkStore extends SafeAbsractDimension
   /**
    * to compare the two byte array
    *
-   * @param index        index of first byte array
+   * @param rowId        index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     return ByteUtil.UnsafeComparer.INSTANCE
-        .compareTo(data, index * columnValueSize, columnValueSize, compareValue, 0,
+        .compareTo(data, rowId * columnValueSize, columnValueSize, compareValue, 0,
             columnValueSize);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
index db83198..f498c6e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
@@ -162,7 +162,7 @@ public class SafeVariableLengthDimensionDataChunkStore extends SafeAbsractDimens
     }
   }
 
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     // now to get the row from memory block we need to do following thing
     // 1. first get the current offset
     // 2. if it's not a last row- get the next row offset
@@ -171,11 +171,11 @@ public class SafeVariableLengthDimensionDataChunkStore extends SafeAbsractDimens
     // length
 
     // get the offset of set of data
-    int currentDataOffset = dataOffsets[index];
+    int currentDataOffset = dataOffsets[rowId];
     short length = 0;
     // calculating the length of data
-    if (index < numberOfRows - 1) {
-      length = (short) (dataOffsets[index + 1] - (currentDataOffset
+    if (rowId < numberOfRows - 1) {
+      length = (short) (dataOffsets[rowId + 1] - (currentDataOffset
           + CarbonCommonConstants.SHORT_SIZE_IN_BYTE));
     } else {
       // for last record


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 07989b2..d5c2609 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -19,7 +19,13 @@ package org.apache.carbondata.core.metadata.schema.table;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -33,7 +39,10 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.model.QueryProjection;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
@@ -136,10 +145,7 @@ public class CarbonTable implements Serializable {
   /**
    * During creation of TableInfo from hivemetastore the DataMapSchemas and the columns
    * DataTypes are not converted to the appropriate child classes.
-   *
    * This method will cast the same to the appropriate classes
-   *
-   * @param tableInfo
    */
   private static void updateTableInfo(TableInfo tableInfo) {
     List<DataMapSchema> dataMapSchemas = new ArrayList<>();
@@ -153,8 +159,9 @@ public class CarbonTable implements Serializable {
     }
     tableInfo.setDataMapSchemaList(dataMapSchemas);
     for (ColumnSchema columnSchema : tableInfo.getFactTable().getListOfColumns()) {
-      columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-          columnSchema.getPrecision(), columnSchema.getScale()));
+      columnSchema.setDataType(
+          DataTypeUtil.valueOf(
+              columnSchema.getDataType(), columnSchema.getPrecision(), columnSchema.getScale()));
     }
     List<DataMapSchema> childSchema = tableInfo.getDataMapSchemaList();
     for (DataMapSchema dataMapSchema : childSchema) {
@@ -168,10 +175,11 @@ public class CarbonTable implements Serializable {
       }
     }
     if (tableInfo.getFactTable().getBucketingInfo() != null) {
-      for (ColumnSchema columnSchema : tableInfo.getFactTable()
-          .getBucketingInfo().getListOfColumns()) {
-        columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-            columnSchema.getPrecision(), columnSchema.getScale()));
+      for (ColumnSchema columnSchema :
+          tableInfo.getFactTable().getBucketingInfo().getListOfColumns()) {
+        columnSchema.setDataType(
+            DataTypeUtil.valueOf(
+                columnSchema.getDataType(), columnSchema.getPrecision(), columnSchema.getScale()));
       }
     }
     if (tableInfo.getFactTable().getPartitionInfo() != null) {
@@ -211,6 +219,7 @@ public class CarbonTable implements Serializable {
 
   /**
    * fill columns as per user provided order
+   *
    * @param tableName
    */
   private void fillCreateOrderColumn(String tableName) {
@@ -229,7 +238,6 @@ public class CarbonTable implements Serializable {
     this.createOrderColumn.put(tableName, columns);
   }
 
-
   /**
    * Fill allDimensions and allMeasures for carbon table
    *
@@ -254,8 +262,8 @@ public class CarbonTable implements Serializable {
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++,
-                  columnSchema.getSchemaOrdinal(), -1, -1, ++complexTypeOrdinal);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
+                  -1, -1, ++complexTypeOrdinal);
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           allDimensions.add(complexDimension);
           dimensionOrdinal =
@@ -268,9 +276,8 @@ public class CarbonTable implements Serializable {
             this.numberOfSortColumns++;
           }
           if (!columnSchema.getEncodingList().contains(Encoding.DICTIONARY)) {
-            CarbonDimension dimension =
-                new CarbonDimension(columnSchema, dimensionOrdinal++,
-                    columnSchema.getSchemaOrdinal(), -1, -1, -1);
+            CarbonDimension dimension = new CarbonDimension(
+                columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), -1, -1, -1);
             if (!columnSchema.isInvisible() && columnSchema.isSortColumn()) {
               this.numberOfNoDictSortColumns++;
             }
@@ -278,25 +285,25 @@ public class CarbonTable implements Serializable {
             primitiveDimensions.add(dimension);
           } else if (columnSchema.getEncodingList().contains(Encoding.DICTIONARY)
               && columnSchema.getColumnGroupId() == -1) {
-            CarbonDimension dimension =
-                new CarbonDimension(columnSchema, dimensionOrdinal++,
-                    columnSchema.getSchemaOrdinal(), keyOrdinal++, -1, -1);
+            CarbonDimension dimension = new CarbonDimension(
+                columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), keyOrdinal++,
+                -1, -1);
             allDimensions.add(dimension);
             primitiveDimensions.add(dimension);
           } else {
             columnGroupOrdinal =
                 previousColumnGroupId == columnSchema.getColumnGroupId() ? ++columnGroupOrdinal : 0;
             previousColumnGroupId = columnSchema.getColumnGroupId();
-            CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++,
-                columnSchema.getSchemaOrdinal(), keyOrdinal++,
+            CarbonDimension dimension = new CarbonDimension(
+                columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), keyOrdinal++,
                 columnGroupOrdinal, -1);
             allDimensions.add(dimension);
             primitiveDimensions.add(dimension);
           }
         }
       } else {
-        allMeasures.add(new CarbonMeasure(columnSchema, measureOrdinal++,
-            columnSchema.getSchemaOrdinal()));
+        allMeasures.add(
+            new CarbonMeasure(columnSchema, measureOrdinal++, columnSchema.getSchemaOrdinal()));
       }
     }
     fillVisibleDimensions(tableSchema.getTableName());
@@ -347,8 +354,8 @@ public class CarbonTable implements Serializable {
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++,
-                  columnSchema.getSchemaOrdinal(), -1, -1, -1);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
+                  -1, -1, -1);
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           parentDimension.getListOfChildDimensions().add(complexDimension);
           dimensionOrdinal =
@@ -356,8 +363,8 @@ public class CarbonTable implements Serializable {
                   listOfColumns, complexDimension, primitiveDimensions);
         } else {
           CarbonDimension carbonDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++,
-                  columnSchema.getSchemaOrdinal(), -1, -1, -1);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
+                  -1, -1, -1);
           parentDimension.getListOfChildDimensions().add(carbonDimension);
           primitiveDimensions.add(carbonDimension);
         }
@@ -569,6 +576,7 @@ public class CarbonTable implements Serializable {
     }
     return null;
   }
+
   /**
    * gets all children dimension for complex type
    *
@@ -771,18 +779,13 @@ public class CarbonTable implements Serializable {
     return dimensionOrdinalMax;
   }
 
-  public void setDimensionOrdinalMax(int dimensionOrdinalMax) {
-    this.dimensionOrdinalMax = dimensionOrdinalMax;
-  }
-
-
   public boolean hasDataMapSchema() {
     return hasDataMapSchema;
   }
 
   public boolean isChildDataMap() {
-    return null != tableInfo.getParentRelationIdentifiers()
-        && !tableInfo.getParentRelationIdentifiers().isEmpty();
+    return null != tableInfo.getParentRelationIdentifiers() &&
+        !tableInfo.getParentRelationIdentifiers().isEmpty();
   }
 
   /**
@@ -806,4 +809,63 @@ public class CarbonTable implements Serializable {
     }
     return dataSize + indexSize;
   }
+
+  /**
+   * Create a new QueryModel with projection all columns in the table.
+   */
+  public QueryModel createQueryModelWithProjectAllColumns(DataTypeConverter converter) {
+    QueryProjection projection = new QueryProjection();
+
+    List<CarbonDimension> dimensions = getDimensionByTableName(getTableName());
+    for (int i = 0; i < dimensions.size(); i++) {
+      projection.addDimension(dimensions.get(i), i);
+    }
+    List<CarbonMeasure> measures = getMeasureByTableName(getTableName());
+    for (int i = 0; i < measures.size(); i++) {
+      projection.addMeasure(measures.get(i), i);
+    }
+    QueryModel model = QueryModel.newInstance(this);
+    model.setProjection(projection);
+    model.setConverter(converter);
+    return model;
+  }
+
+  /**
+   * Create a new QueryModel with specified projection
+   */
+  public QueryModel createQueryWithProjection(String[] projectionColumnNames,
+      DataTypeConverter converter) {
+    QueryProjection projection = createProjection(projectionColumnNames);
+    QueryModel queryModel = QueryModel.newInstance(this);
+    queryModel.setProjection(projection);
+    queryModel.setConverter(converter);
+    return queryModel;
+  }
+
+  private QueryProjection createProjection(String[] projectionColumnNames) {
+    String factTableName = getTableName();
+    QueryProjection projection = new QueryProjection();
+    // fill dimensions
+    // If columns are null, set all dimensions and measures
+    int i = 0;
+    if (projectionColumnNames != null) {
+      for (String projectionColumnName : projectionColumnNames) {
+        CarbonDimension dimension = getDimensionByName(factTableName, projectionColumnName);
+        if (dimension != null) {
+          projection.addDimension(dimension, i);
+          i++;
+        } else {
+          CarbonMeasure measure = getMeasureByName(factTableName, projectionColumnName);
+          if (measure == null) {
+            throw new RuntimeException(projectionColumnName +
+                " column not found in the table " + factTableName);
+          }
+          projection.addMeasure(measure, i);
+          i++;
+        }
+      }
+    }
+
+    return projection;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
index 20e539b..9a1dad1 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
@@ -26,18 +26,6 @@ import java.io.Serializable;
  */
 public class RelationIdentifier implements Serializable, Writable {
 
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  public void setTableId(String tableId) {
-    this.tableId = tableId;
-  }
-
   private String databaseName;
 
   private String tableName;
@@ -50,10 +38,6 @@ public class RelationIdentifier implements Serializable, Writable {
     this.tableId = tableId;
   }
 
-  public RelationIdentifier() {
-    this(null, null, null);
-  }
-
   public String getDatabaseName() {
     return databaseName;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 4deafd4..0d796c7 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -24,7 +24,11 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
index 13e7d49..c888418 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
@@ -45,7 +45,7 @@ public class CarbonColumn implements Serializable {
   /**
    * order in which user has created table
    */
-  protected int schemaOrdinal;
+  private int schemaOrdinal;
 
   /**
    * Column identifier

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
index b4c052f..d15cde7 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
@@ -122,18 +122,6 @@ public class CarbonDimension extends CarbonColumn {
     return getEncoder().contains(Encoding.DICTIONARY);
   }
 
-  public int getNumDimensionsExpanded() {
-    if (listOfChildDimensions == null) {
-      // there is no child, return 1 column
-      return 1;
-    }
-    int columnCount = 1;
-    for (CarbonDimension dimension: listOfChildDimensions) {
-      columnCount += dimension.getNumDimensionsExpanded();
-    }
-    return columnCount;
-  }
-
   /**
    * @return is column participated in sorting or not
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index 4b99307..920155b 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
@@ -125,8 +125,6 @@ public class CarbonUpdateUtil {
       lockStatus = updateLock.lockWithRetries();
       if (lockStatus) {
 
-        AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
         // read the existing file if present and update the same.
         SegmentUpdateDetails[] oldDetails = segmentUpdateStatusManager
                 .getUpdateStatusDetails();
@@ -367,16 +365,6 @@ public class CarbonUpdateUtil {
         .substring(0, completeBlockName.lastIndexOf(CarbonCommonConstants.HYPHEN));
   }
 
-  /**
-   * returns segment id from segment name
-   *
-   * @param segmentName
-   * @return
-   */
-  public static String getSegmentId(String segmentName) {
-    return segmentName.split(CarbonCommonConstants.UNDERSCORE)[1];
-  }
-
   public static long getLatestTaskIdForSegment(String segmentId, CarbonTablePath tablePath) {
     String segmentDirPath = tablePath.getCarbonDataDirectoryPath(segmentId);
 
@@ -501,7 +489,7 @@ public class CarbonUpdateUtil {
 
           // aborted scenario.
           invalidDeleteDeltaFiles = updateStatusManager
-              .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, false,
+              .getDeleteDeltaInvalidFilesList(block, false,
                   allSegmentFiles, isAbortedFile);
           for (CarbonFile invalidFile : invalidDeleteDeltaFiles) {
             boolean doForceDelete = true;
@@ -511,7 +499,7 @@ public class CarbonUpdateUtil {
           // case 1
           if (CarbonUpdateUtil.isBlockInvalid(block.getSegmentStatus())) {
             completeListOfDeleteDeltaFiles = updateStatusManager
-                    .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, true,
+                    .getDeleteDeltaInvalidFilesList(block, true,
                             allSegmentFiles, isInvalidFile);
             for (CarbonFile invalidFile : completeListOfDeleteDeltaFiles) {
 
@@ -519,7 +507,7 @@ public class CarbonUpdateUtil {
             }
 
             CarbonFile[] blockRelatedFiles = updateStatusManager
-                    .getAllBlockRelatedFiles(block.getBlockName(), allSegmentFiles,
+                    .getAllBlockRelatedFiles(allSegmentFiles,
                             block.getActualBlockName());
 
             // now for each invalid index file need to check the query execution time out
@@ -533,7 +521,7 @@ public class CarbonUpdateUtil {
 
           } else {
             invalidDeleteDeltaFiles = updateStatusManager
-                    .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, false,
+                    .getDeleteDeltaInvalidFilesList(block, false,
                             allSegmentFiles, isInvalidFile);
             for (CarbonFile invalidFile : invalidDeleteDeltaFiles) {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
index 4275cca..e5fdf6f 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
@@ -45,14 +45,6 @@ public class DeleteDeltaBlockDetails implements Serializable {
     blockletDetails = new ArrayList<DeleteDeltaBlockletDetails>();
   }
 
-  public String getBlockName() {
-    return blockName;
-  }
-
-  public void setBlockName(String blockName) {
-    this.blockName = blockName;
-  }
-
   @Override public boolean equals(Object obj) {
     if (this == obj) return true;
     if (obj == null || !(obj instanceof DeleteDeltaBlockDetails)) return false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
index 22785fa..6185134 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
@@ -46,14 +46,6 @@ public class DeleteDeltaBlockletDetails implements Serializable {
     return deletedRows.add(row);
   }
 
-  public String getId() {
-    return id;
-  }
-
-  public void setId(String id) {
-    this.id = id;
-  }
-
   public Integer getPageId() {
     return pageId;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java b/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java
deleted file mode 100644
index 3c0dd4f..0000000
--- a/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java
+++ /dev/null
@@ -1,87 +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.mutate.data;
-
-import java.util.Map;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
-
-/**
- * This class is responsible for loading delete delta file cache based on
- * blocklet id of a particular block
- */
-public class BlockletDeleteDeltaCacheLoader implements DeleteDeltaCacheLoaderIntf {
-  private String blockletID;
-  private DataRefNode blockletNode;
-  private AbsoluteTableIdentifier absoluteIdentifier;
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(BlockletDeleteDeltaCacheLoader.class.getName());
-
-  public BlockletDeleteDeltaCacheLoader(String blockletID, DataRefNode blockletNode,
-      AbsoluteTableIdentifier absoluteIdentifier) {
-    this.blockletID = blockletID;
-    this.blockletNode = blockletNode;
-    this.absoluteIdentifier = absoluteIdentifier;
-  }
-
-  /**
-   * This method will load the delete delta cache based on blocklet id of particular block with
-   * the help of SegmentUpdateStatusManager.
-   */
-  public void loadDeleteDeltaFileDataToCache() {
-    SegmentUpdateStatusManager segmentUpdateStatusManager =
-        new SegmentUpdateStatusManager(absoluteIdentifier);
-    Map<Integer, Integer[]> deleteDeltaFileData = null;
-    BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache = null;
-    if (null == blockletNode.getDeleteDeltaDataCache()) {
-      try {
-        deleteDeltaFileData =
-            segmentUpdateStatusManager.getDeleteDeltaDataFromAllFiles(blockletID);
-        deleteDeltaDataCache = new BlockletLevelDeleteDeltaDataCache(deleteDeltaFileData,
-            segmentUpdateStatusManager.getTimestampForRefreshCache(blockletID, null));
-      } catch (Exception e) {
-        if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug("Unable to retrieve delete delta files");
-        }
-      }
-    } else {
-      deleteDeltaDataCache = blockletNode.getDeleteDeltaDataCache();
-      // if already cache is present then validate the cache using timestamp
-      String cacheTimeStamp = segmentUpdateStatusManager
-          .getTimestampForRefreshCache(blockletID, deleteDeltaDataCache.getCacheTimeStamp());
-      if (null != cacheTimeStamp) {
-        try {
-          deleteDeltaFileData =
-              segmentUpdateStatusManager.getDeleteDeltaDataFromAllFiles(blockletID);
-          deleteDeltaDataCache = new BlockletLevelDeleteDeltaDataCache(deleteDeltaFileData,
-              segmentUpdateStatusManager.getTimestampForRefreshCache(blockletID, cacheTimeStamp));
-        } catch (Exception e) {
-          if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Unable to retrieve delete delta files");
-          }
-        }
-      }
-    }
-    blockletNode.setDeleteDeltaDataCache(deleteDeltaDataCache);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
index b14660c..88b3065 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.core.scan.collector;
 
 import java.util.List;
 
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 
 /**
@@ -27,17 +27,13 @@ import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 public interface ScannedResultCollector {
 
   /**
-   * Below method will be used to aggregate the scanned result
-   *
-   * @param scannedResult scanned result
-   * @return how many records was aggregated
+   * Return the aggregated scanned result in list of rows
    */
-  List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize);
+  List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize);
 
   /**
-   * Collects data in columnar format.
-   * @param scannedResult
-   * @param columnarBatch
+   * Collects result and set it in the specified columnar batch
    */
-  void collectVectorBatch(AbstractScannedResult scannedResult, CarbonColumnarBatch columnarBatch);
+  void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
+      CarbonColumnarBatch columnarBatch);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
index dc78ac6..694271e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
@@ -19,8 +19,6 @@ package org.apache.carbondata.core.scan.collector.impl;
 import java.math.BigDecimal;
 import java.math.RoundingMode;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -29,8 +27,8 @@ import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
 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.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
@@ -39,38 +37,35 @@ import org.apache.carbondata.core.util.DataTypeUtil;
  */
 public abstract class AbstractScannedResultCollector implements ScannedResultCollector {
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractScannedResultCollector.class.getName());
-
   /**
    * table block execution infos
    */
-  protected BlockExecutionInfo tableBlockExecutionInfos;
+  BlockExecutionInfo executionInfo;
 
   /**
    * maintains the measure information like datatype, ordinal, measure existence
    */
-  protected MeasureInfo measureInfo;
+  MeasureInfo measureInfo;
 
   /**
    * maintains the dimension information like datatype, ordinal, measure existence
    */
-  protected DimensionInfo dimensionInfo;
+  DimensionInfo dimensionInfo;
 
-  public AbstractScannedResultCollector(BlockExecutionInfo blockExecutionInfos) {
-    this.tableBlockExecutionInfos = blockExecutionInfos;
+  AbstractScannedResultCollector(BlockExecutionInfo blockExecutionInfos) {
+    this.executionInfo = blockExecutionInfos;
     measureInfo = blockExecutionInfos.getMeasureInfo();
     dimensionInfo = blockExecutionInfos.getDimensionInfo();
   }
 
   protected void fillMeasureData(Object[] msrValues, int offset,
-      AbstractScannedResult scannedResult) {
+      BlockletScannedResult scannedResult) {
     int measureExistIndex = 0;
     for (short i = 0; i < measureInfo.getMeasureDataTypes().length; i++) {
       // if measure exists is block then pass measure column
       // data chunk to the collector
       if (measureInfo.getMeasureExists()[i]) {
-        QueryMeasure queryMeasure = tableBlockExecutionInfos.getQueryMeasures()[measureExistIndex];
+        ProjectionMeasure queryMeasure = executionInfo.getProjectionMeasures()[measureExistIndex];
         msrValues[i + offset] = getMeasureData(
             scannedResult.getMeasureChunk(measureInfo.getMeasureOrdinals()[measureExistIndex]),
             scannedResult.getCurrentRowId(), queryMeasure.getMeasure());
@@ -87,8 +82,7 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
     }
   }
 
-  protected Object getMeasureData(ColumnPage dataChunk, int index,
-      CarbonMeasure carbonMeasure) {
+  Object getMeasureData(ColumnPage dataChunk, int index, CarbonMeasure carbonMeasure) {
     if (!dataChunk.getNullBits().get(index)) {
       DataType dataType = carbonMeasure.getDataType();
       if (dataType == DataTypes.BOOLEAN) {
@@ -114,7 +108,8 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
     return null;
   }
 
-  @Override public void collectVectorBatch(AbstractScannedResult scannedResult,
+  @Override
+  public void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch) {
     throw new UnsupportedOperationException("Works only for batch collectors");
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
index cf6e6d6..4322034 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
@@ -29,9 +29,9 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
@@ -42,49 +42,50 @@ import org.apache.commons.lang3.ArrayUtils;
  */
 public class DictionaryBasedResultCollector extends AbstractScannedResultCollector {
 
-  protected QueryDimension[] queryDimensions;
+  protected ProjectionDimension[] queryDimensions;
 
-  protected QueryMeasure[] queryMeasures;
+  protected ProjectionMeasure[] queryMeasures;
 
-  protected DirectDictionaryGenerator[] directDictionaryGenerators;
+  private DirectDictionaryGenerator[] directDictionaryGenerators;
 
   /**
    * query order
    */
   protected int[] order;
 
-  protected int[] actualIndexInSurrogateKey;
+  private int[] actualIndexInSurrogateKey;
 
-  protected boolean[] dictionaryEncodingArray;
+  boolean[] dictionaryEncodingArray;
 
-  protected boolean[] directDictionaryEncodingArray;
+  boolean[] directDictionaryEncodingArray;
 
-  protected boolean[] implictColumnArray;
+  private boolean[] implictColumnArray;
 
-  protected boolean[] complexDataTypeArray;
+  private boolean[] complexDataTypeArray;
 
-  protected int dictionaryColumnIndex;
-  protected int noDictionaryColumnIndex;
-  protected int complexTypeColumnIndex;
+  int dictionaryColumnIndex;
+  int noDictionaryColumnIndex;
+  int complexTypeColumnIndex;
 
-  protected boolean isDimensionExists;
+  boolean isDimensionExists;
 
-  protected Map<Integer, GenericQueryType> comlexDimensionInfoMap;
+  private Map<Integer, GenericQueryType> comlexDimensionInfoMap;
 
   public DictionaryBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
-    queryDimensions = tableBlockExecutionInfos.getQueryDimensions();
-    queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+    queryDimensions = executionInfo.getProjectionDimensions();
+    queryMeasures = executionInfo.getProjectionMeasures();
     initDimensionAndMeasureIndexesForFillingData();
     isDimensionExists = queryDimensions.length > 0;
-    this.comlexDimensionInfoMap = tableBlockExecutionInfos.getComlexDimensionInfoMap();
+    this.comlexDimensionInfoMap = executionInfo.getComlexDimensionInfoMap();
   }
 
   /**
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
 
     // scan the record and add to list
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
@@ -118,13 +119,13 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     return listBasedResult;
   }
 
-  protected void fillDimensionData(AbstractScannedResult scannedResult, int[] surrogateResult,
+  void fillDimensionData(BlockletScannedResult scannedResult, int[] surrogateResult,
       byte[][] noDictionaryKeys, byte[][] complexTypeKeyArray,
       Map<Integer, GenericQueryType> comlexDimensionInfoMap, Object[] row, int i) {
     if (!dictionaryEncodingArray[i]) {
       if (implictColumnArray[i]) {
         if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID
-            .equals(queryDimensions[i].getDimension().getColName())) {
+            .equals(queryDimensions[i].getColumnName())) {
           row[order[i]] = DataTypeUtil.getDataBasedOnDataType(
               scannedResult.getBlockletId() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult
                   .getCurrentPageCounter() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult
@@ -153,7 +154,7 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     }
   }
 
-  protected void fillMeasureData(AbstractScannedResult scannedResult, Object[] row) {
+  void fillMeasureData(BlockletScannedResult scannedResult, Object[] row) {
     if (measureInfo.getMeasureDataTypes().length > 0) {
       Object[] msrValues = new Object[measureInfo.getMeasureDataTypes().length];
       fillMeasureData(msrValues, 0, scannedResult);
@@ -163,7 +164,7 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     }
   }
 
-  protected void initDimensionAndMeasureIndexesForFillingData() {
+  void initDimensionAndMeasureIndexesForFillingData() {
     List<Integer> dictionaryIndexes = new ArrayList<Integer>();
     for (int i = 0; i < queryDimensions.length; i++) {
       if (queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY) || queryDimensions[i]
@@ -190,10 +191,10 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     complexDataTypeArray = CarbonUtil.getComplexDataTypeArray(queryDimensions);
     order = new int[queryDimensions.length + queryMeasures.length];
     for (int i = 0; i < queryDimensions.length; i++) {
-      order[i] = queryDimensions[i].getQueryOrder();
+      order[i] = queryDimensions[i].getOrdinal();
     }
     for (int i = 0; i < queryMeasures.length; i++) {
-      order[i + queryDimensions.length] = queryMeasures[i].getQueryOrder();
+      order[i + queryDimensions.length] = queryMeasures[i].getOrdinal();
     }
     directDictionaryGenerators = new DirectDictionaryGenerator[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
index 5e6c99a..e6cc817 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
@@ -23,9 +23,9 @@ import java.util.List;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
 import org.apache.carbondata.core.scan.result.vector.MeasureDataVectorProcessor;
@@ -35,35 +35,35 @@ import org.apache.carbondata.core.scan.result.vector.MeasureDataVectorProcessor;
  */
 public class DictionaryBasedVectorResultCollector extends AbstractScannedResultCollector {
 
-  protected QueryDimension[] queryDimensions;
+  protected ProjectionDimension[] queryDimensions;
 
-  protected QueryMeasure[] queryMeasures;
+  protected ProjectionMeasure[] queryMeasures;
 
-  protected ColumnVectorInfo[] dictionaryInfo;
+  private ColumnVectorInfo[] dictionaryInfo;
 
-  protected ColumnVectorInfo[] noDictionaryInfo;
+  private ColumnVectorInfo[] noDictionaryInfo;
 
-  protected ColumnVectorInfo[] complexInfo;
+  private ColumnVectorInfo[] complexInfo;
 
-  protected ColumnVectorInfo[] measureColumnInfo;
+  private ColumnVectorInfo[] measureColumnInfo;
 
-  protected ColumnVectorInfo[] allColumnInfo;
+  ColumnVectorInfo[] allColumnInfo;
 
-  protected ColumnVectorInfo[] implictColumnInfo;
+  private ColumnVectorInfo[] implictColumnInfo;
 
   public DictionaryBasedVectorResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
     // initialize only if the current block is not a restructured block else the initialization
     // will be taken care by RestructureBasedVectorResultCollector
     if (!blockExecutionInfos.isRestructuredBlock()) {
-      queryDimensions = tableBlockExecutionInfos.getQueryDimensions();
-      queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+      queryDimensions = executionInfo.getProjectionDimensions();
+      queryMeasures = executionInfo.getProjectionMeasures();
       allColumnInfo = new ColumnVectorInfo[queryDimensions.length + queryMeasures.length];
       prepareDimensionAndMeasureColumnVectors();
     }
   }
 
-  protected void prepareDimensionAndMeasureColumnVectors() {
+  void prepareDimensionAndMeasureColumnVectors() {
     measureColumnInfo = new ColumnVectorInfo[queryMeasures.length];
     List<ColumnVectorInfo> dictInfoList = new ArrayList<>();
     List<ColumnVectorInfo> noDictInfoList = new ArrayList<>();
@@ -78,13 +78,13 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
         implictColumnList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         noDictInfoList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         dictInfoList.add(columnVectorInfo);
@@ -92,21 +92,21 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
         columnVectorInfo.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
             .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else if (queryDimensions[i].getDimension().isComplex()) {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         complexList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
         columnVectorInfo.genericQueryType =
-            tableBlockExecutionInfos.getComlexDimensionInfoMap().get(columnVectorInfo.ordinal);
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+            executionInfo.getComlexDimensionInfoMap().get(columnVectorInfo.ordinal);
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         dictInfoList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       }
     }
     for (int i = 0; i < queryMeasures.length; i++) {
@@ -116,7 +116,7 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
       columnVectorInfo.ordinal = queryMeasures[i].getMeasure().getOrdinal();
       columnVectorInfo.measure = queryMeasures[i];
       this.measureColumnInfo[i] = columnVectorInfo;
-      allColumnInfo[queryMeasures[i].getQueryOrder()] = columnVectorInfo;
+      allColumnInfo[queryMeasures[i].getOrdinal()] = columnVectorInfo;
     }
     dictionaryInfo = dictInfoList.toArray(new ColumnVectorInfo[dictInfoList.size()]);
     noDictionaryInfo = noDictInfoList.toArray(new ColumnVectorInfo[noDictInfoList.size()]);
@@ -126,11 +126,13 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
     Arrays.sort(complexInfo);
   }
 
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
-    throw new UnsupportedOperationException("collectData is not supported here");
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
+    throw new UnsupportedOperationException("collectResultInRow is not supported here");
   }
 
-  @Override public void collectVectorBatch(AbstractScannedResult scannedResult,
+  @Override
+  public void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch) {
     int numberOfPages = scannedResult.numberOfpages();
     int filteredRows = 0;
@@ -150,14 +152,15 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
         return;
       }
       fillColumnVectorDetails(columnarBatch, rowCounter, requiredRows);
-      filteredRows = scannedResult
-          .markFilteredRows(columnarBatch, rowCounter, requiredRows, columnarBatch.getRowCounter());
-      scanAndFillResult(scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
+      filteredRows = scannedResult.markFilteredRows(
+          columnarBatch, rowCounter, requiredRows, columnarBatch.getRowCounter());
+      fillResultToColumnarBatch(
+          scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
       columnarBatch.setActualSize(columnarBatch.getActualSize() + requiredRows - filteredRows);
     }
   }
 
-  protected void scanAndFillResult(AbstractScannedResult scannedResult,
+  void fillResultToColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch, int rowCounter, int availableRows, int requiredRows) {
     scannedResult.fillColumnarDictionaryBatch(dictionaryInfo);
     scannedResult.fillColumnarNoDictionaryBatch(noDictionaryInfo);
@@ -174,8 +177,8 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
     columnarBatch.setRowCounter(columnarBatch.getRowCounter() + requiredRows);
   }
 
-  protected void fillColumnVectorDetails(CarbonColumnarBatch columnarBatch, int rowCounter,
-      int requiredRows) {
+  void fillColumnVectorDetails(CarbonColumnarBatch columnarBatch, int rowCounter, int requiredRows)
+  {
     for (int i = 0; i < allColumnInfo.length; i++) {
       allColumnInfo[i].size = requiredRows;
       allColumnInfo[i].offset = rowCounter;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
index a09676a..0780675 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
@@ -16,14 +16,12 @@
  */
 package org.apache.carbondata.core.scan.collector.impl;
 
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 
 /**
@@ -31,15 +29,11 @@ import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
  */
 public class RawBasedResultCollector extends AbstractScannedResultCollector {
 
-  protected ByteArrayWrapper wrapper;
+  byte[] dictionaryKeyArray;
 
-  protected byte[] dictionaryKeyArray;
+  byte[][] noDictionaryKeyArray;
 
-  protected byte[][] noDictionaryKeyArray;
-
-  protected byte[][] complexTypeKeyArray;
-
-  protected byte[] implicitColumnByteArray;
+  private byte[][] complexTypeKeyArray;
 
   public RawBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
@@ -49,9 +43,10 @@ public class RawBasedResultCollector extends AbstractScannedResultCollector {
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
-    QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+    ProjectionMeasure[] queryMeasures = executionInfo.getProjectionMeasures();
     // scan the record and add to list
     int rowCounter = 0;
     while (scannedResult.hasNext() && rowCounter < batchSize) {
@@ -65,24 +60,21 @@ public class RawBasedResultCollector extends AbstractScannedResultCollector {
     return listBasedResult;
   }
 
-  protected void prepareRow(AbstractScannedResult scannedResult, List<Object[]> listBasedResult,
-      QueryMeasure[] queryMeasures) {
+  void prepareRow(BlockletScannedResult scannedResult, List<Object[]> listBasedResult,
+      ProjectionMeasure[] queryMeasures) {
     Object[] row = new Object[1 + queryMeasures.length];
-    wrapper = new ByteArrayWrapper();
+    ByteArrayWrapper wrapper = new ByteArrayWrapper();
     wrapper.setDictionaryKey(dictionaryKeyArray);
     wrapper.setNoDictionaryKeys(noDictionaryKeyArray);
     wrapper.setComplexTypesKeys(complexTypeKeyArray);
-    wrapper.setImplicitColumnByteArray(implicitColumnByteArray);
     row[0] = wrapper;
     fillMeasureData(row, 1, scannedResult);
     listBasedResult.add(row);
   }
 
-  protected void scanResultAndGetData(AbstractScannedResult scannedResult) {
+  void scanResultAndGetData(BlockletScannedResult scannedResult) {
     dictionaryKeyArray = scannedResult.getDictionaryKeyArray();
     noDictionaryKeyArray = scannedResult.getNoDictionaryKeyArray();
     complexTypeKeyArray = scannedResult.getComplexTypeKeyArray();
-    implicitColumnByteArray = scannedResult.getBlockletId()
-        .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
index 296d9a6..8b42a4a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
@@ -23,8 +23,8 @@ import java.util.Map;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 
 /**
  * class for handling restructure scenarios for filling result
@@ -35,8 +35,8 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
 
   public RestructureBasedDictionaryResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
-    queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
-    queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures();
+    queryDimensions = executionInfo.getActualQueryDimensions();
+    queryMeasures = executionInfo.getActualQueryMeasures();
     measureDefaultValues = new Object[queryMeasures.length];
     fillMeasureDefaultValues();
     initDimensionAndMeasureIndexesForFillingData();
@@ -61,7 +61,8 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
     // scan the record and add to list
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
     int rowCounter = 0;
@@ -69,7 +70,7 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
     byte[][] noDictionaryKeys;
     byte[][] complexTypeKeyArray;
     Map<Integer, GenericQueryType> comlexDimensionInfoMap =
-        tableBlockExecutionInfos.getComlexDimensionInfoMap();
+        executionInfo.getComlexDimensionInfoMap();
     while (scannedResult.hasNext() && rowCounter < batchSize) {
       Object[] row = new Object[queryDimensions.length + queryMeasures.length];
       if (isDimensionExists) {
@@ -107,13 +108,13 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
   }
 
   protected void fillMeasureData(Object[] msrValues, int offset,
-      AbstractScannedResult scannedResult) {
+      BlockletScannedResult scannedResult) {
     int measureExistIndex = 0;
     for (short i = 0; i < measureInfo.getMeasureDataTypes().length; i++) {
       // if measure exists is block then pass measure column
       // data chunk to the collector
       if (measureInfo.getMeasureExists()[i]) {
-        QueryMeasure queryMeasure = tableBlockExecutionInfos.getQueryMeasures()[measureExistIndex];
+        ProjectionMeasure queryMeasure = executionInfo.getProjectionMeasures()[measureExistIndex];
         msrValues[i + offset] = getMeasureData(
             scannedResult.getMeasureChunk(measureInfo.getMeasureOrdinals()[measureExistIndex]),
             scannedResult.getCurrentRowId(), queryMeasure.getMeasure());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
index b08a1d4..6544a75 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
@@ -30,9 +30,9 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.apache.commons.lang3.ArrayUtils;
@@ -70,15 +70,15 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    */
   private void initRestructuredKeyGenerator() {
     SegmentProperties segmentProperties =
-        tableBlockExecutionInfos.getDataBlock().getSegmentProperties();
-    QueryDimension[] queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
+        executionInfo.getDataBlock().getSegmentProperties();
+    ProjectionDimension[] queryDimensions = executionInfo.getActualQueryDimensions();
     List<Integer> updatedColumnCardinality = new ArrayList<>(queryDimensions.length);
     List<Integer> updatedDimensionPartitioner = new ArrayList<>(queryDimensions.length);
-    int[] dictionaryColumnBlockIndex = tableBlockExecutionInfos.getDictionaryColumnBlockIndex();
+    int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex();
     int dimCounterInCurrentBlock = 0;
     for (int i = 0; i < queryDimensions.length; i++) {
       if (queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        if (tableBlockExecutionInfos.getDimensionInfo().getDimensionExists()[i]) {
+        if (executionInfo.getDimensionInfo().getDimensionExists()[i]) {
           // get the dictionary key ordinal as column cardinality in segment properties
           // will only be for dictionary encoded columns
           CarbonDimension currentBlockDimension = segmentProperties.getDimensions()
@@ -124,8 +124,8 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    */
   private void initCurrentBlockKeyGenerator() {
     SegmentProperties segmentProperties =
-        tableBlockExecutionInfos.getDataBlock().getSegmentProperties();
-    int[] dictionaryColumnBlockIndex = tableBlockExecutionInfos.getDictionaryColumnBlockIndex();
+        executionInfo.getDataBlock().getSegmentProperties();
+    int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex();
     int[] updatedColumnCardinality = new int[dictionaryColumnBlockIndex.length];
     int[] updatedDimensionPartitioner = new int[dictionaryColumnBlockIndex.length];
     for (int i = 0; i < dictionaryColumnBlockIndex.length; i++) {
@@ -149,9 +149,10 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
-    QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures();
+    ProjectionMeasure[] queryMeasures = executionInfo.getActualQueryMeasures();
     // scan the record and add to list
     int rowCounter = 0;
     while (scannedResult.hasNext() && rowCounter < batchSize) {
@@ -179,7 +180,7 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    * @return
    */
   private byte[] fillDictionaryKeyArrayWithLatestSchema(byte[] dictionaryKeyArray) {
-    QueryDimension[] actualQueryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
+    ProjectionDimension[] actualQueryDimensions = executionInfo.getActualQueryDimensions();
     int newKeyArrayLength = dimensionInfo.getNewDictionaryColumnCount();
     long[] keyArray = null;
     if (null != updatedCurrentBlockKeyGenerator) {
@@ -222,7 +223,7 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    * @return
    */
   private byte[][] fillNoDictionaryKeyArrayWithLatestSchema(byte[][] noDictionaryKeyArray) {
-    QueryDimension[] actualQueryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
+    ProjectionDimension[] actualQueryDimensions = executionInfo.getActualQueryDimensions();
     byte[][] noDictionaryKeyArrayWithNewlyAddedColumns =
         new byte[noDictionaryKeyArray.length + dimensionInfo.getNewNoDictionaryColumnCount()][];
     int existingColumnValueIndex = 0;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
index 1e29e98..c54994a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
@@ -43,8 +43,8 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
 
   public RestructureBasedVectorResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
-    queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
-    queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures();
+    queryDimensions = executionInfo.getActualQueryDimensions();
+    queryMeasures = executionInfo.getActualQueryMeasures();
     measureDefaultValues = new Object[queryMeasures.length];
     allColumnInfo = new ColumnVectorInfo[queryDimensions.length + queryMeasures.length];
     createVectorForNewlyAddedDimensions();
@@ -66,7 +66,7 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
           columnVectorInfo.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
               .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
         }
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       }
     }
   }
@@ -79,7 +79,7 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
       if (!measureInfo.getMeasureExists()[i]) {
         // add a dummy column vector result collector object
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
-        allColumnInfo[queryMeasures[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryMeasures[i].getOrdinal()] = columnVectorInfo;
         columnVectorInfo.measure = queryMeasures[i];
         measureDefaultValues[i] = getMeasureDefaultValue(queryMeasures[i].getMeasure());
       }
@@ -98,11 +98,13 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
 
 
 
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
-    throw new UnsupportedOperationException("collectData is not supported here");
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
+    throw new UnsupportedOperationException("collectResultInRow is not supported here");
   }
 
-  @Override public void collectVectorBatch(AbstractScannedResult scannedResult,
+  @Override
+  public void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch) {
     int numberOfPages = scannedResult.numberOfpages();
     while (scannedResult.getCurrentPageCounter() < numberOfPages) {
@@ -125,7 +127,8 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
       fillDataForNonExistingDimensions();
       fillDataForNonExistingMeasures();
       // fill existing dimensions and measures data
-      scanAndFillResult(scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
+      fillResultToColumnarBatch(
+          scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
       columnarBatch.setActualSize(columnarBatch.getActualSize() + requiredRows - filteredRows);
     }
   }
@@ -134,11 +137,11 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
    * This method will fill the default values of non existing dimensions in the current block
    */
   private void fillDataForNonExistingDimensions() {
-    for (int i = 0; i < tableBlockExecutionInfos.getActualQueryDimensions().length; i++) {
+    for (int i = 0; i < executionInfo.getActualQueryDimensions().length; i++) {
       if (!dimensionInfo.getDimensionExists()[i]) {
-        int queryOrder = tableBlockExecutionInfos.getActualQueryDimensions()[i].getQueryOrder();
+        int queryOrder = executionInfo.getActualQueryDimensions()[i].getOrdinal();
         CarbonDimension dimension =
-            tableBlockExecutionInfos.getActualQueryDimensions()[i].getDimension();
+            executionInfo.getActualQueryDimensions()[i].getDimension();
         if (dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
           // fill direct dictionary column data
           fillDirectDictionaryData(allColumnInfo[queryOrder].vector, allColumnInfo[queryOrder],
@@ -216,10 +219,10 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
    * This method will fill the default values of non existing measures in the current block
    */
   private void fillDataForNonExistingMeasures() {
-    for (int i = 0; i < tableBlockExecutionInfos.getActualQueryMeasures().length; i++) {
+    for (int i = 0; i < executionInfo.getActualQueryMeasures().length; i++) {
       if (!measureInfo.getMeasureExists()[i]) {
-        int queryOrder = tableBlockExecutionInfos.getActualQueryMeasures()[i].getQueryOrder();
-        CarbonMeasure measure = tableBlockExecutionInfos.getActualQueryMeasures()[i].getMeasure();
+        int queryOrder = executionInfo.getActualQueryMeasures()[i].getOrdinal();
+        CarbonMeasure measure = executionInfo.getActualQueryMeasures()[i].getMeasure();
         ColumnVectorInfo columnVectorInfo = allColumnInfo[queryOrder];
         CarbonColumnVector vector = columnVectorInfo.vector;
         Object defaultValue = measureDefaultValues[i];

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
index 1fcccca..30dd1dd 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 
 import org.apache.spark.sql.catalyst.util.GenericArrayData;
 import org.apache.spark.sql.types.ArrayType;
@@ -86,7 +86,7 @@ public class ArrayQueryType extends ComplexQueryType implements GenericQueryType
     return new ArrayType(null, true);
   }
 
-  @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder)
+  @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
       throws IOException {
     readBlockDataChunk(blockChunkHolder);
     children.fillRequiredBlockData(blockChunkHolder);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
index ee43a10..98f0715 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.scan.complextypes;
 import java.io.IOException;
 
 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;
 
 public class ComplexQueryType {
   protected String name;
@@ -42,7 +42,7 @@ public class ComplexQueryType {
   protected byte[] copyBlockDataChunk(DimensionRawColumnChunk[] rawColumnChunks,
       int rowNumber, int pageNumber) {
     byte[] data =
-        rawColumnChunks[blockIndex].convertToDimColDataChunk(pageNumber).getChunkData(rowNumber);
+        rawColumnChunks[blockIndex].decodeColumnPage(pageNumber).getChunkData(rowNumber);
     byte[] output = new byte[data.length];
     System.arraycopy(data, 0, output, 0, output.length);
     return output;
@@ -51,10 +51,10 @@ public class ComplexQueryType {
   /*
    * This method will read the block data chunk from the respective block
    */
-  protected void readBlockDataChunk(BlocksChunkHolder blockChunkHolder) throws IOException {
-    if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-      blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+  protected void readBlockDataChunk(RawBlockletColumnChunks blockChunkHolder) throws IOException {
+    if (null == blockChunkHolder.getDimensionRawColumnChunks()[blockIndex]) {
+      blockChunkHolder.getDimensionRawColumnChunks()[blockIndex] = blockChunkHolder.getDataBlock()
+          .readDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
index 978a652..b8aa912 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionary
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.keygenerator.mdkey.Bits;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 import org.apache.spark.sql.types.BooleanType$;
@@ -113,7 +113,7 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
     }
   }
 
-  @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder)
+  @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
       throws IOException {
     readBlockDataChunk(blockChunkHolder);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
index 23a9f81..1d4f141 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
@@ -25,7 +25,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 
 import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 import org.apache.spark.sql.types.DataType;
@@ -106,7 +106,7 @@ public class StructQueryType extends ComplexQueryType implements GenericQueryTyp
     return new StructType(fields);
   }
 
-  @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder)
+  @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
       throws IOException {
     readBlockDataChunk(blockChunkHolder);
 


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

Posted by ja...@apache.org.
[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/15b4e192
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/15b4e192
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/15b4e192

Branch: refs/heads/carbonstore
Commit: 15b4e192ee904a2e7c845ac67e0fcf1ba151a683
Parents: 71c2d8c
Author: Jacky Li <ja...@qq.com>
Authored: Tue Jan 30 21:24:04 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Feb 1 10:15:05 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      | 184 +++++
 .../BlockletDataRefNodeWrapper.java             | 197 ------
 .../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      |  48 +-
 .../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 +++++++
 .../StandardPartitionTableQueryTestCase.scala   |   7 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   4 +-
 .../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 +-
 212 files changed, 5804 insertions(+), 7359 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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;
+  }
+}


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/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 c33d5ac..1e73dbb 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;
@@ -122,7 +121,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
@@ -169,12 +167,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
@@ -199,9 +197,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());
@@ -253,8 +253,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(),
@@ -278,20 +278,19 @@ 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)
       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,
+            queryModel.getProjectionDimensions(), tableBlockDimensions,
             segmentProperties.getComplexDimensions());
     int tableFactPathLength = CarbonStorePath
         .getCarbonTablePath(queryModel.getAbsoluteTableIdentifier().getTablePath(),
@@ -301,27 +300,26 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     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;
@@ -355,45 +353,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(
@@ -405,27 +404,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
@@ -433,15 +432,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;
   }
@@ -454,7 +453,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
@@ -465,12 +464,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++;
       }
@@ -492,26 +490,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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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 288aea7..4874f1c 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,11 +61,11 @@ 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) {
-    List<QueryDimension> presentDimension =
-        new ArrayList<QueryDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    List<ProjectionDimension> presentDimension =
+        new ArrayList<ProjectionDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     boolean[] isDimensionExists = new boolean[queryDimensions.size()];
     Object[] defaultValues = new Object[queryDimensions.size()];
     // create dimension information instance
@@ -74,22 +74,21 @@ 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;
       } 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;
             break;
@@ -102,11 +101,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;
             break;
@@ -242,39 +240,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
@@ -361,30 +326,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/15b4e192/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/15b4e192/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/15b4e192/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/15b4e192/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 5a1b7df..72ca1a4 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
@@ -104,8 +104,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
@@ -400,7 +399,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
       case FALSE:
         return new RowLevelFilterResolverImpl(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
@@ -409,7 +408,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());
           }
         }
@@ -435,7 +433,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
                   tableIdentifier);
             }
             return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-                tableIdentifier,
                 currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
           }
           // getting new dim index.
@@ -461,14 +458,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();
@@ -491,8 +486,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()
@@ -515,11 +509,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;
 
@@ -533,7 +525,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/15b4e192/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/15b4e192/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 3268ca3..b8ffe0a 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;
@@ -168,10 +168,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:
@@ -545,10 +543,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);
@@ -560,22 +557,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());
     }
@@ -587,17 +582,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;
@@ -613,8 +601,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);
@@ -724,14 +711,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);
     }
@@ -755,21 +740,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) {
@@ -815,8 +795,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)) {
@@ -1769,14 +1747,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/15b4e192/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/15b4e192/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;


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
index 7f735c2..fd92fc1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -35,11 +35,10 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -72,7 +71,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       comparator = Comparator.getComparatorByDataTypeForMeasure(measure.getDataType());
     }
     ifDefaultValueMatchesFilter();
-    if (isDimensionPresentInCurrentBlock[0] == true) {
+    if (isDimensionPresentInCurrentBlock[0]) {
       isNaturalSorted = dimColEvaluatorInfoList.get(0).getDimension().isUseInvertedIndex()
           && dimColEvaluatorInfoList.get(0).getDimension().isSortColumn();
     }
@@ -119,11 +118,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     boolean isScanRequired = false;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        minValue = blockMinValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        minValue = blockMinValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        minValue = blockMinValue[dimensionBlocksIndex[0]];
+        minValue = blockMinValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(minValue, filterRangeValues);
       }
     } else {
@@ -169,67 +168,69 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) {
-            BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                 rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
-    } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+    } else {
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.msrFilterRangeValues,
               msrColEvalutorInfoList.get(0).getType())) {
             BitSet bitSet =
-                getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+                getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                     rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     }
-    return null;
   }
 
   private BitSet getFilteredIndexesForMeasures(ColumnPage columnPage,
@@ -262,7 +263,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     return bitSet;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     byte[] defaultValue = null;
     if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
@@ -271,7 +272,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
               dimColEvaluatorInfoList.get(0).getDimension().getDataType());
       int key = directDictionaryGenerator.generateDirectSurrogateKey(null);
       CarbonDimension currentBlockDimension =
-          segmentProperties.getDimensions().get(dimensionBlocksIndex[0]);
+          segmentProperties.getDimensions().get(dimensionChunkIndex[0]);
       if (currentBlockDimension.isSortColumn()) {
         defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
             this.segmentProperties.getSortColumnsGenerator());
@@ -282,14 +283,14 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows,
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows,
           defaultValue);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -301,12 +302,12 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows,
+      DimensionColumnPage dimensionColumnPage, int numerOfRows,
       byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
@@ -318,7 +319,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     //find the number of default values to skip the null value in case of direct dictionary
     if (null != defaultValue) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               defaultValue, true);
       if (start < 0) {
         skip = -(start + 1);
@@ -336,7 +337,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
 
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], false);
       if (start >= 0) {
         // Logic will handle the case where the range filter member is not present in block
@@ -344,7 +345,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
         // set inorder to apply filters. this is Lesser than filter so the range will be taken
         // from the prev element which is Lesser than filter member.
         start =
-            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[i]);
+            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[i]);
       }
       if (start < 0) {
         start = -(start + 1);
@@ -355,14 +356,14 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
         // will be pointing to the next consecutive position. So compare it again and point to the
         // previous value returned from getFirstIndexUsingBinarySearch.
         if (ByteUtil.compare(filterValues[i],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             < 0) {
           start = start - 1;
         }
       }
       last = start;
       for (int j = start; j >= skip; j--) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last--;
       }
       startIndex = last;
@@ -379,11 +380,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
    * be called if the column is sorted default so column index
    * mapping will be present for accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows, byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -396,7 +397,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       //find the number of default values to skip the null value in case of direct dictionary
       if (null != defaultValue) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, defaultValue, true);
         if (start < 0) {
           skip = -(start + 1);
@@ -413,11 +414,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       }
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], false);
         if (start >= 0) {
           start =
-              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[k]);
+              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[k]);
         }
         if (start < 0) {
           start = -(start + 1);
@@ -428,7 +429,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
           // When negative value of start is returned from getFirstIndexUsingBinarySearch the Start
           // will be pointing to the next consecutive position. So compare it again and point to the
           // previous value returned from getFirstIndexUsingBinarySearch.
-          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) < 0) {
             start = start - 1;
           }
         }
@@ -445,7 +446,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) < 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) < 0) {
             bitSet.set(i);
           }
         }
@@ -454,21 +455,24 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
index 92396ae..c5bea69 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
@@ -22,7 +22,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 TrueFilterExecutor implements FilterExecuter {
@@ -33,19 +33,20 @@ public class TrueFilterExecutor implements FilterExecuter {
    * @return
    * @throws FilterUnsupportedException
    */
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    int numberOfPages = blockChunkHolder.getDataBlock().numberOfPages();
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    int numberOfPages = rawBlockletColumnChunks.getDataBlock().numberOfPages();
     BitSetGroup group = new BitSetGroup(numberOfPages);
     for (int i = 0; i < numberOfPages; i++) {
       BitSet set = new BitSet();
-      set.flip(0, blockChunkHolder.getDataBlock().nodeSize());
+      set.flip(0, rawBlockletColumnChunks.getDataBlock().numRows());
       group.setBitSet(set, i);
     }
     return group;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
     return true;
   }
 
@@ -66,9 +67,9 @@ public class TrueFilterExecutor implements FilterExecuter {
   /**
    * It just reads necessary block for filter executor, it does not uncompress the data.
    *
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    */
-  public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) {
     // do nothing
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
index 516447f..bc3b39c 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
@@ -21,27 +21,29 @@ 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 ValueBasedFilterExecuterImpl implements FilterExecuter {
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
     return new BitSetGroup(0);
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException {
     throw new FilterUnsupportedException("Unsupported ValueBasedFilterExecuterImpl on row");
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     return new BitSet(1);
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/FilterOptimizerBasic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/FilterOptimizerBasic.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/FilterOptimizerBasic.java
deleted file mode 100644
index b3ed34e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/FilterOptimizerBasic.java
+++ /dev/null
@@ -1,26 +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.scan.filter.intf;
-
-import org.apache.carbondata.core.scan.expression.Expression;
-
-public class FilterOptimizerBasic implements FilterOptimizer {
-  @Override public Expression optimizeFilter() {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
index e5cb0ad..bbd33ec 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
@@ -19,14 +19,13 @@ package org.apache.carbondata.core.scan.filter.optimizer;
 
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.RangeExpressionEvaluator;
-import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer;
 import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerImpl;
 
 public class RangeFilterOptmizer extends FilterOptimizerImpl {
 
   RangeExpressionEvaluator rangeExpEvaluator;
 
-  public RangeFilterOptmizer(FilterOptimizer filterOptimizer, Expression filterExpression) {
+  public RangeFilterOptmizer(Expression filterExpression) {
     super(filterExpression);
     this.rangeExpEvaluator = new RangeExpressionEvaluator(filterExpression);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
index bb66d18..996965d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@ -46,17 +46,13 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
   protected boolean isIncludeFilter;
   private DimColumnResolvedFilterInfo dimColResolvedFilterInfo;
   private MeasureColumnResolvedFilterInfo msrColResolvedFilterInfo;
-  private AbsoluteTableIdentifier tableIdentifier;
-  private boolean isMeasure;
 
   public ConditionalFilterResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier, boolean isMeasure) {
+      boolean isIncludeFilter, boolean isMeasure) {
     this.exp = exp;
     this.isExpressionResolve = isExpressionResolve;
     this.isIncludeFilter = isIncludeFilter;
-    this.tableIdentifier = tableIdentifier;
-    this.isMeasure = isMeasure;
-    if (isMeasure == false) {
+    if (!isMeasure) {
       this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo();
     } else {
       this.msrColResolvedFilterInfo = new MeasureColumnResolvedFilterInfo();
@@ -230,10 +226,6 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
     return msrColResolvedFilterInfo;
   }
 
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
   /**
    * method will calculates the start key based on the filter surrogates
    */
@@ -245,12 +237,6 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
       FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
           setOfStartKeyByteArray);
     }
-// else {
-//      FilterUtil.getStartKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
-//          segmentProperties, startKey, startKeyList);
-//      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-//          setOfStartKeyByteArray);
-//    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
index 33bafb1..c846c02 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
@@ -40,7 +40,7 @@ public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl {
 
   public RowLevelFilterResolverImpl(Expression exp, boolean isExpressionResolve,
       boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
-    super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false);
+    super(exp, isExpressionResolve, isIncludeFilter, false);
     dimColEvaluatorInfoList =
         new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
index 0e7666c..e85e9d1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@ -61,7 +61,7 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
 
   public RowLevelRangeFilterResolverImpl(Expression exp, boolean isExpressionResolve,
       boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
-    super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false);
+    super(exp, isExpressionResolve, isIncludeFilter, false);
     dimColEvaluatorInfoList =
         new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
index f8586ed..4165424 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
@@ -77,10 +77,6 @@ public class MeasureColumnResolvedFilterInfo extends ColumnResolvedFilterInfo
     }
   }
 
-  public Map<CarbonMeasure, List<ColumnFilterInfo>> getMeasureResolvedFilterInstance() {
-    return measureResolvedFilter;
-  }
-
   public ColumnFilterInfo getFilterValues() {
     return resolvedFilterValueObj;
   }
@@ -109,10 +105,6 @@ public class MeasureColumnResolvedFilterInfo extends ColumnResolvedFilterInfo
     this.type = dataType;
   }
 
-  public boolean isMeasureExistsInCurrentSlice() {
-    return true;
-  }
-
   public CarbonColumn getCarbonColumn() {
     return carbonColumn;
   }
@@ -125,10 +117,6 @@ public class MeasureColumnResolvedFilterInfo extends ColumnResolvedFilterInfo
     return carbonMeasure;
   }
 
-  public boolean isMeasureExistsInCurrentSilce() {
-    return isMeasureExistsInCurrentSilce;
-  }
-
   public void setMeasureExistsInCurrentSilce(boolean measureExistsInCurrentSilce) {
     isMeasureExistsInCurrentSilce = measureExistsInCurrentSilce;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
index d403ef4..104f69e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
@@ -28,9 +28,9 @@ import org.apache.carbondata.core.scan.filter.resolver.ConditionalFilterResolver
 public class TrueConditionalResolverImpl extends ConditionalFilterResolverImpl {
 
   public TrueConditionalResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
+      boolean isIncludeFilter) {
 
-    super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false);
+    super(exp, isExpressionResolve, isIncludeFilter, false);
   }
 
   @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier,
@@ -47,16 +47,4 @@ public class TrueConditionalResolverImpl extends ConditionalFilterResolverImpl {
     return FilterExecuterType.TRUE;
   }
 
-  /**
-   * Method will the read filter expression corresponding to the resolver.
-   * This method is required in row level executer inorder to evaluate the filter
-   * expression against spark, as mentioned above row level is a special type
-   * filter resolver.
-   *
-   * @return Expression
-   */
-  public Expression getFilterExpresion() {
-    return exp;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
index c90816a..b1e5a7d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
@@ -19,7 +19,6 @@ package org.apache.carbondata.core.scan.filter.resolver.resolverinfo.visitor;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
@@ -47,7 +46,6 @@ public class RangeDictionaryColumnVisitor extends DictionaryColumnVisitor
     if (visitableObj instanceof DimColumnResolvedFilterInfo) {
       DimColumnResolvedFilterInfo resolveDimension = (DimColumnResolvedFilterInfo) visitableObj;
       ColumnFilterInfo resolvedFilterObject = null;
-      List<String> evaluateResultListFinal;
       resolvedFilterObject = FilterUtil
           .getFilterListForAllValues(metadata.getTableIdentifier(), metadata.getExpression(),
               metadata.getColumnExpression(), metadata.isIncludeFilter(),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java b/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java
deleted file mode 100644
index 5a5b042..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java
+++ /dev/null
@@ -1,151 +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.scan.model;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.scan.expression.Expression;
-
-/**
- * This class contains all the logical information about the query like dimensions,measures,
- * sort order, topN etc..
- */
-public class CarbonQueryPlan implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = -9036044826928017164L;
-
-  /**
-   * Database name
-   */
-  private String databaseName;
-
-  /**
-   * Table name
-   */
-  private String tableName;
-
-  /**
-   * List of dimensions.
-   * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name
-   * and department_name are dimensions
-   * If there is no dimensions asked in query then it would be remained as empty.
-   */
-  private List<QueryDimension> dimensions =
-      new ArrayList<QueryDimension>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  /**
-   * List of measures.
-   * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary)
-   * would be measure.
-   * If there is no dimensions asked in query then it would be remained as empty.
-   */
-  private List<QueryMeasure> measures =
-      new ArrayList<QueryMeasure>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  /**
-   * expression
-   */
-  private Expression expression;
-
-  /**
-   * queryId
-   */
-  private String queryId;
-
-  /**
-   * If it is raw detail query, no need to aggregate in backend. And it returns with dictionary data
-   * with out decoding.
-   */
-  private boolean rawDetailQuery;
-
-  /**
-   * Constructor created with database name and table name.
-   *
-   * @param databaseName
-   * @param tableName
-   */
-  public CarbonQueryPlan(String databaseName, String tableName) {
-    this.tableName = tableName;
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * @return the dimensions
-   */
-  public List<QueryDimension> getDimensions() {
-    return dimensions;
-  }
-
-  public void addDimension(QueryDimension dimension) {
-    this.dimensions.add(dimension);
-  }
-
-  /**
-   * @return the measures
-   */
-  public List<QueryMeasure> getMeasures() {
-    return measures;
-  }
-
-  public void addMeasure(QueryMeasure measure) {
-    this.measures.add(measure);
-  }
-
-  public Expression getFilterExpression() {
-    return expression;
-  }
-
-  public void setFilterExpression(Expression expression) {
-    this.expression = expression;
-  }
-
-  /**
-   * @return the databaseName
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * @return the tableName
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  public String getQueryId() {
-    return queryId;
-  }
-
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  public boolean isRawDetailQuery() {
-    return rawDetailQuery;
-  }
-
-  public void setRawDetailQuery(boolean rawDetailQuery) {
-    this.rawDetailQuery = rawDetailQuery;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java
new file mode 100644
index 0000000..0768221
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java
@@ -0,0 +1,59 @@
+/*
+ * 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.scan.model;
+
+/**
+ * Contains information for a column for projection
+ */
+public class ProjectionColumn {
+
+  /**
+   * name of the column
+   */
+  protected String columnName;
+
+  /**
+   * query order in which result of the query will be send
+   */
+  private int projectionOrdinal;
+
+  ProjectionColumn(String columnName) {
+    this.columnName = columnName;
+  }
+
+  /**
+   * @return the columnName
+   */
+  public String getColumnName() {
+    return columnName;
+  }
+
+  /**
+   * @return the projectionOrdinal
+   */
+  public int getOrdinal() {
+    return projectionOrdinal;
+  }
+
+  /**
+   * @param projectionOrdinal the projectionOrdinal to set
+   */
+  public void setOrdinal(int projectionOrdinal) {
+    this.projectionOrdinal = projectionOrdinal;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java
new file mode 100644
index 0000000..806de85
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java
@@ -0,0 +1,45 @@
+/*
+ * 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.scan.model;
+
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+
+/**
+ * dimension column which holds the information about the column
+ * this is done to avoid heavy object serialization
+ */
+public class ProjectionDimension extends ProjectionColumn {
+
+  /**
+   * actual dimension column
+   */
+  private CarbonDimension dimension;
+
+  public ProjectionDimension(CarbonDimension dimension) {
+    super(dimension.getColName());
+    this.dimension = dimension;
+  }
+
+  /**
+   * @return the dimension
+   */
+  public CarbonDimension getDimension() {
+    return dimension;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionMeasure.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionMeasure.java
new file mode 100644
index 0000000..93d9c82
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionMeasure.java
@@ -0,0 +1,46 @@
+/*
+ * 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.scan.model;
+
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+
+/**
+ * query plan measure, this class will holds the information
+ * about measure present in the query, this is done to avoid the serialization
+ * of the heavy object
+ */
+public class ProjectionMeasure extends ProjectionColumn {
+
+  /**
+   * actual carbon measure object
+   */
+  private CarbonMeasure measure;
+
+  public ProjectionMeasure(CarbonMeasure measure) {
+    super(measure.getColName());
+    this.measure = measure;
+  }
+
+  /**
+   * @return the measure
+   */
+  public CarbonMeasure getMeasure() {
+    return measure;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java
deleted file mode 100644
index 1336b87..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java
+++ /dev/null
@@ -1,66 +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.scan.model;
-
-import java.io.Serializable;
-
-/**
- * query column  which will have information about column
- */
-public class QueryColumn implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -4222306600480181084L;
-
-  /**
-   * name of the column
-   */
-  protected String columnName;
-
-  /**
-   * query order in which result of the query will be send
-   */
-  private int queryOrder;
-
-  public QueryColumn(String columnName) {
-    this.columnName = columnName;
-  }
-
-  /**
-   * @return the columnName
-   */
-  public String getColumnName() {
-    return columnName;
-  }
-
-  /**
-   * @return the queryOrder
-   */
-  public int getQueryOrder() {
-    return queryOrder;
-  }
-
-  /**
-   * @param queryOrder the queryOrder to set
-   */
-  public void setQueryOrder(int queryOrder) {
-    this.queryOrder = queryOrder;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/QueryDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryDimension.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryDimension.java
deleted file mode 100644
index e8f07a7..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryDimension.java
+++ /dev/null
@@ -1,56 +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.scan.model;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-
-/**
- * query plan dimension which will holds the information about the query plan dimension
- * this is done to avoid heavy object serialization
- */
-public class QueryDimension extends QueryColumn implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8492704093776645651L;
-  /**
-   * actual dimension column
-   */
-  private transient CarbonDimension dimension;
-
-  public QueryDimension(String columnName) {
-    super(columnName);
-  }
-
-  /**
-   * @return the dimension
-   */
-  public CarbonDimension getDimension() {
-    return dimension;
-  }
-
-  /**
-   * @param dimension the dimension to set
-   */
-  public void setDimension(CarbonDimension dimension) {
-    this.dimension = dimension;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/QueryMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryMeasure.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryMeasure.java
deleted file mode 100644
index 73a8ad5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryMeasure.java
+++ /dev/null
@@ -1,59 +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.scan.model;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-
-/**
- * query plan measure, this class will holds the information
- * about measure present in the query, this is done to avoid the serialization
- * of the heavy object
- */
-public class QueryMeasure extends QueryColumn implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 1035512411375495414L;
-
-  /**
-   * actual carbon measure object
-   */
-  private transient CarbonMeasure measure;
-
-  public QueryMeasure(String columnName) {
-    super(columnName);
-  }
-
-  /**
-   * @return the measure
-   */
-  public CarbonMeasure getMeasure() {
-    return measure;
-  }
-
-  /**
-   * @param measure the measure to set
-   */
-  public void setMeasure(CarbonMeasure measure) {
-    this.measure = measure;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
index 655ed5e..4e3665f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.core.scan.model;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -46,25 +45,17 @@ import org.apache.carbondata.core.util.DataTypeConverter;
  * about the query, This will be sent from driver to executor '
  * This will be refereed to executing the query.
  */
-public class QueryModel implements Serializable {
+public class QueryModel {
 
   /**
-   * serialization version
-   */
-  private static final long serialVersionUID = -4674677234007089052L;
-  /**
    * this will hold the information about the dictionary dimension
    * which to
    */
-  public transient Map<String, Dictionary> columnToDictionaryMapping;
-  /**
-   * list of dimension selected for in query
-   */
-  private List<QueryDimension> queryDimension;
+  private transient Map<String, Dictionary> columnToDictionaryMapping;
   /**
-   * list of measure selected in query
+   * list of projection columns in query
    */
-  private List<QueryMeasure> queryMeasures;
+  private QueryProjection projection;
   /**
    * query id
    */
@@ -79,10 +70,6 @@ public class QueryModel implements Serializable {
    */
   private List<TableBlockInfo> tableBlockInfos;
   /**
-   * absolute table identifier
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-  /**
    * To handle most of the computation in query engines like spark and hive, carbon should give
    * raw detailed records to it.
    */
@@ -117,43 +104,15 @@ public class QueryModel implements Serializable {
    */
   private boolean readPageByPage;
 
-  public QueryModel() {
+  private QueryModel(CarbonTable carbonTable) {
     tableBlockInfos = new ArrayList<TableBlockInfo>();
-    queryDimension = new ArrayList<QueryDimension>();
-    queryMeasures = new ArrayList<QueryMeasure>();
     invalidSegmentIds = new ArrayList<>();
+    this.table = carbonTable;
+    this.queryId = String.valueOf(System.nanoTime());
   }
 
-  public static QueryModel createModel(AbsoluteTableIdentifier absoluteTableIdentifier,
-      CarbonQueryPlan queryPlan, CarbonTable carbonTable, DataTypeConverter converter) {
-    QueryModel queryModel = new QueryModel();
-    queryModel.setAbsoluteTableIdentifier(absoluteTableIdentifier);
-
-    fillQueryModel(queryPlan, carbonTable, queryModel);
-
-    queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery());
-    queryModel.setQueryId(queryPlan.getQueryId());
-    queryModel.setConverter(converter);
-    return queryModel;
-  }
-
-  private static void fillQueryModel(CarbonQueryPlan queryPlan, CarbonTable carbonTable,
-      QueryModel queryModel) {
-    queryModel.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-    queryModel.setQueryDimension(queryPlan.getDimensions());
-    queryModel.setQueryMeasures(queryPlan.getMeasures());
-    if (null != queryPlan.getFilterExpression()) {
-      boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
-      boolean[] isFilterMeasures =
-          new boolean[carbonTable.getNumberOfMeasures(carbonTable.getTableName())];
-      processFilterExpression(carbonTable, queryPlan.getFilterExpression(), isFilterDimensions,
-          isFilterMeasures);
-      queryModel.setIsFilterDimensions(isFilterDimensions);
-      queryModel.setIsFilterMeasures(isFilterMeasures);
-    }
-    //TODO need to remove this code, and executor will load the table
-    // from file metadata
-    queryModel.setTable(carbonTable);
+  public static QueryModel newInstance(CarbonTable carbonTable) {
+    return new QueryModel(carbonTable);
   }
 
   public static void processFilterExpression(CarbonTable carbonTable, Expression filterExpression,
@@ -239,42 +198,26 @@ public class QueryModel implements Serializable {
    */
   public CarbonColumn[] getProjectionColumns() {
     CarbonColumn[] carbonColumns =
-        new CarbonColumn[getQueryDimension().size() + getQueryMeasures().size()];
-    for (QueryDimension dimension : getQueryDimension()) {
-      carbonColumns[dimension.getQueryOrder()] = dimension.getDimension();
+        new CarbonColumn[getProjectionDimensions().size() + getProjectionMeasures().size()];
+    for (ProjectionDimension dimension : getProjectionDimensions()) {
+      carbonColumns[dimension.getOrdinal()] = dimension.getDimension();
     }
-    for (QueryMeasure msr : getQueryMeasures()) {
-      carbonColumns[msr.getQueryOrder()] = msr.getMeasure();
+    for (ProjectionMeasure msr : getProjectionMeasures()) {
+      carbonColumns[msr.getOrdinal()] = msr.getMeasure();
     }
     return carbonColumns;
   }
 
-  /**
-   * @return the queryDimension
-   */
-  public List<QueryDimension> getQueryDimension() {
-    return queryDimension;
+  public void setProjection(QueryProjection projection) {
+    this.projection = projection;
   }
 
-  /**
-   * @param queryDimension the queryDimension to set
-   */
-  public void setQueryDimension(List<QueryDimension> queryDimension) {
-    this.queryDimension = queryDimension;
+  public List<ProjectionDimension> getProjectionDimensions() {
+    return projection.getDimensions();
   }
 
-  /**
-   * @return the queryMeasures
-   */
-  public List<QueryMeasure> getQueryMeasures() {
-    return queryMeasures;
-  }
-
-  /**
-   * @param queryMeasures the queryMeasures to set
-   */
-  public void setQueryMeasures(List<QueryMeasure> queryMeasures) {
-    this.queryMeasures = queryMeasures;
+  public List<ProjectionMeasure> getProjectionMeasures() {
+    return projection.getMeasures();
   }
 
   /**
@@ -285,13 +228,6 @@ public class QueryModel implements Serializable {
   }
 
   /**
-   * @param queryId the queryId to set
-   */
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  /**
    * @return the tableBlockInfos
    */
   public List<TableBlockInfo> getTableBlockInfos() {
@@ -320,14 +256,7 @@ public class QueryModel implements Serializable {
    * @return the absoluteTableIdentifier
    */
   public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  /**
-   * @param absoluteTableIdentifier the absoluteTableIdentifier to set
-   */
-  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
+    return table.getAbsoluteTableIdentifier();
   }
 
   /**
@@ -337,13 +266,6 @@ public class QueryModel implements Serializable {
     return table;
   }
 
-  /**
-   * @param table the table to set
-   */
-  public void setTable(CarbonTable table) {
-    this.table = table;
-  }
-
   public boolean isForcedDetailRawQuery() {
     return forcedDetailRawQuery;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java
new file mode 100644
index 0000000..d4496e0
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java
@@ -0,0 +1,83 @@
+/*
+ * 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.scan.model;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+
+/**
+ * Contains projection columns in the query
+ */
+public class QueryProjection {
+
+  /**
+   * List of dimensions.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name
+   * and department_name are dimensions
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<ProjectionDimension> dimensions =
+      new ArrayList<ProjectionDimension>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * List of measures.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary)
+   * would be measure.
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<ProjectionMeasure> measures =
+      new ArrayList<ProjectionMeasure>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * Constructor created with database name and table name.
+   *
+   */
+  public QueryProjection() {
+  }
+
+  /**
+   * @return the dimensions
+   */
+  public List<ProjectionDimension> getDimensions() {
+    return dimensions;
+  }
+
+  public void addDimension(CarbonDimension dimension, int queryOrdinal) {
+    ProjectionDimension queryDimension = new ProjectionDimension(dimension);
+    queryDimension.setOrdinal(queryOrdinal);
+    this.dimensions.add(queryDimension);
+  }
+
+  /**
+   * @return the measures
+   */
+  public List<ProjectionMeasure> getMeasures() {
+    return measures;
+  }
+
+  public void addMeasure(CarbonMeasure measure, int queryOrdinal) {
+    ProjectionMeasure queryMeasure = new ProjectionMeasure(measure);
+    queryMeasure.setOrdinal(queryOrdinal);
+    this.measures.add(queryMeasure);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java
deleted file mode 100644
index eb5e3f9..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java
+++ /dev/null
@@ -1,251 +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.scan.processor;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.scan.collector.ResultCollectorFactory;
-import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.scan.scanner.BlockletScanner;
-import org.apache.carbondata.core.scan.scanner.impl.FilterScanner;
-import org.apache.carbondata.core.scan.scanner.impl.NonFilterScanner;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-import org.apache.carbondata.core.util.TaskMetricsMap;
-
-/**
- * This abstract class provides a skeletal implementation of the
- * Block iterator.
- */
-public abstract class AbstractDataBlockIterator extends CarbonIterator<List<Object[]>> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName());
-
-  /**
-   * iterator which will be used to iterate over data blocks
-   */
-  protected CarbonIterator<DataRefNode> dataBlockIterator;
-
-  /**
-   * result collector which will be used to aggregate the scanned result
-   */
-  protected ScannedResultCollector scannerResultAggregator;
-
-  /**
-   * processor which will be used to process the block processing can be
-   * filter processing or non filter processing
-   */
-  protected BlockletScanner blockletScanner;
-
-  /**
-   * batch size of result
-   */
-  protected int batchSize;
-
-  protected ExecutorService executorService;
-
-  private Future<AbstractScannedResult> future;
-
-  private Future<BlocksChunkHolder> futureIo;
-
-  protected AbstractScannedResult scannedResult;
-
-  private BlockExecutionInfo blockExecutionInfo;
-
-  private FileHolder fileReader;
-
-  private AtomicBoolean nextBlock;
-
-  private AtomicBoolean nextRead;
-
-  public AbstractDataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader,
-      int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) {
-    this.blockExecutionInfo = blockExecutionInfo;
-    this.fileReader = fileReader;
-    dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
-        blockExecutionInfo.getNumberOfBlockToScan());
-    if (blockExecutionInfo.getFilterExecuterTree() != null) {
-      blockletScanner = new FilterScanner(blockExecutionInfo, queryStatisticsModel);
-    } else {
-      blockletScanner = new NonFilterScanner(blockExecutionInfo, queryStatisticsModel);
-    }
-    this.scannerResultAggregator =
-        ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo);
-    this.batchSize = batchSize;
-    this.executorService = executorService;
-    this.nextBlock = new AtomicBoolean(false);
-    this.nextRead = new AtomicBoolean(false);
-  }
-
-  public boolean hasNext() {
-    if (scannedResult != null && scannedResult.hasNext()) {
-      return true;
-    } else {
-      if (null != scannedResult) {
-        scannedResult.freeMemory();
-      }
-      return dataBlockIterator.hasNext() || nextBlock.get() || nextRead.get();
-    }
-  }
-
-  protected boolean updateScanner() {
-    try {
-      if (scannedResult != null && scannedResult.hasNext()) {
-        return true;
-      } else {
-        scannedResult = getNextScannedResult();
-        while (scannedResult != null) {
-          if (scannedResult.hasNext()) {
-            return true;
-          }
-          scannedResult = getNextScannedResult();
-        }
-        nextBlock.set(false);
-        nextRead.set(false);
-        return false;
-      }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-
-  private AbstractScannedResult getNextScannedResult() throws Exception {
-    AbstractScannedResult result = null;
-    if (blockExecutionInfo.isPrefetchBlocklet()) {
-      if (dataBlockIterator.hasNext() || nextBlock.get() || nextRead.get()) {
-        if (future == null) {
-          future = execute();
-        }
-        result = future.get();
-        nextBlock.set(false);
-        if (dataBlockIterator.hasNext() || nextRead.get()) {
-          nextBlock.set(true);
-          future = execute();
-        }
-      }
-    } else {
-      if (dataBlockIterator.hasNext()) {
-        BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolder();
-        if (blocksChunkHolder != null) {
-          result = blockletScanner.scanBlocklet(blocksChunkHolder);
-        }
-      }
-    }
-    return result;
-  }
-
-  private BlocksChunkHolder getBlocksChunkHolder() throws IOException {
-    BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolderInternal();
-    while (blocksChunkHolder == null && dataBlockIterator.hasNext()) {
-      blocksChunkHolder = getBlocksChunkHolderInternal();
-    }
-    return blocksChunkHolder;
-  }
-
-  private BlocksChunkHolder getBlocksChunkHolderInternal() throws IOException {
-    BlocksChunkHolder blocksChunkHolder =
-        new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(),
-            blockExecutionInfo.getTotalNumberOfMeasureBlock(), fileReader);
-    blocksChunkHolder.setDataBlock(dataBlockIterator.next());
-    if (blocksChunkHolder.getDataBlock().getColumnsMaxValue() == null) {
-      return blocksChunkHolder;
-    }
-    if (blockletScanner.isScanRequired(blocksChunkHolder)) {
-      return blocksChunkHolder;
-    }
-    return null;
-  }
-
-  private Future<AbstractScannedResult> execute() {
-    return executorService.submit(new Callable<AbstractScannedResult>() {
-      @Override public AbstractScannedResult call() throws Exception {
-        if (futureIo == null) {
-          futureIo = executeRead();
-        }
-        BlocksChunkHolder blocksChunkHolder = futureIo.get();
-        futureIo = null;
-        nextRead.set(false);
-        if (blocksChunkHolder != null) {
-          if (dataBlockIterator.hasNext()) {
-            nextRead.set(true);
-            futureIo = executeRead();
-          }
-          return blockletScanner.scanBlocklet(blocksChunkHolder);
-        }
-        return null;
-      }
-    });
-  }
-
-  private Future<BlocksChunkHolder> executeRead() {
-    return executorService.submit(new Callable<BlocksChunkHolder>() {
-      @Override public BlocksChunkHolder call() throws Exception {
-        try {
-          TaskMetricsMap.getInstance().registerThreadCallback();
-          if (dataBlockIterator.hasNext()) {
-            BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolder();
-            if (blocksChunkHolder != null) {
-              blockletScanner.readBlocklet(blocksChunkHolder);
-              return blocksChunkHolder;
-            }
-          }
-          return null;
-        } finally {
-          // update read bytes metrics for this thread
-          TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId());
-        }
-      }
-    });
-  }
-
-  public abstract void processNextBatch(CarbonColumnarBatch columnarBatch);
-
-  /**
-   * Close the resources
-   */
-  public void close() {
-    // free the current scanned result
-    if (null != scannedResult && !scannedResult.hasNext()) {
-      scannedResult.freeMemory();
-    }
-    // free any pre-fetched memory if present
-    if (null != future) {
-      try {
-        AbstractScannedResult abstractScannedResult = future.get();
-        if (abstractScannedResult != null) {
-          abstractScannedResult.freeMemory();
-        }
-      } catch (InterruptedException | ExecutionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
index 3ea4e1d..f0d81ff 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
@@ -23,7 +23,7 @@ import org.apache.carbondata.core.datastore.DataRefNode;
 /**
  * Below class will be used to iterate over data block
  */
-public class BlockletIterator extends CarbonIterator<DataRefNode> {
+class BlockletIterator extends CarbonIterator<DataRefNode> {
   /**
    * data store block
    */
@@ -47,17 +47,18 @@ public class BlockletIterator extends CarbonIterator<DataRefNode> {
    * Constructor
    *
    * @param datablock                 first data block
-   * @param totalNumberOfBlocksToScan total number of blocks to be scanned
+   * @param totalNumberOfBlockletToScan total number of blocklets to be scanned
    */
-  public BlockletIterator(DataRefNode datablock, long totalNumberOfBlocksToScan) {
+  BlockletIterator(DataRefNode datablock, long totalNumberOfBlockletToScan) {
     this.datablock = datablock;
-    this.totalNumberOfBlocksToScan = totalNumberOfBlocksToScan;
+    this.totalNumberOfBlocksToScan = totalNumberOfBlockletToScan;
   }
 
   /**
    * is all the blocks assigned to this iterator has been processed
    */
-  @Override public boolean hasNext() {
+  @Override
+  public boolean hasNext() {
     return hasNext;
   }
 
@@ -67,7 +68,8 @@ public class BlockletIterator extends CarbonIterator<DataRefNode> {
    * @return next data block
    *
    */
-  @Override public DataRefNode next() {
+  @Override
+  public DataRefNode next() {
     // get the current blocks
     DataRefNode datablockTemp = datablock;
     // store the next data block

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/scan/processor/BlocksChunkHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlocksChunkHolder.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlocksChunkHolder.java
deleted file mode 100644
index 60090d0..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlocksChunkHolder.java
+++ /dev/null
@@ -1,141 +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.scan.processor;
-
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.util.BitSetGroup;
-
-/**
- * Block chunk holder which will hold the dimension and
- * measure chunk
- */
-public class BlocksChunkHolder {
-
-  /**
-   * dimension column data chunk
-   */
-  private DimensionRawColumnChunk[] dimensionRawDataChunk;
-
-  /**
-   * measure column data chunk
-   */
-  private MeasureRawColumnChunk[] measureRawDataChunk;
-
-  /**
-   * file reader which will use to read the block from file
-   */
-  private FileHolder fileReader;
-
-  /**
-   * data block
-   */
-  private DataRefNode dataBlock;
-
-  private BitSetGroup bitSetGroup;
-
-  public BlocksChunkHolder(int numberOfDimensionBlock, int numberOfMeasureBlock) {
-    dimensionRawDataChunk = new DimensionRawColumnChunk[numberOfDimensionBlock];
-    measureRawDataChunk = new MeasureRawColumnChunk[numberOfMeasureBlock];
-  }
-
-  public BlocksChunkHolder(int numberOfDimensionBlock, int numberOfMeasureBlock,
-      FileHolder fileReader) {
-    dimensionRawDataChunk = new DimensionRawColumnChunk[numberOfDimensionBlock];
-    measureRawDataChunk = new MeasureRawColumnChunk[numberOfMeasureBlock];
-    this.fileReader = fileReader;
-  }
-
-  /**
-   * @return the dimensionRawDataChunk
-   */
-  public DimensionRawColumnChunk[] getDimensionRawDataChunk() {
-    return dimensionRawDataChunk;
-  }
-
-  /**
-   * @param dimensionRawDataChunk the dimensionRawDataChunk to set
-   */
-  public void setDimensionRawDataChunk(DimensionRawColumnChunk[] dimensionRawDataChunk) {
-    this.dimensionRawDataChunk = dimensionRawDataChunk;
-  }
-
-  /**
-   * @return the measureRawDataChunk
-   */
-  public MeasureRawColumnChunk[] getMeasureRawDataChunk() {
-    return measureRawDataChunk;
-  }
-
-  /**
-   * @param measureRawDataChunk the measureRawDataChunk to set
-   */
-  public void setMeasureRawDataChunk(MeasureRawColumnChunk[] measureRawDataChunk) {
-    this.measureRawDataChunk = measureRawDataChunk;
-  }
-
-  /**
-   * @return the fileReader
-   */
-  public FileHolder getFileReader() {
-    return fileReader;
-  }
-
-  /**
-   * @param fileReader the fileReader to set
-   */
-  public void setFileReader(FileHolder fileReader) {
-    this.fileReader = fileReader;
-  }
-
-  /**
-   * @return the dataBlock
-   */
-  public DataRefNode getDataBlock() {
-    return dataBlock;
-  }
-
-  /**
-   * @param dataBlock the dataBlock to set
-   */
-  public void setDataBlock(DataRefNode dataBlock) {
-    this.dataBlock = dataBlock;
-  }
-
-  /***
-   * To reset the measure chunk and dimension chunk
-   * array
-   */
-  public void reset() {
-    for (int i = 0; i < measureRawDataChunk.length; i++) {
-      this.measureRawDataChunk[i] = null;
-    }
-    for (int i = 0; i < dimensionRawDataChunk.length; i++) {
-      this.dimensionRawDataChunk[i] = null;
-    }
-  }
-
-  public BitSetGroup getBitSetGroup() {
-    return bitSetGroup;
-  }
-
-  public void setBitSetGroup(BitSetGroup bitSetGroup) {
-    this.bitSetGroup = bitSetGroup;
-  }
-}


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index fdb5310..b40a8e2 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -30,7 +30,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
 import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -44,7 +44,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -266,8 +266,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValue() {
     byte[] dataChunks = { 5, 6, 7, 8, 9 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, compareValues);
     assertEquals(result, 1);
   }
@@ -275,8 +275,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValueToTarget() {
     byte[] dataChunks = { 7, 7, 7, 8, 9 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, compareValues);
     assertEquals(result, -1);
   }
@@ -284,8 +284,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValue() {
     byte[] dataChunks = { 5, 6, 7, 8, 9 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, compareValues, 5);
     assertEquals(result, 3);
   }
@@ -301,8 +301,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValueToTarget() {
     byte[] dataChunks = { 5, 6, 7, 7, 7 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, compareValues, 5);
     assertEquals(result, 5);
   }
@@ -524,23 +524,23 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetDictionaryEncodingArray() {
-    QueryDimension column1 = new QueryDimension("Column1");
-    QueryDimension column2 = new QueryDimension("Column2");
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
     column1Schema.setColumnName("Column1");
     List<Encoding> encoding = new ArrayList<>();
     encoding.add(Encoding.DICTIONARY);
     column1Schema.setEncodingList(encoding);
-    column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
     List<Encoding> encoding2 = new ArrayList<>();
     encoding2.add(Encoding.DELTA);
     column2Schema.setEncodingList(encoding2);
-    column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
-    QueryDimension[] queryDimensions = { column1, column2 };
+    ProjectionDimension[] queryDimensions = { column1, column2 };
 
     boolean[] dictionaryEncoding = CarbonUtil.getDictionaryEncodingArray(queryDimensions);
     boolean[] expectedDictionaryEncoding = { true, false };
@@ -550,23 +550,23 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetDirectDictionaryEncodingArray() {
-    QueryDimension column1 = new QueryDimension("Column1");
-    QueryDimension column2 = new QueryDimension("Column2");
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
     column1Schema.setColumnName("Column1");
     List<Encoding> encoding = new ArrayList<>();
     encoding.add(Encoding.DIRECT_DICTIONARY);
     column1Schema.setEncodingList(encoding);
-    column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
     List<Encoding> encoding2 = new ArrayList<>();
     encoding2.add(Encoding.DELTA);
     column2Schema.setEncodingList(encoding2);
-    column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
-    QueryDimension[] queryDimensions = { column1, column2 };
+    ProjectionDimension[] queryDimensions = { column1, column2 };
 
     boolean[] dictionaryEncoding = CarbonUtil.getDirectDictionaryEncodingArray(queryDimensions);
     boolean[] expectedDictionaryEncoding = { true, false };
@@ -576,19 +576,19 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetComplexDataTypeArray() {
-    QueryDimension column1 = new QueryDimension("Column1");
-    QueryDimension column2 = new QueryDimension("Column2");
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
     column1Schema.setColumnName("Column1");
     column1Schema.setDataType(DataTypes.DATE);
-    column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
     column2Schema.setDataType(DataTypes.createDefaultArrayType());
-    column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
-    QueryDimension[] queryDimensions = { column1, column2 };
+    ProjectionDimension[] queryDimensions = { column1, column2 };
 
     boolean[] dictionaryEncoding = CarbonUtil.getComplexDataTypeArray(queryDimensions);
     boolean[] expectedDictionaryEncoding = { false, true };
@@ -805,8 +805,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithCompareTo1() {
     byte[] dataChunks = { 10, 20, 30, 40, 50, 60 };
     byte[] compareValue = { 5 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false);
     assertEquals(-2, result);
@@ -815,8 +815,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithCompareToLessThan0() {
     byte[] dataChunks = { 10, 20, 30, 40, 50, 60 };
     byte[] compareValue = { 30 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false);
     assertEquals(2, result);
@@ -825,8 +825,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithCompareTo0() {
     byte[] dataChunks = { 10, 10, 10, 40, 50, 60 };
     byte[] compareValue = { 10 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false);
     assertEquals(0, result);
@@ -835,8 +835,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithMatchUpLimitTrue() {
     byte[] dataChunks = { 10, 10, 10, 40, 50, 60 };
     byte[] compareValue = { 10 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, true);
     assertEquals(2, result);
@@ -846,13 +846,13 @@ public class CarbonUtilTest {
   public void testBinaryRangeSearch() {
 
     byte[] dataChunk = new byte[10];
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
     byte[] keyWord = new byte[1];
     int[] range;
 
     dataChunk = "abbcccddddeffgggh".getBytes();
     byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length];
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) {
@@ -884,7 +884,7 @@ public class CarbonUtilTest {
     assertRangeIndex(dataArr, dataChunk, fixedLengthDimensionDataChunk, keyWord, expectRangeIndex);
 
     dataChunk = "ab".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -898,7 +898,7 @@ public class CarbonUtilTest {
     assertEquals(1, range[1]);
 
     dataChunk = "aabb".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -912,7 +912,7 @@ public class CarbonUtilTest {
     assertEquals(3, range[1]);
 
     dataChunk = "a".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -921,7 +921,7 @@ public class CarbonUtilTest {
     assertEquals(0, range[1]);
 
     dataChunk = "aa".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -930,7 +930,7 @@ public class CarbonUtilTest {
     assertEquals(1, range[1]);
 
     dataChunk = "aabbbbbbbbbbcc".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
     keyWord[0] = Byte.valueOf("98");
     range = CarbonUtil.getRangeIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 0, dataChunk.length - 1, keyWord);
@@ -943,14 +943,14 @@ public class CarbonUtilTest {
   public void IndexUsingBinarySearchLengthTwo() {
 
     byte[] dataChunk = new byte[10];
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
 
     byte[] keyWord = new byte[2];
 
     dataChunk = "aabbbbbbbbbbcc".getBytes();
     byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length];
 
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) {
@@ -985,14 +985,14 @@ public class CarbonUtilTest {
   public void IndexUsingBinarySearchLengthThree() {
 
     byte[] dataChunk = new byte[10];
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
 
     byte[] keyWord = new byte[3];
 
     dataChunk = "aaabbbbbbbbbccc".getBytes();
     byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length];
 
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) {
@@ -1054,7 +1054,7 @@ public class CarbonUtilTest {
   }
 
   private void assertRangeIndex(byte[][] dataArr, byte[] dataChunk,
-      FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk, byte[] keyWord, int[] expectRangeIndex) {
+      FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk, byte[] keyWord, int[] expectRangeIndex) {
     int[] range;
     range = CarbonUtil.getRangeIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 0,
         (dataChunk.length - 1) / keyWord.length, keyWord);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
index de64c0a..e506994 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
@@ -25,10 +25,10 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.datastore.impl.FileHolderImpl;
+import org.apache.carbondata.core.datastore.impl.FileReaderImpl;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
@@ -229,13 +229,13 @@ public class DataFileFooterConverterTest {
       }
 
       @SuppressWarnings("unused") @Mock
-      public FileHolder getFileHolder(FileFactory.FileType fileType) {
-        return new FileHolderImpl();
+      public FileReader getFileHolder(FileFactory.FileType fileType) {
+        return new FileReaderImpl();
       }
 
     };
 
-    new MockUp<FileHolderImpl>() {
+    new MockUp<FileReaderImpl>() {
       @SuppressWarnings("unused") @Mock public long readLong(String filePath, long offset) {
         return 1;
       }
@@ -249,7 +249,6 @@ public class DataFileFooterConverterTest {
     SegmentInfo segmentInfo = new SegmentInfo();
     int[] arr = { 1, 2, 3 };
     segmentInfo.setColumnCardinality(arr);
-    segmentInfo.setNumberOfColumns(segmentInfo1.getNum_cols());
     dataFileFooter.setNumberOfRows(3);
     dataFileFooter.setSegmentInfo(segmentInfo);
     TableBlockInfo info = new TableBlockInfo("/file.carbondata", 1, "0", new String[0], 1, ColumnarFormatVersion.V1, null);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
index 4c9a784..4fb5dcc 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
@@ -36,7 +36,6 @@ import org.apache.carbondata.core.scan.expression.logical.RangeExpression;
 import org.apache.carbondata.core.scan.expression.logical.TrueExpression;
 import org.apache.carbondata.core.scan.filter.executer.RangeValueFilterExecuterImpl;
 import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer;
-import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerBasic;
 import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer;
 
 import mockit.Deencapsulation;
@@ -102,7 +101,7 @@ public class RangeFilterProcessorTest {
         new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
             new LiteralExpression("20", DataTypes.STRING))), new TrueExpression(null));
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, output);
     Assert.assertTrue(result);
@@ -143,7 +142,7 @@ public class RangeFilterProcessorTest {
         new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
             new LiteralExpression("05", DataTypes.STRING)));
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, output);
     // no change
@@ -218,7 +217,7 @@ public class RangeFilterProcessorTest {
     Expression Andb3 = new AndExpression(Andb2, new TrueExpression(null));
 
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, new AndExpression(Andb3, new TrueExpression(null)));
     // no change
@@ -302,7 +301,7 @@ public class RangeFilterProcessorTest {
     Expression Orb3 = new OrExpression(Orb2, lessThanb2);
 
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, new OrExpression(Orb3, lessThanb1));
     // no change

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java b/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
deleted file mode 100644
index 94c3f68..0000000
--- a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
+++ /dev/null
@@ -1,160 +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.scanner.impl;
-
-import static junit.framework.TestCase.assertEquals;
-
-public class FilterScannerTest {
-//
-//  private static FilterScanner filterScanner;
-//  private static BlockletIndex blockletIndex;
-//  private static BlockletMinMaxIndex blockletMinMaxIndex;
-//  private static BTreeBuilderInfo bTreeBuilderInfo;
-//  private static DataFileFooter dataFileFooter;
-//
-//  @BeforeClass public static void setUp() {
-//    BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
-//    FilterExecuter filterExecutor = new AndFilterExecuterImpl(null, null);
-//    blockExecutionInfo.setFilterExecuterTree(filterExecutor);
-//    blockExecutionInfo.setFixedLengthKeySize(1);
-//    blockExecutionInfo.setNoDictionaryBlockIndexes(new int[] { 1, 2 });
-//    blockExecutionInfo.setDictionaryColumnBlockIndex(new int[] { 1 });
-//    blockExecutionInfo.setColumnGroupToKeyStructureInfo(new HashMap<Integer, KeyStructureInfo>());
-//    blockExecutionInfo.setComplexDimensionInfoMap(new HashMap<Integer, GenericQueryType>());
-//    blockExecutionInfo.setComplexColumnParentBlockIndexes(new int[] { 1 });
-//    blockExecutionInfo.setQueryDimensions(new QueryDimension[] { new QueryDimension("Col1") });
-//    blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[][] { { 0, 0 } });
-//    blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[][] { { 0, 0 } });
-//    blockExecutionInfo.setTotalNumberOfMeasureBlock(1);
-//    blockExecutionInfo.setTotalNumberDimensionBlock(1);
-//    QueryStatisticsModel queryStatisticsModel = new QueryStatisticsModel();
-//    QueryStatistic queryStatistic = new QueryStatistic();
-//    queryStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, 1);
-//    Map<String, QueryStatistic> statisticsTypeAndObjMap = new HashMap<>();
-//    statisticsTypeAndObjMap.put(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, queryStatistic);
-//    statisticsTypeAndObjMap.put(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM, queryStatistic);
-//    queryStatisticsModel.setStatisticsTypeAndObjMap(statisticsTypeAndObjMap);
-//    QueryStatisticsRecorder queryStatisticsRecorder = new QueryStatisticsRecorderImpl("1");
-//    queryStatisticsModel.setRecorder(queryStatisticsRecorder);
-//    filterScanner = new FilterScanner(blockExecutionInfo, queryStatisticsModel);
-//    blockletIndex = new BlockletIndex();
-//    blockletMinMaxIndex = new BlockletMinMaxIndex();
-//    blockletMinMaxIndex.setMinValues(new byte[][] { { 1, 2 } });
-//    blockletMinMaxIndex.setMaxValues(new byte[][] { { 10, 12 } });
-//    blockletIndex.setMinMaxIndex(blockletMinMaxIndex);
-//    dataFileFooter = new DataFileFooter();
-//    dataFileFooter.setBlockletIndex(blockletIndex);
-//    bTreeBuilderInfo = new BTreeBuilderInfo(Arrays.asList(dataFileFooter), new int[] { 1 });
-//  }
-//
-//  @Test public void testToScanBlockletWithEmptyBitSet() throws QueryExecutionException {
-//    new MockUp<AndFilterExecuterImpl>() {
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-//        return new BitSet();
-//      }
-//    };
-//    BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1);
-//    DataRefNode dataRefNode = new BlockBTreeLeafNode(bTreeBuilderInfo, 0, 1);
-//    blocksChunkHolder.setDataBlock(dataRefNode);
-//    AbstractScannedResult abstractScannedResult = filterScanner.scanBlocklet(blocksChunkHolder);
-//    assertEquals(0, abstractScannedResult.numberOfOutputRows());
-//  }
-//
-//  @Test public void testToScanBlockletWithNonEmptyBitSet() throws QueryExecutionException {
-//    new MockUp<AndFilterExecuterImpl>() {
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-//        BitSet bitSet = new BitSet();
-//        bitSet.set(1);
-//        bitSet.set(2);
-//        bitSet.set(1);
-//        return bitSet;
-//      }
-//
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
-//          throws FilterUnsupportedException {
-//        BitSet bitSet = new BitSet();
-//        bitSet.set(1);
-//        bitSet.set(2);
-//        bitSet.set(1);
-//        return bitSet;
-//      }
-//    };
-//    DataRefNode dataRefNode = new MockUp<DataRefNode>() {
-//      @Mock @SuppressWarnings("unused") DimensionColumnDataChunk[] getDimensionChunks(
-//          FileHolder fileReader, int[][] blockIndexes) {
-//        DimensionColumnDataChunk[] dimensionChunkAttributes =
-//            { new ColumnGroupDimensionDataChunk(null, null) };
-//        return dimensionChunkAttributes;
-//      }
-//
-//      @Mock @SuppressWarnings("unused") ColumnPage[] getMeasureChunks(
-//          FileHolder fileReader, int[][] blockIndexes) {
-//
-//        ColumnPage[] ColumnPages = { new ColumnPage() };
-//        return ColumnPages;
-//      }
-//    }.getMockInstance();
-//
-//    BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1);
-//    blocksChunkHolder.setDataBlock(dataRefNode);
-//    DimensionChunkAttributes dimensionChunkAttributes = new DimensionChunkAttributes();
-//    DimensionColumnDataChunk dimensionColumnDataChunk =
-//        new FixedLengthDimensionDataChunk(new byte[] { 0, 1 }, dimensionChunkAttributes);
-//    blocksChunkHolder.setDimensionRawDataChunk(new DimensionColumnDataChunk[]
-//
-//        { dimensionColumnDataChunk });
-//    ColumnPage ColumnPage = new ColumnPage();
-//    blocksChunkHolder.setMeasureDataChunk(new ColumnPage[]
-//
-//        { ColumnPage });
-//    FileHolder fileHolder = new DFSFileHolderImpl();
-//    blocksChunkHolder.setFileReader(fileHolder);
-//    AbstractScannedResult abstractScannedResult = filterScanner.scanBlocklet(blocksChunkHolder);
-//
-//    assertEquals(2, abstractScannedResult.numberOfOutputRows());
-//  }
-//
-//  @Test(expected = QueryExecutionException.class) public void testToScanBlockletWithException()
-//      throws QueryExecutionException {
-//    new MockUp<AndFilterExecuterImpl>() {
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-//        BitSet bitSet = new BitSet();
-//        bitSet.set(1);
-//        bitSet.set(2);
-//        bitSet.set(1);
-//        return bitSet;
-//      }
-//
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
-//          throws FilterUnsupportedException {
-//        throw new FilterUnsupportedException("Filter unsupported");
-//      }
-//    };
-//    BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1);
-//    BTreeBuilderInfo bTreeBuilderInfo =
-//        new BTreeBuilderInfo(Arrays.asList(dataFileFooter), new int[] { 1 });
-//    DataRefNode dataRefNode = new BlockBTreeLeafNode(bTreeBuilderInfo, 0, 1);
-//    blocksChunkHolder.setDataBlock(dataRefNode);
-//    filterScanner.scanBlocklet(blocksChunkHolder);
-//  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/dev/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/dev/findbugs-exclude.xml b/dev/findbugs-exclude.xml
index 1520cd4..b19db85 100644
--- a/dev/findbugs-exclude.xml
+++ b/dev/findbugs-exclude.xml
@@ -31,7 +31,7 @@
     <Bug pattern="OBL_UNSATISFIED_OBLIGATION_EXCEPTION_EDGE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datastore.impl.FileHolderImpl"/>
+    <Class name="org.apache.carbondata.core.datastore.impl.FileReaderImpl"/>
     <Method name="getDataInputStream"/>
     <Bug pattern="OBL_UNSATISFIED_OBLIGATION_EXCEPTION_EDGE"/>
   </Match>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
index 76afcbf..7a15327 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
@@ -62,7 +62,7 @@ object CarbonSessionExample {
 
     spark.sql(
       s"""
-         | SELECT *
+         | SELECT charField, stringField, intField
          | FROM carbon_table
          | WHERE stringfield = 'spark' AND decimalField > 40
       """.stripMargin).show()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 6f1e123..b1faa6a 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -57,7 +57,6 @@ import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -110,11 +109,11 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   // comma separated list of input segment numbers
   public static final String INPUT_SEGMENT_NUMBERS =
       "mapreduce.input.carboninputformat.segmentnumbers";
-  public static final String VALIDATE_INPUT_SEGMENT_IDs =
+  private static final String VALIDATE_INPUT_SEGMENT_IDs =
       "mapreduce.input.carboninputformat.validsegments";
   // comma separated list of input files
   public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files";
-  public static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
+  private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
   private static final Log LOG = LogFactory.getLog(CarbonTableInputFormat.class);
   private static final String FILTER_PREDICATE =
       "mapreduce.input.carboninputformat.filter.predicate";
@@ -125,7 +124,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
   public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
   public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
-  public static final String PARTITIONS_TO_PRUNE =
+  private static final String PARTITIONS_TO_PRUNE =
       "mapreduce.input.carboninputformat.partitions.to.prune";
   public static final String UPADTE_T =
       "mapreduce.input.carboninputformat.partitions.to.prune";
@@ -307,7 +306,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   /**
    * get list of partitions to prune
    */
-  public static List<String> getPartitionsToPrune(Configuration configuration) throws IOException {
+  private static List<String> getPartitionsToPrune(Configuration configuration) throws IOException {
     String partitionString = configuration.get(PARTITIONS_TO_PRUNE);
     if (partitionString != null) {
       return (List<String>) ObjectSerializationUtil.convertStringToObject(partitionString);
@@ -335,7 +334,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
    * @return List<InputSplit> list of CarbonInputSplit
    * @throws IOException
    */
-  @Override public List<InputSplit> getSplits(JobContext job) throws IOException {
+  @Override
+  public List<InputSplit> getSplits(JobContext job) throws IOException {
     AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
     SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
     CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
@@ -789,28 +789,29 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     return split;
   }
 
-  @Override public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
+  @Override
+  public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
       TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
-    QueryModel queryModel = getQueryModel(inputSplit, taskAttemptContext);
+    QueryModel queryModel = createQueryModel(inputSplit, taskAttemptContext);
     CarbonReadSupport<T> readSupport = getReadSupportClass(configuration);
     return new CarbonRecordReader<T>(queryModel, readSupport);
   }
 
-  public QueryModel getQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+  public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
     TableProvider tableProvider = new SingleTableProvider(carbonTable);
-    // getting the table absoluteTableIdentifier from the carbonTable
-    // to avoid unnecessary deserialization
-    AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
 
     // query plan includes projection column
-    String projection = getColumnProjection(configuration);
-    CarbonQueryPlan queryPlan = CarbonInputFormatUtil.createQueryPlan(carbonTable, projection);
-    QueryModel queryModel = QueryModel.createModel(identifier, queryPlan, carbonTable,
-        getDataTypeConverter(configuration));
+    String projectionString = getColumnProjection(configuration);
+    String[] projectionColumnNames = null;
+    if (projectionString != null) {
+      projectionColumnNames = projectionString.split(",");
+    }
+    QueryModel queryModel = carbonTable.createQueryWithProjection(
+        projectionColumnNames, getDataTypeConverter(configuration));
 
     // set the filter to the query model in order to filter blocklet before scan
     Expression filter = getFilterPredicates(configuration);
@@ -865,7 +866,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     return readSupport;
   }
 
-  @Override protected boolean isSplitable(JobContext context, Path filename) {
+  @Override
+  protected boolean isSplitable(JobContext context, Path filename) {
     try {
       // Don't split the file if it is local file system
       FileSystem fileSystem = filename.getFileSystem(context.getConfiguration());
@@ -879,19 +881,9 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   }
 
   /**
-   * required to be moved to core
-   *
-   * @return updateExtension
-   */
-  private String getUpdateExtension() {
-    // TODO: required to modify when supporting update, mostly will be update timestamp
-    return "update";
-  }
-
-  /**
    * return valid segment to access
    */
-  public String[] getSegmentsToAccess(JobContext job) {
+  private String[] getSegmentsToAccess(JobContext job) {
     String segmentString = job.getConfiguration().get(INPUT_SEGMENT_NUMBERS, "");
     if (segmentString.trim().isEmpty()) {
       return new String[0];

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
index a590a5b..0fe0cbf 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
@@ -176,9 +176,7 @@ class InMemoryBTreeIndex implements Index {
         filterredBlocks = filterExpressionProcessor.getFilterredBlocks(
             abstractIndex.getDataRefNode(),
             resolver,
-            abstractIndex,
-            identifier
-        );
+            abstractIndex);
       }
       resultFilterredBlocks.addAll(filterredBlocks);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
index 19626f0..e7c6dda 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
@@ -152,7 +152,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
     hadoopConf = context.getConfiguration();
     if (model == null) {
       CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
-      model = format.getQueryModel(split, context);
+      model = format.createQueryModel(split, context);
     }
     carbonTable = model.getTable();
     List<CarbonDimension> dimensions =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
index 89a4a9a..2f28861 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
@@ -67,7 +67,7 @@ public class BlockLevelTraverser {
       blockName = CarbonTablePath.getCarbonDataFileName(blockName);
       blockName = blockName + CarbonTablePath.getCarbonDataExtension();
 
-      long rowCount = currentBlock.nodeSize();
+      long rowCount = currentBlock.numRows();
 
       String key = CarbonUpdateUtil.getSegmentBlockNameKey(segId, blockName);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 056c27b..9f8c5ec 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -26,18 +26,12 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer;
-import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerBasic;
 import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
-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.hadoop.api.CarbonTableInputFormat;
 
@@ -52,45 +46,14 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
  */
 public class CarbonInputFormatUtil {
 
-  public static CarbonQueryPlan createQueryPlan(CarbonTable carbonTable, String columnString) {
-    String[] columns = null;
-    if (columnString != null) {
-      columns = columnString.split(",");
-    }
-    String factTableName = carbonTable.getTableName();
-    CarbonQueryPlan plan = new CarbonQueryPlan(carbonTable.getDatabaseName(), factTableName);
-    // fill dimensions
-    // If columns are null, set all dimensions and measures
-    int i = 0;
-    if (columns != null) {
-      for (String column : columns) {
-        CarbonDimension dimensionByName = carbonTable.getDimensionByName(factTableName, column);
-        if (dimensionByName != null) {
-          addQueryDimension(plan, i, dimensionByName);
-          i++;
-        } else {
-          CarbonMeasure measure = carbonTable.getMeasureByName(factTableName, column);
-          if (measure == null) {
-            throw new RuntimeException(column + " column not found in the table " + factTableName);
-          }
-          addQueryMeasure(plan, i, measure);
-          i++;
-        }
-      }
-    }
-
-    plan.setQueryId(System.nanoTime() + "");
-    return plan;
-  }
-
   public static <V> CarbonTableInputFormat<V> createCarbonInputFormat(
       AbsoluteTableIdentifier identifier,
       Job job) throws IOException {
     CarbonTableInputFormat<V> carbonInputFormat = new CarbonTableInputFormat<>();
-    carbonInputFormat.setDatabaseName(job.getConfiguration(),
-        identifier.getCarbonTableIdentifier().getDatabaseName());
-    carbonInputFormat
-        .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
+    CarbonTableInputFormat.setDatabaseName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getDatabaseName());
+    CarbonTableInputFormat.setTableName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
     FileInputFormat.addInputPath(job, new Path(identifier.getTablePath()));
     return carbonInputFormat;
   }
@@ -98,30 +61,16 @@ public class CarbonInputFormatUtil {
   public static <V> CarbonTableInputFormat<V> createCarbonTableInputFormat(
       AbsoluteTableIdentifier identifier, List<String> partitionId, Job job) throws IOException {
     CarbonTableInputFormat<V> carbonTableInputFormat = new CarbonTableInputFormat<>();
-    carbonTableInputFormat.setPartitionIdList(job.getConfiguration(), partitionId);
-    carbonTableInputFormat.setDatabaseName(job.getConfiguration(),
-        identifier.getCarbonTableIdentifier().getDatabaseName());
-    carbonTableInputFormat
-        .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
+    CarbonTableInputFormat.setPartitionIdList(
+        job.getConfiguration(), partitionId);
+    CarbonTableInputFormat.setDatabaseName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getDatabaseName());
+    CarbonTableInputFormat.setTableName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
     FileInputFormat.addInputPath(job, new Path(identifier.getTablePath()));
     return carbonTableInputFormat;
   }
 
-  private static void addQueryMeasure(CarbonQueryPlan plan, int order, CarbonMeasure measure) {
-    QueryMeasure queryMeasure = new QueryMeasure(measure.getColName());
-    queryMeasure.setQueryOrder(order);
-    queryMeasure.setMeasure(measure);
-    plan.addMeasure(queryMeasure);
-  }
-
-  private static void addQueryDimension(CarbonQueryPlan plan, int order,
-      CarbonDimension dimension) {
-    QueryDimension queryDimension = new QueryDimension(dimension.getColName());
-    queryDimension.setQueryOrder(order);
-    queryDimension.setDimension(dimension);
-    plan.addDimension(queryDimension);
-  }
-
   public static void processFilterExpression(Expression filterExpression, CarbonTable carbonTable,
       boolean[] isFilterDimensions, boolean[] isFilterMeasures) {
     QueryModel.processFilterExpression(carbonTable, filterExpression, isFilterDimensions,
@@ -130,7 +79,7 @@ public class CarbonInputFormatUtil {
     if (null != filterExpression) {
       // Optimize Filter Expression and fit RANGE filters is conditions apply.
       FilterOptimizer rangeFilterOptimizer =
-          new RangeFilterOptmizer(new FilterOptimizerBasic(), filterExpression);
+          new RangeFilterOptmizer(filterExpression);
       rangeFilterOptimizer.optimizeFilter();
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
index f109e1c..1b57f93 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
@@ -30,7 +30,6 @@ import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.util.DataTypeConverterImpl;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
@@ -140,11 +139,11 @@ public class MapredCarbonInputFormat extends CarbonTableInputFormat<ArrayWritabl
 
     AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
 
-    String projection = getProjection(configuration, carbonTable,
+    String projectionString = getProjection(configuration, carbonTable,
         identifier.getCarbonTableIdentifier().getTableName());
-    CarbonQueryPlan queryPlan = CarbonInputFormatUtil.createQueryPlan(carbonTable, projection);
-    QueryModel queryModel =
-        QueryModel.createModel(identifier, queryPlan, carbonTable, new DataTypeConverterImpl());
+    String[] projectionColumns = projectionString.split(",");
+    QueryModel queryModel = carbonTable.createQueryWithProjection(
+        projectionColumns, new DataTypeConverterImpl());
     // set the filter to the query model in order to filter blocklet before scan
     Expression filter = getFilterPredicates(configuration);
     CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
deleted file mode 100644
index 9a8f8c5..0000000
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
+++ /dev/null
@@ -1,243 +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.presto;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.datatype.StructField;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.scan.executor.QueryExecutor;
-import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
-import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-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.result.iterator.AbstractDetailQueryResultIterator;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.hadoop.AbstractRecordReader;
-import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the
- * carbondata column APIs and fills the data directly into columns.
- */
-class CarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
-
-  private int batchIdx = 0;
-
-  private int numBatched = 0;
-
-  private CarbonVectorBatch columnarBatch;
-
-  private CarbonColumnarBatch carbonColumnarBatch;
-
-  /**
-   * If true, this class returns batches instead of rows.
-   */
-  private boolean returnColumnarBatch;
-
-  private QueryModel queryModel;
-
-  private AbstractDetailQueryResultIterator iterator;
-
-  private QueryExecutor queryExecutor;
-
-  public CarbonVectorizedRecordReader(QueryExecutor queryExecutor, QueryModel queryModel, AbstractDetailQueryResultIterator iterator) {
-    this.queryModel = queryModel;
-    this.iterator = iterator;
-    this.queryExecutor = queryExecutor;
-    enableReturningBatches();
-  }
-
-  /**
-   * Implementation of RecordReader API.
-   */
-  @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
-      throws IOException, InterruptedException, UnsupportedOperationException {
-    // The input split can contain single HDFS block or multiple blocks, so firstly get all the
-    // blocks and then set them in the query model.
-    List<CarbonInputSplit> splitList;
-    if (inputSplit instanceof CarbonInputSplit) {
-      splitList = new ArrayList<>(1);
-      splitList.add((CarbonInputSplit) inputSplit);
-    } else if (inputSplit instanceof CarbonMultiBlockSplit) {
-      // contains multiple blocks, this is an optimization for concurrent query.
-      CarbonMultiBlockSplit multiBlockSplit = (CarbonMultiBlockSplit) inputSplit;
-      splitList = multiBlockSplit.getAllSplits();
-    } else {
-      throw new RuntimeException("unsupported input split type: " + inputSplit);
-    }
-    List<TableBlockInfo> tableBlockInfoList = CarbonInputSplit.createBlocks(splitList);
-    queryModel.setTableBlockInfos(tableBlockInfoList);
-    queryModel.setVectorReader(true);
-    try {
-      queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
-      iterator = (AbstractDetailQueryResultIterator) queryExecutor.execute(queryModel);
-    } catch (QueryExecutionException e) {
-      throw new InterruptedException(e.getMessage());
-    }
-  }
-
-  @Override public void close() throws IOException {
-    logStatistics(rowCount, queryModel.getStatisticsRecorder());
-    if (columnarBatch != null) {
-      columnarBatch = null;
-    }
-    // clear dictionary cache
-    Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
-    if (null != columnToDictionaryMapping) {
-      for (Map.Entry<String, Dictionary> entry : columnToDictionaryMapping.entrySet()) {
-        CarbonUtil.clearDictionaryCache(entry.getValue());
-      }
-    }
-    try {
-      queryExecutor.finish();
-    } catch (QueryExecutionException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
-    resultBatch();
-
-    if (returnColumnarBatch) return nextBatch();
-
-    if (batchIdx >= numBatched) {
-      if (!nextBatch()) return false;
-    }
-    ++batchIdx;
-    return true;
-  }
-
-  @Override public Object getCurrentValue() throws IOException, InterruptedException {
-    if (returnColumnarBatch) {
-      rowCount += columnarBatch.numValidRows();
-      return columnarBatch;
-    } else {
-      return null;
-    }
-  }
-
-  @Override public Void getCurrentKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @Override public float getProgress() throws IOException, InterruptedException {
-    // TODO : Implement it based on total number of rows it is going to retrive.
-    return 0;
-  }
-
-  /**
-   * Returns the ColumnarBatch object that will be used for all rows returned by this reader.
-   * This object is reused. Calling this enables the vectorized reader. This should be called
-   * before any calls to nextKeyValue/nextBatch.
-   */
-
-  private void initBatch() {
-    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
-    List<QueryMeasure> queryMeasures = queryModel.getQueryMeasures();
-    StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()];
-    for (int i = 0; i < queryDimension.size(); i++) {
-      QueryDimension dim = queryDimension.get(i);
-      if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(dim.getDimension().getDataType());
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-           generator.getReturnType());
-      } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-            dim.getDimension().getDataType());
-      } else if (dim.getDimension().isComplex()) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-           dim.getDimension().getDataType());
-      } else {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-            DataTypes.INT);
-      }
-    }
-
-    for (QueryMeasure msr : queryMeasures) {
-      DataType dataType = msr.getMeasure().getDataType();
-      if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT || dataType == DataTypes.INT
-          || dataType == DataTypes.LONG) {
-        fields[msr.getQueryOrder()] =
-            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
-      } else if (DataTypes.isDecimal(dataType)) {
-        fields[msr.getQueryOrder()] =
-            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
-      } else {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(), DataTypes.DOUBLE);
-      }
-    }
-
-    columnarBatch = CarbonVectorBatch.allocate(fields);
-    CarbonColumnVector[] vectors = new CarbonColumnVector[fields.length];
-    boolean[] filteredRows = new boolean[columnarBatch.capacity()];
-    for (int i = 0; i < fields.length; i++) {
-      vectors[i] = new CarbonColumnVectorWrapper(columnarBatch.column(i), filteredRows);
-    }
-    carbonColumnarBatch = new CarbonColumnarBatch(vectors, columnarBatch.capacity(), filteredRows);
-  }
-
-
-  private CarbonVectorBatch resultBatch() {
-    if (columnarBatch == null) initBatch();
-    return columnarBatch;
-  }
-
-  /*
-   * Can be called before any rows are returned to enable returning columnar batches directly.
-   */
-  private void enableReturningBatches() {
-    returnColumnarBatch = true;
-  }
-
-  /**
-   * Advances to the next batch of rows. Returns false if there are no more.
-   */
-  private boolean nextBatch() {
-    columnarBatch.reset();
-    carbonColumnarBatch.reset();
-    if (iterator.hasNext()) {
-      iterator.processNextBatch(carbonColumnarBatch);
-      int actualSize = carbonColumnarBatch.getActualSize();
-      columnarBatch.setNumRows(actualSize);
-      numBatched = actualSize;
-      batchIdx = 0;
-      return true;
-    }
-    return false;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
index 1679f29..5f1f90a 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
@@ -54,7 +54,7 @@ class CarbondataPageSource implements ConnectorPageSource {
   private final List<Type> types;
   private final PageBuilder pageBuilder;
   private boolean closed;
-  private CarbonVectorizedRecordReader vectorReader;
+  private PrestoCarbonVectorizedRecordReader vectorReader;
   private CarbonDictionaryDecodeReadSupport<Object[]> readSupport;
   private long sizeOfData = 0;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
index c614fa9..5772fbf 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
@@ -54,7 +54,7 @@ public class CarbondataRecordCursor implements RecordCursor {
   private CarbondataSplit split;
   private CarbonDictionaryDecodeReadSupport readSupport;
   private Tuple3<DataType, Dictionary, Int>[] dictionary;
-  CarbonVectorizedRecordReader vectorizedRecordReader;
+  PrestoCarbonVectorizedRecordReader vectorizedRecordReader;
 
   private long totalBytes;
   private long nanoStart;
@@ -63,7 +63,7 @@ public class CarbondataRecordCursor implements RecordCursor {
 
 
   public CarbondataRecordCursor(CarbonDictionaryDecodeReadSupport readSupport,
-       CarbonVectorizedRecordReader vectorizedRecordReader,
+       PrestoCarbonVectorizedRecordReader vectorizedRecordReader,
       List<CarbondataColumnHandle> columnHandles,
       CarbondataSplit split) {
     this.vectorizedRecordReader = vectorizedRecordReader;
@@ -194,7 +194,7 @@ public class CarbondataRecordCursor implements RecordCursor {
     //todo  delete cache from readSupport
   }
 
-  public CarbonVectorizedRecordReader getVectorizedRecordReader() {
+  public PrestoCarbonVectorizedRecordReader getVectorizedRecordReader() {
     return vectorizedRecordReader;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
index 0f8fe87..286ff0e 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
@@ -78,8 +78,8 @@ public class CarbondataRecordSet implements RecordSet {
       readSupport
           .initialize(queryModel.getProjectionColumns(), queryModel.getTable());
       CarbonIterator iterator = queryExecutor.execute(queryModel);
-      CarbonVectorizedRecordReader vectorReader =
-          new CarbonVectorizedRecordReader(queryExecutor, queryModel,
+      PrestoCarbonVectorizedRecordReader vectorReader =
+          new PrestoCarbonVectorizedRecordReader(queryExecutor, queryModel,
               (AbstractDetailQueryResultIterator) iterator);
       return new CarbondataRecordCursor(readSupport, vectorReader, columns, split);
     } catch (QueryExecutionException e) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
index f039daf..5a2f831 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
@@ -105,7 +105,7 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
           new TaskAttemptContextImpl(jobConf, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
       CarbonInputSplit carbonInputSplit =
           CarbonLocalInputSplit.convertSplit(carbondataSplit.getLocalInputSplit());
-      queryModel = carbonTableInputFormat.getQueryModel(carbonInputSplit, hadoopAttemptContext);
+      queryModel = carbonTableInputFormat.createQueryModel(carbonInputSplit, hadoopAttemptContext);
       queryModel.setVectorReader(true);
     } catch (IOException e) {
       throw new RuntimeException("Unable to get the Query Model ", e);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
new file mode 100644
index 0000000..a1907db
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
@@ -0,0 +1,243 @@
+/*
+ * 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.presto;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.scan.executor.QueryExecutor;
+import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
+import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
+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.scan.result.iterator.AbstractDetailQueryResultIterator;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.hadoop.AbstractRecordReader;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the
+ * carbondata column APIs and fills the data directly into columns.
+ */
+class PrestoCarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
+
+  private int batchIdx = 0;
+
+  private int numBatched = 0;
+
+  private CarbonVectorBatch columnarBatch;
+
+  private CarbonColumnarBatch carbonColumnarBatch;
+
+  /**
+   * If true, this class returns batches instead of rows.
+   */
+  private boolean returnColumnarBatch;
+
+  private QueryModel queryModel;
+
+  private AbstractDetailQueryResultIterator iterator;
+
+  private QueryExecutor queryExecutor;
+
+  public PrestoCarbonVectorizedRecordReader(QueryExecutor queryExecutor, QueryModel queryModel, AbstractDetailQueryResultIterator iterator) {
+    this.queryModel = queryModel;
+    this.iterator = iterator;
+    this.queryExecutor = queryExecutor;
+    enableReturningBatches();
+  }
+
+  /**
+   * Implementation of RecordReader API.
+   */
+  @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+      throws IOException, InterruptedException, UnsupportedOperationException {
+    // The input split can contain single HDFS block or multiple blocks, so firstly get all the
+    // blocks and then set them in the query model.
+    List<CarbonInputSplit> splitList;
+    if (inputSplit instanceof CarbonInputSplit) {
+      splitList = new ArrayList<>(1);
+      splitList.add((CarbonInputSplit) inputSplit);
+    } else if (inputSplit instanceof CarbonMultiBlockSplit) {
+      // contains multiple blocks, this is an optimization for concurrent query.
+      CarbonMultiBlockSplit multiBlockSplit = (CarbonMultiBlockSplit) inputSplit;
+      splitList = multiBlockSplit.getAllSplits();
+    } else {
+      throw new RuntimeException("unsupported input split type: " + inputSplit);
+    }
+    List<TableBlockInfo> tableBlockInfoList = CarbonInputSplit.createBlocks(splitList);
+    queryModel.setTableBlockInfos(tableBlockInfoList);
+    queryModel.setVectorReader(true);
+    try {
+      queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
+      iterator = (AbstractDetailQueryResultIterator) queryExecutor.execute(queryModel);
+    } catch (QueryExecutionException e) {
+      throw new InterruptedException(e.getMessage());
+    }
+  }
+
+  @Override public void close() throws IOException {
+    logStatistics(rowCount, queryModel.getStatisticsRecorder());
+    if (columnarBatch != null) {
+      columnarBatch = null;
+    }
+    // clear dictionary cache
+    Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
+    if (null != columnToDictionaryMapping) {
+      for (Map.Entry<String, Dictionary> entry : columnToDictionaryMapping.entrySet()) {
+        CarbonUtil.clearDictionaryCache(entry.getValue());
+      }
+    }
+    try {
+      queryExecutor.finish();
+    } catch (QueryExecutionException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+    resultBatch();
+
+    if (returnColumnarBatch) return nextBatch();
+
+    if (batchIdx >= numBatched) {
+      if (!nextBatch()) return false;
+    }
+    ++batchIdx;
+    return true;
+  }
+
+  @Override public Object getCurrentValue() throws IOException, InterruptedException {
+    if (returnColumnarBatch) {
+      rowCount += columnarBatch.numValidRows();
+      return columnarBatch;
+    } else {
+      return null;
+    }
+  }
+
+  @Override public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  @Override public float getProgress() throws IOException, InterruptedException {
+    // TODO : Implement it based on total number of rows it is going to retrive.
+    return 0;
+  }
+
+  /**
+   * Returns the ColumnarBatch object that will be used for all rows returned by this reader.
+   * This object is reused. Calling this enables the vectorized reader. This should be called
+   * before any calls to nextKeyValue/nextBatch.
+   */
+
+  private void initBatch() {
+    List<ProjectionDimension> queryDimension = queryModel.getProjectionDimensions();
+    List<ProjectionMeasure> queryMeasures = queryModel.getProjectionMeasures();
+    StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()];
+    for (int i = 0; i < queryDimension.size(); i++) {
+      ProjectionDimension dim = queryDimension.get(i);
+      if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+        DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(dim.getDimension().getDataType());
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+           generator.getReturnType());
+      } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+            dim.getDimension().getDataType());
+      } else if (dim.getDimension().isComplex()) {
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+           dim.getDimension().getDataType());
+      } else {
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+            DataTypes.INT);
+      }
+    }
+
+    for (ProjectionMeasure msr : queryMeasures) {
+      DataType dataType = msr.getMeasure().getDataType();
+      if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT || dataType == DataTypes.INT
+          || dataType == DataTypes.LONG) {
+        fields[msr.getOrdinal()] =
+            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
+      } else if (DataTypes.isDecimal(dataType)) {
+        fields[msr.getOrdinal()] =
+            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
+      } else {
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), DataTypes.DOUBLE);
+      }
+    }
+
+    columnarBatch = CarbonVectorBatch.allocate(fields);
+    CarbonColumnVector[] vectors = new CarbonColumnVector[fields.length];
+    boolean[] filteredRows = new boolean[columnarBatch.capacity()];
+    for (int i = 0; i < fields.length; i++) {
+      vectors[i] = new CarbonColumnVectorWrapper(columnarBatch.column(i), filteredRows);
+    }
+    carbonColumnarBatch = new CarbonColumnarBatch(vectors, columnarBatch.capacity(), filteredRows);
+  }
+
+
+  private CarbonVectorBatch resultBatch() {
+    if (columnarBatch == null) initBatch();
+    return columnarBatch;
+  }
+
+  /*
+   * Can be called before any rows are returned to enable returning columnar batches directly.
+   */
+  private void enableReturningBatches() {
+    returnColumnarBatch = true;
+  }
+
+  /**
+   * Advances to the next batch of rows. Returns false if there are no more.
+   */
+  private boolean nextBatch() {
+    columnarBatch.reset();
+    carbonColumnarBatch.reset();
+    if (iterator.hasNext()) {
+      iterator.processNextBatch(carbonColumnarBatch);
+      int actualSize = carbonColumnarBatch.getActualSize();
+      columnarBatch.setNumRows(actualSize);
+      numBatched = actualSize;
+      batchIdx = 0;
+      return true;
+    }
+    return false;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
index b1fc0a7..95345de 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
@@ -274,17 +274,17 @@ test("Creation of partition table should fail if the colname in table schema and
 
   test("drop partition on preAggregate table should fail"){
     sql("drop table if exists partitionTable")
-    sql("drop datamap if exists preaggTable on table partitionTable")
     sql("create table partitionTable (id int,city string,age int) partitioned by(name string) stored by 'carbondata'".stripMargin)
     sql(
-      s"""create datamap preaggTable on table partitionTable using 'preaggregate' as select id,sum(age) from partitionTable group by id"""
-        .stripMargin)
+    s"""create datamap preaggTable on table partitionTable using 'preaggregate' as select id,sum(age) from partitionTable group by id"""
+      .stripMargin)
     sql("insert into partitionTable select 1,'Bangalore',30,'John'")
     sql("insert into partitionTable select 2,'Chennai',20,'Huawei'")
     checkAnswer(sql("show partitions partitionTable"), Seq(Row("name=John"),Row("name=Huawei")))
     intercept[Exception]{
       sql("alter table partitionTable drop PARTITION(name='John')")
     }
+    sql("drop datamap if exists preaggTable on table partitionTable")
   }
 
 
@@ -318,7 +318,6 @@ test("Creation of partition table should fail if the colname in table schema and
     sql("drop table if exists badrecordsPartitionintnull")
     sql("drop table if exists badrecordsPartitionintnullalt")
     sql("drop table if exists partitionTable")
-    sql("drop datamap if exists preaggTable on table partitionTable")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index a9f8667..0d0f024 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -183,9 +183,9 @@ class CarbonMergerRDD[K, V](
           .checkIfAnyRestructuredBlockExists(segmentMapping,
             dataFileMetadataSegMapping,
             carbonTable.getTableLastUpdatedTime)
-        DataTypeUtil.setDataTypeConverter(new SparkDataTypeConverterImpl)
         exec = new CarbonCompactionExecutor(segmentMapping, segmentProperties,
-          carbonTable, dataFileMetadataSegMapping, restructuredBlockExists)
+          carbonTable, dataFileMetadataSegMapping, restructuredBlockExists,
+          new SparkDataTypeConverterImpl)
 
         // fire a query and get the results.
         var result2: java.util.List[RawResultIterator] = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 49c0225..084a748 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -331,7 +331,7 @@ class CarbonScanRDD(
     TaskMetricsMap.getInstance().registerThreadCallback()
     inputMetricsStats.initBytesReadCallback(context, inputSplit)
     val iterator = if (inputSplit.getAllSplits.size() > 0) {
-      val model = format.getQueryModel(inputSplit, attemptContext)
+      val model = format.createQueryModel(inputSplit, attemptContext)
       // get RecordReader by FileFormat
       val reader: RecordReader[Void, Object] = inputSplit.getFileFormat match {
         case FileFormat.ROW_V1 =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
index 7d42130..432d50a 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
@@ -40,7 +40,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   private DataType blockDataType;
 
-  public ColumnarVectorWrapper(ColumnVector columnVector, boolean[] filteredRows) {
+  ColumnarVectorWrapper(ColumnVector columnVector, boolean[] filteredRows) {
     this.columnVector = columnVector;
     this.filteredRows = filteredRows;
     this.dataType = CarbonScalaUtil.convertSparkToCarbonDataType(columnVector.dataType());


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
index 8c8d08f..a689d8e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
@@ -124,22 +124,22 @@ public class UnsafeFixedLengthDimensionDataChunkStore
   /**
    * to compare the two byte array
    *
-   * @param index        index of first byte array
+   * @param rowId        index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     // based on index we need to calculate the actual position in memory block
-    index = index * columnValueSize;
+    rowId = rowId * columnValueSize;
     int compareResult = 0;
     for (int i = 0; i < compareValue.length; i++) {
       compareResult = (CarbonUnsafe.getUnsafe()
-          .getByte(dataPageMemoryBlock.getBaseObject(), dataPageMemoryBlock.getBaseOffset() + index)
+          .getByte(dataPageMemoryBlock.getBaseObject(), dataPageMemoryBlock.getBaseOffset() + rowId)
           & 0xff) - (compareValue[i] & 0xff);
       if (compareResult != 0) {
         break;
       }
-      index++;
+      rowId++;
     }
     return compareResult;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
index 36b2bd8..e1eb378 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
@@ -189,11 +189,11 @@ public class UnsafeVariableLengthDimesionDataChunkStore
   /**
    * to compare the two byte array
    *
-   * @param index index of first byte array
+   * @param rowId index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     // now to get the row from memory block we need to do following thing
     // 1. first get the current offset
     // 2. if it's not a last row- get the next row offset
@@ -201,13 +201,13 @@ public class UnsafeVariableLengthDimesionDataChunkStore
     // else subtract the current row offset
     // with complete data length get the offset of set of data
     int currentDataOffset = CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-        dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((long)index
+        dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((long) rowId
             * CarbonCommonConstants.INT_SIZE_IN_BYTE * 1L));
     short length = 0;
     // calculating the length of data
-    if (index < numberOfRows - 1) {
+    if (rowId < numberOfRows - 1) {
       int OffsetOfNextdata = CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-          dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((index + 1)
+          dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((rowId + 1)
               * CarbonCommonConstants.INT_SIZE_IN_BYTE));
       length = (short) (OffsetOfNextdata - (currentDataOffset
           + CarbonCommonConstants.SHORT_SIZE_IN_BYTE));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
index 74d268a..e2a4161 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
@@ -29,13 +29,6 @@ public class ColumnGroupModel {
   private int noOfColumnsStore;
 
   /**
-   * whether given index is columnar or not
-   * true: columnar
-   * false: row block
-   */
-  private boolean[] columnarStore;
-
-  /**
    * column groups
    * e.g
    * {{0,1,2},3,4,{5,6}}
@@ -77,15 +70,6 @@ public class ColumnGroupModel {
   }
 
   /**
-   * it's an identifier for row block or single column block
-   *
-   * @param columnarStore
-   */
-  public void setColumnarStore(boolean[] columnarStore) {
-    this.columnarStore = columnarStore;
-  }
-
-  /**
    * set column groups
    *
    * @param columnGroups
@@ -95,16 +79,6 @@ public class ColumnGroupModel {
   }
 
   /**
-   * check if given column group is columnar
-   *
-   * @param colGroup
-   * @return true if given block is columnar
-   */
-  public boolean isColumnar(int colGroup) {
-    return columnarStore[colGroup];
-  }
-
-  /**
    * @return columngroups
    */
   public int[][] getColumnGroup() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
deleted file mode 100644
index 182c8eb..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
+++ /dev/null
@@ -1,166 +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.impl;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-public class DFSFileHolderImpl implements FileHolder {
-  /**
-   * cache to hold filename and its stream
-   */
-  private Map<String, FSDataInputStream> fileNameAndStreamCache;
-
-  private String queryId;
-
-  private boolean readPageByPage;
-
-
-  public DFSFileHolderImpl() {
-    this.fileNameAndStreamCache =
-        new HashMap<String, FSDataInputStream>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  @Override public byte[] readByteArray(String filePath, long offset, int length)
-      throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    return read(fileChannel, length, offset);
-  }
-
-  /**
-   * This method will be used to check whether stream is already present in
-   * cache or not for filepath if not present then create it and then add to
-   * cache, other wise get from cache
-   *
-   * @param filePath fully qualified file path
-   * @return channel
-   */
-  public FSDataInputStream updateCache(String filePath) throws IOException {
-    FSDataInputStream fileChannel = fileNameAndStreamCache.get(filePath);
-    if (null == fileChannel) {
-      Path pt = new Path(filePath);
-      FileSystem fs = pt.getFileSystem(FileFactory.getConfiguration());
-      fileChannel = fs.open(pt);
-      fileNameAndStreamCache.put(filePath, fileChannel);
-    }
-    return fileChannel;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @param offset  position
-   * @return byte buffer
-   */
-  private byte[] read(FSDataInputStream channel, int size, long offset) throws IOException {
-    byte[] byteBffer = new byte[size];
-    channel.seek(offset);
-    channel.readFully(byteBffer);
-    return byteBffer;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @return byte buffer
-   */
-  private byte[] read(FSDataInputStream channel, int size) throws IOException {
-    byte[] byteBffer = new byte[size];
-    channel.readFully(byteBffer);
-    return byteBffer;
-  }
-
-  @Override public int readInt(String filePath, long offset) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    fileChannel.seek(offset);
-    return fileChannel.readInt();
-  }
-
-  @Override public long readDouble(String filePath, long offset) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    fileChannel.seek(offset);
-    return fileChannel.readLong();
-  }
-
-  @Override public void finish() throws IOException {
-    for (Entry<String, FSDataInputStream> entry : fileNameAndStreamCache.entrySet()) {
-      FSDataInputStream channel = entry.getValue();
-      if (null != channel) {
-        channel.close();
-      }
-    }
-  }
-
-  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    return read(fileChannel, length);
-  }
-
-  @Override public long readLong(String filePath, long offset) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    fileChannel.seek(offset);
-    return fileChannel.readLong();
-  }
-
-  @Override public int readInt(String filePath) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    return fileChannel.readInt();
-  }
-
-  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
-      throws IOException {
-    byte[] readByteArray = readByteArray(filePath, offset, length);
-    ByteBuffer byteBuffer = ByteBuffer.wrap(readByteArray);
-    byteBuffer.rewind();
-    return byteBuffer;
-  }
-
-  @Override public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  @Override public String getQueryId() {
-    return queryId;
-  }
-
-  @Override public void setReadPageByPage(boolean isReadPageByPage) {
-    this.readPageByPage = isReadPageByPage;
-  }
-
-  @Override public boolean isReadPageByPage() {
-    return readPageByPage;
-  }
-
-  public Map<String, FSDataInputStream> getFileNameAndStreamCache() {
-    return fileNameAndStreamCache;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java
new file mode 100644
index 0000000..1a0cd41
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java
@@ -0,0 +1,155 @@
+/*
+ * 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.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.FileReader;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class DFSFileReaderImpl implements FileReader {
+  /**
+   * cache to hold filename and its stream
+   */
+  private Map<String, FSDataInputStream> fileNameAndStreamCache;
+
+  private boolean readPageByPage;
+
+  public DFSFileReaderImpl() {
+    this.fileNameAndStreamCache =
+        new HashMap<String, FSDataInputStream>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  @Override public byte[] readByteArray(String filePath, long offset, int length)
+      throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    return read(fileChannel, length, offset);
+  }
+
+  /**
+   * This method will be used to check whether stream is already present in
+   * cache or not for filepath if not present then create it and then add to
+   * cache, other wise get from cache
+   *
+   * @param filePath fully qualified file path
+   * @return channel
+   */
+  private FSDataInputStream updateCache(String filePath) throws IOException {
+    FSDataInputStream fileChannel = fileNameAndStreamCache.get(filePath);
+    if (null == fileChannel) {
+      Path pt = new Path(filePath);
+      FileSystem fs = pt.getFileSystem(FileFactory.getConfiguration());
+      fileChannel = fs.open(pt);
+      fileNameAndStreamCache.put(filePath, fileChannel);
+    }
+    return fileChannel;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @param offset  position
+   * @return byte buffer
+   */
+  private byte[] read(FSDataInputStream channel, int size, long offset) throws IOException {
+    byte[] byteBffer = new byte[size];
+    channel.seek(offset);
+    channel.readFully(byteBffer);
+    return byteBffer;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @return byte buffer
+   */
+  private byte[] read(FSDataInputStream channel, int size) throws IOException {
+    byte[] byteBffer = new byte[size];
+    channel.readFully(byteBffer);
+    return byteBffer;
+  }
+
+  @Override public int readInt(String filePath, long offset) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    fileChannel.seek(offset);
+    return fileChannel.readInt();
+  }
+
+  @Override public long readDouble(String filePath, long offset) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    fileChannel.seek(offset);
+    return fileChannel.readLong();
+  }
+
+  @Override public void finish() throws IOException {
+    for (Entry<String, FSDataInputStream> entry : fileNameAndStreamCache.entrySet()) {
+      FSDataInputStream channel = entry.getValue();
+      if (null != channel) {
+        channel.close();
+      }
+    }
+  }
+
+  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    return read(fileChannel, length);
+  }
+
+  @Override public long readLong(String filePath, long offset) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    fileChannel.seek(offset);
+    return fileChannel.readLong();
+  }
+
+  @Override public int readInt(String filePath) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    return fileChannel.readInt();
+  }
+
+  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException {
+    byte[] readByteArray = readByteArray(filePath, offset, length);
+    ByteBuffer byteBuffer = ByteBuffer.wrap(readByteArray);
+    byteBuffer.rewind();
+    return byteBuffer;
+  }
+
+  @Override public void setReadPageByPage(boolean isReadPageByPage) {
+    this.readPageByPage = isReadPageByPage;
+  }
+
+  @Override public boolean isReadPageByPage() {
+    return readPageByPage;
+  }
+
+  public Map<String, FSDataInputStream> getFileNameAndStreamCache() {
+    return fileNameAndStreamCache;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
index 67648fe..b58a473 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
@@ -17,24 +17,28 @@
 
 package org.apache.carbondata.core.datastore.impl;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.filesystem.*;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.filesystem.AlluxioCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.HDFSCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.ViewFSCarbonFile;
 
 import org.apache.hadoop.conf.Configuration;
 
 public class DefaultFileTypeProvider implements FileTypeInerface {
 
-  public FileHolder getFileHolder(FileFactory.FileType fileType) {
+  public FileReader getFileHolder(FileFactory.FileType fileType) {
     switch (fileType) {
       case LOCAL:
-        return new FileHolderImpl();
+        return new FileReaderImpl();
       case HDFS:
       case ALLUXIO:
       case VIEWFS:
       case S3:
-        return new DFSFileHolderImpl();
+        return new DFSFileReaderImpl();
       default:
-        return new FileHolderImpl();
+        return new FileReaderImpl();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index daf6d93..29c692f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -28,7 +28,7 @@ import java.nio.channels.FileChannel;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 
 import org.apache.commons.io.FileUtils;
@@ -62,7 +62,7 @@ public final class FileFactory {
     return configuration;
   }
 
-  public static FileHolder getFileHolder(FileType fileType) {
+  public static FileReader getFileHolder(FileType fileType) {
     return fileFileTypeInerface.getFileHolder(fileType);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
deleted file mode 100644
index cc589b7..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
+++ /dev/null
@@ -1,224 +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.impl;
-
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
-
-public class FileHolderImpl implements FileHolder {
-  /**
-   * cache to hold filename and its stream
-   */
-  private Map<String, FileChannel> fileNameAndStreamCache;
-  private String queryId;
-
-  private boolean readPageByPage;
-
-  /**
-   * FileHolderImpl Constructor
-   * It will create the cache
-   */
-  public FileHolderImpl() {
-    this.fileNameAndStreamCache =
-        new HashMap<String, FileChannel>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  public FileHolderImpl(int capacity) {
-    this.fileNameAndStreamCache = new HashMap<String, FileChannel>(capacity);
-  }
-
-  /**
-   * 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
-   */
-  @Override public byte[] readByteArray(String filePath, long offset, int length)
-      throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, length, offset);
-    return byteBffer.array();
-  }
-
-  /**
-   * This method will be used to close all the streams currently present in the cache
-   */
-  @Override public void finish() throws IOException {
-    for (Entry<String, FileChannel> entry : fileNameAndStreamCache.entrySet()) {
-      FileChannel channel = entry.getValue();
-      if (null != channel) {
-        channel.close();
-      }
-    }
-  }
-
-  /**
-   * 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
-   */
-  @Override public int readInt(String filePath, long offset) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE, offset);
-    return byteBffer.getInt();
-  }
-
-  /**
-   * 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
-   */
-  @Override public int readInt(String filePath) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE);
-    return byteBffer.getInt();
-  }
-
-  /**
-   * 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
-   */
-  @Override public long readDouble(String filePath, long offset) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
-    return byteBffer.getLong();
-  }
-
-  /**
-   * This method will be used to check whether stream is already present in
-   * cache or not for filepath if not present then create it and then add to
-   * cache, other wise get from cache
-   *
-   * @param filePath fully qualified file path
-   * @return channel
-   */
-  private FileChannel updateCache(String filePath) throws FileNotFoundException {
-    FileChannel fileChannel = fileNameAndStreamCache.get(filePath);
-    if (null == fileChannel) {
-      FileInputStream stream = new FileInputStream(filePath);
-      fileChannel = stream.getChannel();
-      fileNameAndStreamCache.put(filePath, fileChannel);
-    }
-    return fileChannel;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @param offset  position
-   * @return byte buffer
-   */
-  private ByteBuffer read(FileChannel channel, int size, long offset) throws IOException {
-    ByteBuffer byteBffer = ByteBuffer.allocate(size);
-    channel.position(offset);
-    channel.read(byteBffer);
-    byteBffer.rewind();
-    return byteBffer;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @return byte buffer
-   */
-  private ByteBuffer read(FileChannel channel, int size) throws IOException {
-    ByteBuffer byteBffer = ByteBuffer.allocate(size);
-    channel.read(byteBffer);
-    byteBffer.rewind();
-    return byteBffer;
-  }
-
-
-  /**
-   * 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
-   */
-  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, length);
-    return byteBffer.array();
-  }
-
-  /**
-   * 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
-   */
-  @Override public long readLong(String filePath, long offset) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
-    return byteBffer.getLong();
-  }
-
-  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
-      throws IOException {
-    ByteBuffer byteBuffer = ByteBuffer.allocate(length);
-    FileChannel fileChannel = updateCache(filePath);
-    fileChannel.position(offset);
-    fileChannel.read(byteBuffer);
-    byteBuffer.rewind();
-    return byteBuffer;
-  }
-
-  @Override public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  @Override public String getQueryId() {
-    return queryId;
-  }
-
-  @Override public void setReadPageByPage(boolean isReadPageByPage) {
-    this.readPageByPage = isReadPageByPage;
-  }
-
-  @Override public boolean isReadPageByPage() {
-    return readPageByPage;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java
new file mode 100644
index 0000000..6fef278
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java
@@ -0,0 +1,215 @@
+/*
+ * 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.impl;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.FileReader;
+
+public class FileReaderImpl implements FileReader {
+  /**
+   * cache to hold filename and its stream
+   */
+  private Map<String, FileChannel> fileNameAndStreamCache;
+
+  private boolean readPageByPage;
+
+  /**
+   * FileReaderImpl Constructor
+   * It will create the cache
+   */
+  public FileReaderImpl() {
+    this.fileNameAndStreamCache =
+        new HashMap<String, FileChannel>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  public FileReaderImpl(int capacity) {
+    this.fileNameAndStreamCache = new HashMap<String, FileChannel>(capacity);
+  }
+
+  /**
+   * 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
+   */
+  @Override public byte[] readByteArray(String filePath, long offset, int length)
+      throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, length, offset);
+    return byteBffer.array();
+  }
+
+  /**
+   * This method will be used to close all the streams currently present in the cache
+   */
+  @Override public void finish() throws IOException {
+    for (Entry<String, FileChannel> entry : fileNameAndStreamCache.entrySet()) {
+      FileChannel channel = entry.getValue();
+      if (null != channel) {
+        channel.close();
+      }
+    }
+  }
+
+  /**
+   * 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
+   */
+  @Override public int readInt(String filePath, long offset) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE, offset);
+    return byteBffer.getInt();
+  }
+
+  /**
+   * 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
+   */
+  @Override public int readInt(String filePath) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE);
+    return byteBffer.getInt();
+  }
+
+  /**
+   * 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
+   */
+  @Override public long readDouble(String filePath, long offset) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
+    return byteBffer.getLong();
+  }
+
+  /**
+   * This method will be used to check whether stream is already present in
+   * cache or not for filepath if not present then create it and then add to
+   * cache, other wise get from cache
+   *
+   * @param filePath fully qualified file path
+   * @return channel
+   */
+  private FileChannel updateCache(String filePath) throws FileNotFoundException {
+    FileChannel fileChannel = fileNameAndStreamCache.get(filePath);
+    if (null == fileChannel) {
+      FileInputStream stream = new FileInputStream(filePath);
+      fileChannel = stream.getChannel();
+      fileNameAndStreamCache.put(filePath, fileChannel);
+    }
+    return fileChannel;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @param offset  position
+   * @return byte buffer
+   */
+  private ByteBuffer read(FileChannel channel, int size, long offset) throws IOException {
+    ByteBuffer byteBffer = ByteBuffer.allocate(size);
+    channel.position(offset);
+    channel.read(byteBffer);
+    byteBffer.rewind();
+    return byteBffer;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @return byte buffer
+   */
+  private ByteBuffer read(FileChannel channel, int size) throws IOException {
+    ByteBuffer byteBffer = ByteBuffer.allocate(size);
+    channel.read(byteBffer);
+    byteBffer.rewind();
+    return byteBffer;
+  }
+
+
+  /**
+   * 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
+   */
+  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, length);
+    return byteBffer.array();
+  }
+
+  /**
+   * 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
+   */
+  @Override public long readLong(String filePath, long offset) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
+    return byteBffer.getLong();
+  }
+
+  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException {
+    ByteBuffer byteBuffer = ByteBuffer.allocate(length);
+    FileChannel fileChannel = updateCache(filePath);
+    fileChannel.position(offset);
+    fileChannel.read(byteBuffer);
+    byteBuffer.rewind();
+    return byteBuffer;
+  }
+
+  @Override public void setReadPageByPage(boolean isReadPageByPage) {
+    this.readPageByPage = isReadPageByPage;
+  }
+
+  @Override public boolean isReadPageByPage() {
+    return readPageByPage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
index 4676278..413261c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
@@ -17,14 +17,14 @@
 
 package org.apache.carbondata.core.datastore.impl;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 
 import org.apache.hadoop.conf.Configuration;
 
 public interface FileTypeInerface {
 
-  FileHolder getFileHolder(FileFactory.FileType fileType);
+  FileReader getFileHolder(FileFactory.FileType fileType);
   CarbonFile getCarbonFile(String path, FileFactory.FileType fileType);
   CarbonFile getCarbonFile(String path, FileFactory.FileType fileType, Configuration configuration);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
index 19b1f1c..fe4cf83 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
@@ -18,9 +18,8 @@ package org.apache.carbondata.core.datastore.impl.btree;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
 import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
@@ -31,38 +30,31 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 public abstract class AbstractBTreeLeafNode implements BTreeNode {
 
   /**
-   * Below method will be used to load the data block
-   *
-   * @param blockInfo block detail
-   */
-  protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  /**
    * number of keys in a btree
    */
-  protected int numberOfKeys;
+  int numberOfKeys;
 
   /**
    * node number
    */
-  protected long nodeNumber;
+  long nodeNumber;
 
   /**
    * Next node of the leaf
    */
-  protected BTreeNode nextNode;
+  private BTreeNode nextNode;
 
   /**
    * max key of the column this will be used to check whether this leaf will
    * be used for scanning or not
    */
-  protected byte[][] maxKeyOfColumns;
+  byte[][] maxKeyOfColumns;
 
   /**
    * min key of the column this will be used to check whether this leaf will
    * be used for scanning or not
    */
-  protected byte[][] minKeyOfColumns;
+  byte[][] minKeyOfColumns;
 
   /**
    * Method to get the next block this can be used while scanning when
@@ -70,7 +62,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    *
    * @return next block
    */
-  @Override public int nodeSize() {
+  @Override public int numRows() {
     return this.numberOfKeys;
   }
 
@@ -109,7 +101,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    *
    * @return block number
    */
-  @Override public long nodeNumber() {
+  @Override public long nodeIndex() {
     return nodeNumber;
   }
 
@@ -174,11 +166,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes indexes of the blocks need to be read
+   * @param columnIndexRange indexes of the blocks need to be read
    * @return dimension data chunks
    */
-  @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     // No required here as leaf which will will be use this class will implement its own get
     // dimension chunks
     return null;
@@ -188,11 +180,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * Below method will be used to get the dimension chunk
    *
    * @param fileReader file reader to read the chunk from file
-   * @param blockIndex block index to be read
+   * @param columnIndex block index to be read
    * @return dimension data chunk
    */
-  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader,
-      int blockIndex) throws IOException {
+  @Override public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader,
+      int columnIndex) throws IOException {
     // No required here as leaf which will will be use this class will implement
     // its own get dimension chunks
     return null;
@@ -202,11 +194,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes block indexes to be read from file
+   * @param columnIndexRange block indexes to be read from file
    * @return measure column data chunk
    */
-  @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     // No required here as leaf which will will be use this class will implement its own get
     // measure chunks
     return null;
@@ -216,30 +208,16 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * 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
    */
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex)
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     // No required here as leaf which will will be use this class will implement its own get
     // measure chunks
     return null;
   }
 
-  /**
-   * @param deleteDeltaDataCache
-   */
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) {
-
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-  /**
-   * @return the segmentProperties
-   */
-  public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-
   @Override
   public int getPageRowCount(int pageNumber) {
     throw new UnsupportedOperationException("Unsupported operation");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
index 2f8aadf..688d56a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
@@ -106,7 +106,7 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
   private BTreeNode findFirstLeafNode(IndexKey key, BTreeNode node) {
     int childNodeIndex;
     int low = 0;
-    int high = node.nodeSize() - 1;
+    int high = node.numRows() - 1;
     int mid = 0;
     int compareRes = -1;
     IndexKey[] nodeKeys = node.getNodeKeys();
@@ -156,7 +156,7 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
   private BTreeNode findLastLeafNode(IndexKey key, BTreeNode node) {
     int childNodeIndex;
     int low = 0;
-    int high = node.nodeSize() - 1;
+    int high = node.numRows() - 1;
     int mid = 0;
     int compareRes = -1;
     IndexKey[] nodeKeys = node.getNodeKeys();
@@ -172,7 +172,7 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
       } else {
         int currentPos = mid;
         // if key is matched then get the first entry
-        while (currentPos + 1 < node.nodeSize()
+        while (currentPos + 1 < node.numRows()
             && compareIndexes(key, nodeKeys[currentPos + 1]) == 0) {
           currentPos++;
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
index ccc5e12..c200f8d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
@@ -19,10 +19,9 @@ package org.apache.carbondata.core.datastore.impl.btree;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
@@ -34,13 +33,6 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 public class BTreeNonLeafNode implements BTreeNode {
 
   /**
-   * Below method will be used to load the data block
-   *
-   * @param blockInfo block detail
-   */
-  protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  /**
    * Child nodes
    */
   private BTreeNode[] children;
@@ -50,7 +42,7 @@ public class BTreeNonLeafNode implements BTreeNode {
    */
   private List<IndexKey> listOfKeys;
 
-  public BTreeNonLeafNode() {
+  BTreeNonLeafNode() {
     // creating a list which will store all the indexes
     listOfKeys = new ArrayList<IndexKey>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
   }
@@ -120,7 +112,7 @@ public class BTreeNonLeafNode implements BTreeNode {
    *
    * @return number of keys in the block
    */
-  @Override public int nodeSize() {
+  @Override public int numRows() {
     return listOfKeys.size();
   }
 
@@ -131,11 +123,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    *
    * @return block number
    */
-  @Override public long nodeNumber() {
+  @Override public long nodeIndex() {
     throw new UnsupportedOperationException("Unsupported operation");
   }
 
-  @Override public String blockletId() {
+  @Override public short blockletIndex() {
     throw new UnsupportedOperationException("Unsupported operation");
   }
 
@@ -171,11 +163,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes indexes of the blocks need to be read
+   * @param columnIndexRange indexes of the blocks need to be read
    * @return dimension data chunks
    */
-  @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader,
-      int[][] blockIndexes) {
+  @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) {
 
     // operation of getting the dimension chunks is not supported as its a
     // non leaf node
@@ -191,8 +183,8 @@ public class BTreeNonLeafNode implements BTreeNode {
    * @param fileReader file reader to read the chunk from file
    * @return dimension data chunk
    */
-  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader,
-      int blockIndexes) {
+  @Override public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader,
+      int columnIndex) {
     // operation of getting the dimension chunk is not supported as its a
     // non leaf node
     // and in case of B+Tree data will be stored only in leaf node and
@@ -205,11 +197,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes block indexes to be read from file
+   * @param columnIndexRange block indexes to be read from file
    * @return measure column data chunk
    */
-  @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) {
+  @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) {
     // operation of getting the measure chunk is not supported as its a non
     // leaf node
     // and in case of B+Tree data will be stored only in leaf node and
@@ -222,11 +214,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    * 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
    */
 
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) {
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) {
     // operation of getting the measure chunk is not supported as its a non
     // leaf node
     // and in case of B+Tree data will be stored only in leaf node and
@@ -236,20 +228,6 @@ public class BTreeNonLeafNode implements BTreeNode {
   }
 
   /**
-   * @return the segmentProperties
-   */
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) {
-
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-  /**
-   * @return the segmentProperties
-   */
-  public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-
-  /**
    * number of pages in blocklet
    * @return
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
index 25817f5..8af7eae 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
@@ -38,7 +38,7 @@ public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
    *                      node
    * @param metadataIndex metadata index
    */
-  public BlockBTreeLeafNode(BTreeBuilderInfo builderInfos, int metadataIndex, long nodeNumber) {
+  BlockBTreeLeafNode(BTreeBuilderInfo builderInfos, int metadataIndex, long nodeNumber) {
     DataFileFooter footer = builderInfos.getFooterList().get(metadataIndex);
     BlockletMinMaxIndex minMaxIndex = footer.getBlockletIndex().getMinMaxIndex();
     maxKeyOfColumns = minMaxIndex.getMaxValues();
@@ -63,8 +63,8 @@ public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
    * Below method is suppose to return the Blocklet ID.
    * @return
    */
-  @Override public String blockletId() {
-    return blockInfo.getTableBlockInfo().getDetailInfo().getBlockletId().toString();
+  @Override public short blockletIndex() {
+    return blockInfo.getTableBlockInfo().getDetailInfo().getBlockletId();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
index 94221ba..ddd7fcf 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datastore.BTreeBuilderInfo;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
@@ -69,7 +69,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    *                     this will be used during query execution when we can
    *                     give some leaf node of a btree to one executor some to other
    */
-  public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) {
+  BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) {
     // get a lead node min max
     BlockletMinMaxIndex minMaxIndex =
         builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex).getBlockletIndex()
@@ -124,23 +124,23 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
     }
   }
 
-  @Override public String blockletId() {
-    return "0";
+  @Override public short blockletIndex() {
+    return 0;
   }
 
   /**
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes indexes of the blocks need to be read
+   * @param columnIndexRange indexes of the blocks need to be read
    * @return dimension data chunks
    */
-  @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return dimensionChunksPageLevelReader.readRawDimensionChunks(fileReader, blockIndexes);
+      return dimensionChunksPageLevelReader.readRawDimensionChunks(fileReader, columnIndexRange);
     } else {
-      return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes);
+      return dimensionChunksReader.readRawDimensionChunks(fileReader, columnIndexRange);
     }
   }
 
@@ -148,15 +148,15 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * Below method will be used to get the dimension chunk
    *
    * @param fileReader file reader to read the chunk from file
-   * @param blockIndex block index to be read
+   * @param columnIndex block index to be read
    * @return dimension data chunk
    */
-  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndex)
-      throws IOException {
+  @Override public DimensionRawColumnChunk readDimensionChunk(
+      FileReader fileReader, int columnIndex) throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return dimensionChunksPageLevelReader.readRawDimensionChunk(fileReader, blockIndex);
+      return dimensionChunksPageLevelReader.readRawDimensionChunk(fileReader, columnIndex);
     } else {
-      return dimensionChunksReader.readRawDimensionChunk(fileReader, blockIndex);
+      return dimensionChunksReader.readRawDimensionChunk(fileReader, columnIndex);
     }
   }
 
@@ -164,15 +164,15 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes block indexes to be read from file
+   * @param columnIndexRange block indexes to be read from file
    * @return measure column data chunk
    */
-  @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return measureColumnChunkPageLevelReader.readRawMeasureChunks(fileReader, blockIndexes);
+      return measureColumnChunkPageLevelReader.readRawMeasureChunks(fileReader, columnIndexRange);
     } else {
-      return measureColumnChunkReader.readRawMeasureChunks(fileReader, blockIndexes);
+      return measureColumnChunkReader.readRawMeasureChunks(fileReader, columnIndexRange);
     }
   }
 
@@ -180,15 +180,15 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * 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
    */
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex)
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return measureColumnChunkPageLevelReader.readRawMeasureChunk(fileReader, blockIndex);
+      return measureColumnChunkPageLevelReader.readRawMeasureChunk(fileReader, columnIndex);
     } else {
-      return measureColumnChunkReader.readRawMeasureChunk(fileReader, blockIndex);
+      return measureColumnChunkReader.readRawMeasureChunk(fileReader, columnIndex);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
index 6d96b3b..597def0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
@@ -41,7 +41,13 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.Encoding;
 
-import static org.apache.carbondata.format.Encoding.*;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_DELTA_FLOATING;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_DELTA_INTEGRAL;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_FLOATING;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_INTEGRAL;
+import static org.apache.carbondata.format.Encoding.BOOL_BYTE;
+import static org.apache.carbondata.format.Encoding.DIRECT_COMPRESS;
+import static org.apache.carbondata.format.Encoding.RLE_INTEGRAL;
 
 /**
  * Base class for encoding factory implementation.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
index c7411d6..daba470 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
@@ -27,7 +27,6 @@ import org.apache.carbondata.core.dictionary.service.AbstractDictionaryServer;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.util.CarbonProperties;
 
-
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
index ce05fe2..2865d4b 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
@@ -213,10 +213,6 @@ public class BlockletDetailInfo implements Serializable, Writable {
     return columnSchemas;
   }
 
-  public void setColumnSchemas(List<ColumnSchema> columnSchemas) {
-    this.columnSchemas = columnSchemas;
-  }
-
   public void setColumnSchemaBinary(byte[] columnSchemaBinary) {
     this.columnSchemaBinary = columnSchemaBinary;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
new file mode 100644
index 0000000..4d10fd6
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -0,0 +1,184 @@
+/*
+ * 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.indexstore.blockletindex;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
+import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
+import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
+import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
+
+/**
+ * wrapper for blocklet data map data
+ */
+public class BlockletDataRefNode implements DataRefNode {
+
+  private List<TableBlockInfo> blockInfos;
+
+  private int index;
+
+  private int[] dimensionLens;
+
+  BlockletDataRefNode(List<TableBlockInfo> blockInfos, int index, int[] dimensionLens) {
+    this.blockInfos = blockInfos;
+    // Update row count and page count to blocklet info
+    for (TableBlockInfo blockInfo : blockInfos) {
+      BlockletDetailInfo detailInfo = blockInfo.getDetailInfo();
+      detailInfo.getBlockletInfo().setNumberOfRows(detailInfo.getRowCount());
+      detailInfo.getBlockletInfo().setNumberOfPages(detailInfo.getPagesCount());
+      detailInfo.setBlockletId(blockInfo.getDetailInfo().getBlockletId());
+      int[] pageRowCount = new int[detailInfo.getPagesCount()];
+      int numberOfPagesCompletelyFilled = detailInfo.getRowCount()
+          / CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      int lastPageRowCount = detailInfo.getRowCount()
+          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      for (int i = 0; i < numberOfPagesCompletelyFilled; i++) {
+        pageRowCount[i] =
+            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      }
+      if (lastPageRowCount > 0) {
+        pageRowCount[pageRowCount.length - 1] = lastPageRowCount;
+      }
+      detailInfo.getBlockletInfo().setNumberOfRowsPerPage(pageRowCount);
+    }
+    this.index = index;
+    this.dimensionLens = dimensionLens;
+  }
+
+  @Override public DataRefNode getNextDataRefNode() {
+    if (index + 1 < blockInfos.size()) {
+      return new BlockletDataRefNode(blockInfos, index + 1, dimensionLens);
+    }
+    return null;
+  }
+
+  @Override public int numRows() {
+    return blockInfos.get(index).getDetailInfo().getRowCount();
+  }
+
+  @Override public long nodeIndex() {
+    return index;
+  }
+
+  @Override public short blockletIndex() {
+    return blockInfos.get(index).getDetailInfo().getBlockletId();
+  }
+
+  @Override
+  public byte[][] getColumnsMaxValue() {
+    BlockletIndex blockletIndex =
+        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
+    // In case of blocklet distribution this will be null
+    if (null != blockletIndex) {
+      return blockletIndex.getMinMaxIndex().getMaxValues();
+    }
+    return null;
+  }
+
+  @Override
+  public byte[][] getColumnsMinValue() {
+    BlockletIndex blockletIndex =
+        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
+    // In case of blocklet distribution this will be null
+    if (null != blockletIndex) {
+      return blockletIndex.getMinMaxIndex().getMinValues();
+    }
+    return null;
+  }
+
+  @Override
+  public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, int[][] blockIndexes)
+      throws IOException {
+    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
+    return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes);
+  }
+
+  @Override
+  public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, int columnIndex)
+      throws IOException {
+    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
+    return dimensionChunksReader.readRawDimensionChunk(fileReader, columnIndex);
+  }
+
+  @Override
+  public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, int[][] columnIndexRange)
+      throws IOException {
+    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
+    return measureColumnChunkReader.readRawMeasureChunks(fileReader, columnIndexRange);
+  }
+
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex)
+      throws IOException {
+    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
+    return measureColumnChunkReader.readRawMeasureChunk(fileReader, columnIndex);
+  }
+
+  private DimensionColumnChunkReader getDimensionColumnChunkReader(FileReader fileReader) {
+    ColumnarFormatVersion version =
+        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
+    if (fileReader.isReadPageByPage()) {
+      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
+          blockInfos.get(index).getFilePath(), true);
+    } else {
+      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
+          blockInfos.get(index).getFilePath(), false);
+    }
+  }
+
+  private MeasureColumnChunkReader getMeasureColumnChunkReader(FileReader fileReader) {
+    ColumnarFormatVersion version =
+        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
+    if (fileReader.isReadPageByPage()) {
+      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
+          blockInfos.get(index).getFilePath(), true);
+    } else {
+      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
+          blockInfos.get(index).getFilePath(), false);
+    }
+  }
+
+  @Override public int numberOfPages() {
+    return blockInfos.get(index).getDetailInfo().getPagesCount();
+  }
+
+  @Override public int getPageRowCount(int pageNumber) {
+    return blockInfos.get(index).getDetailInfo().getBlockletInfo()
+        .getNumberOfRowsPerPage()[pageNumber];
+  }
+
+  public int numberOfNodes() {
+    return blockInfos.size();
+  }
+
+  public List<TableBlockInfo> getBlockInfos() {
+    return blockInfos;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java
deleted file mode 100644
index 097dd8c..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java
+++ /dev/null
@@ -1,197 +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.indexstore.blockletindex;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
-import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
-import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
-import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
-import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
-
-/**
- * wrapper for blocklet data map data
- */
-public class BlockletDataRefNodeWrapper implements DataRefNode {
-
-  private List<TableBlockInfo> blockInfos;
-
-  private int index;
-
-  private int[] dimensionLens;
-
-  private BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  public BlockletDataRefNodeWrapper(List<TableBlockInfo> blockInfos, int index,
-      int[] dimensionLens) {
-    this.blockInfos = blockInfos;
-    // Update row count and page count to blocklet info
-    for (TableBlockInfo blockInfo : blockInfos) {
-      BlockletDetailInfo detailInfo = blockInfo.getDetailInfo();
-      detailInfo.getBlockletInfo().setNumberOfRows(detailInfo.getRowCount());
-      detailInfo.getBlockletInfo().setNumberOfPages(detailInfo.getPagesCount());
-      detailInfo.setBlockletId(blockInfo.getDetailInfo().getBlockletId());
-      int[] pageRowCount = new int[detailInfo.getPagesCount()];
-      int numberOfPagesCompletelyFilled = detailInfo.getRowCount()
-          / CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      int lastPageRowCount = detailInfo.getRowCount()
-          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      for (int i = 0; i < numberOfPagesCompletelyFilled; i++) {
-        pageRowCount[i] =
-            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      }
-      if (lastPageRowCount > 0) {
-        pageRowCount[pageRowCount.length - 1] = lastPageRowCount;
-      }
-      detailInfo.getBlockletInfo().setNumberOfRowsPerPage(pageRowCount);
-    }
-    this.index = index;
-    this.dimensionLens = dimensionLens;
-  }
-
-  @Override public DataRefNode getNextDataRefNode() {
-    if (index + 1 < blockInfos.size()) {
-      return new BlockletDataRefNodeWrapper(blockInfos, index + 1, dimensionLens);
-    }
-    return null;
-  }
-
-  @Override public int nodeSize() {
-    return blockInfos.get(index).getDetailInfo().getRowCount();
-  }
-
-  @Override public long nodeNumber() {
-    return index;
-  }
-
-  @Override public String blockletId() {
-    return blockInfos.get(index).getDetailInfo().getBlockletId().toString();
-  }
-
-  @Override
-  public byte[][] getColumnsMaxValue() {
-    BlockletIndex blockletIndex =
-        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
-    // In case of blocklet distribution this will be null
-    if (null != blockletIndex) {
-      return blockletIndex.getMinMaxIndex().getMaxValues();
-    }
-    return null;
-  }
-
-  @Override
-  public byte[][] getColumnsMinValue() {
-    BlockletIndex blockletIndex =
-        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
-    // In case of blocklet distribution this will be null
-    if (null != blockletIndex) {
-      return blockletIndex.getMinMaxIndex().getMinValues();
-    }
-    return null;
-  }
-
-  @Override
-  public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, int[][] blockIndexes)
-      throws IOException {
-    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
-    return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes);
-  }
-
-  @Override
-  public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndexes)
-      throws IOException {
-    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
-    return dimensionChunksReader.readRawDimensionChunk(fileReader, blockIndexes);
-  }
-
-  @Override
-  public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
-      throws IOException {
-    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
-    return measureColumnChunkReader.readRawMeasureChunks(fileReader, blockIndexes);
-  }
-
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex)
-      throws IOException {
-    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
-    return measureColumnChunkReader.readRawMeasureChunk(fileReader, blockIndex);
-  }
-
-  private DimensionColumnChunkReader getDimensionColumnChunkReader(FileHolder fileReader) {
-    ColumnarFormatVersion version =
-        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
-    if (fileReader.isReadPageByPage()) {
-      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
-          blockInfos.get(index).getFilePath(), true);
-    } else {
-      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
-          blockInfos.get(index).getFilePath(), false);
-    }
-  }
-
-  private MeasureColumnChunkReader getMeasureColumnChunkReader(FileHolder fileReader) {
-    ColumnarFormatVersion version =
-        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
-    if (fileReader.isReadPageByPage()) {
-      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
-          blockInfos.get(index).getFilePath(), true);
-    } else {
-      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
-          blockInfos.get(index).getFilePath(), false);
-    }
-  }
-
-  @Override
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) {
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-
-  @Override public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-
-  @Override public int numberOfPages() {
-    return blockInfos.get(index).getDetailInfo().getPagesCount();
-  }
-
-  @Override public int getPageRowCount(int pageNumber) {
-    return blockInfos.get(index).getDetailInfo().getBlockletInfo()
-        .getNumberOfRowsPerPage()[pageNumber];
-  }
-
-  public int numberOfNodes() {
-    return blockInfos.size();
-  }
-
-  public List<TableBlockInfo> getBlockInfos() {
-    return blockInfos;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
index 17ad17f..a30f64c 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
@@ -32,7 +32,7 @@ public class IndexWrapper extends AbstractIndex {
   public IndexWrapper(List<TableBlockInfo> blockInfos) {
     segmentProperties = new SegmentProperties(blockInfos.get(0).getDetailInfo().getColumnSchemas(),
         blockInfos.get(0).getDetailInfo().getDimLens());
-    dataRefNode = new BlockletDataRefNodeWrapper(blockInfos, 0,
+    dataRefNode = new BlockletDataRefNode(blockInfos, 0,
         segmentProperties.getDimensionColumnsValueSize());
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
index a30b04c..a74b5a1 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
@@ -19,7 +19,12 @@ package org.apache.carbondata.core.indexstore.blockletindex;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 5862933..2203b3b 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.core.memory;
 
+import javax.annotation.concurrent.GuardedBy;
 import java.lang.ref.WeakReference;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
-import javax.annotation.concurrent.GuardedBy;
 
 /**
  * Code ported from Apache Spark {org.apache.spark.unsafe.memory} package

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15b4e192/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
index 0cb2918..099fffd 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
@@ -29,31 +29,12 @@ public class SegmentInfo implements Serializable {
   private static final long serialVersionUID = -1749874611112709431L;
 
   /**
-   * number of column in the segment
-   */
-  private int numberOfColumns;
-
-  /**
    * cardinality of each columns
    * column which is not participating in the multidimensional key cardinality will be -1;
    */
   private int[] columnCardinality;
 
   /**
-   * @return the numberOfColumns
-   */
-  public int getNumberOfColumns() {
-    return numberOfColumns;
-  }
-
-  /**
-   * @param numberOfColumns the numberOfColumns to set
-   */
-  public void setNumberOfColumns(int numberOfColumns) {
-    this.numberOfColumns = numberOfColumns;
-  }
-
-  /**
    * @return the columnCardinality
    */
   public int[] getColumnCardinality() {