http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
new file mode 100644
index 0000000..b88d77b
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -0,0 +1,493 @@
+/*
+ * 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.executor.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.common.logging.impl.StandardLogService;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.BlockIndexStore;
+import org.apache.carbondata.core.datastore.IndexKey;
+import org.apache.carbondata.core.datastore.block.AbstractIndex;
+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.keygenerator.KeyGenException;
+import org.apache.carbondata.core.keygenerator.KeyGenerator;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+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.executor.QueryExecutor;
+import 
org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
+import org.apache.carbondata.core.scan.executor.infos.AggregatorInfo;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.executor.util.QueryUtil;
+import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
+import org.apache.carbondata.core.scan.filter.FilterUtil;
+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.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+/**
+ * This class provides a skeletal implementation of the {@link QueryExecutor}
+ * interface to minimize the effort required to implement this interface. This
+ * will be used to prepare all the properties required for query execution
+ */
+public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractQueryExecutor.class.getName());
+  /**
+   * holder for query properties which will be used to execute the query
+   */
+  protected QueryExecutorProperties queryProperties;
+
+  /**
+   * query result iterator which will execute the query
+   * and give the result
+   */
+  protected CarbonIterator queryIterator;
+
+  public AbstractQueryExecutor() {
+    queryProperties = new QueryExecutorProperties();
+  }
+
+  /**
+   * Below method will be used to fill the executor properties based on query
+   * model it will parse the query model and get the detail and fill it in
+   * query properties
+   *
+   * @param queryModel
+   */
+  protected void initQuery(QueryModel queryModel) throws IOException {
+    StandardLogService.setThreadName(StandardLogService.getPartitionID(
+        
queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName()),
+        queryModel.getQueryId());
+    LOGGER.info("Query will be executed on table: " + 
queryModel.getAbsoluteTableIdentifier()
+        .getCarbonTableIdentifier().getTableName());
+    // add executor service for query execution
+    queryProperties.executorService = Executors.newFixedThreadPool(1);
+    // Initializing statistics list to record the query statistics
+    // creating copy on write to handle concurrent scenario
+    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
+    // query execution
+    Collections.sort(queryModel.getTableBlockInfos());
+    // get the table blocks
+    CacheProvider cacheProvider = CacheProvider.getInstance();
+    BlockIndexStore<TableBlockUniqueIdentifier, AbstractIndex> cache =
+        (BlockIndexStore) cacheProvider
+            .createCache(CacheType.EXECUTOR_BTREE, 
queryModel.getTable().getStorePath());
+    // remove the invalid table blocks, block which is deleted or compacted
+    cache.removeTableBlocks(queryModel.getInvalidSegmentIds(),
+        queryModel.getAbsoluteTableIdentifier());
+    List<TableBlockUniqueIdentifier> tableBlockUniqueIdentifiers =
+        prepareTableBlockUniqueIdentifier(queryModel.getTableBlockInfos(),
+            queryModel.getAbsoluteTableIdentifier());
+    cache.removeTableBlocksIfHorizontalCompactionDone(queryModel);
+    queryProperties.dataBlocks = cache.getAll(tableBlockUniqueIdentifiers);
+    queryStatistic
+        .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, 
System.currentTimeMillis());
+    queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
+    //
+    // // updating the restructuring infos for the query
+    queryProperties.keyStructureInfo = getKeyStructureInfo(queryModel,
+        queryProperties.dataBlocks.get(queryProperties.dataBlocks.size() - 
1).getSegmentProperties()
+            .getDimensionKeyGenerator());
+
+    // calculating the total number of aggeragted columns
+    int aggTypeCount = queryModel.getQueryMeasures().size();
+
+    int currentIndex = 0;
+    DataType[] dataTypes = new DataType[aggTypeCount];
+
+    for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
+      // adding the data type and aggregation type of all the measure this
+      // can be used
+      // to select the aggregator
+      dataTypes[currentIndex] = carbonMeasure.getMeasure().getDataType();
+      currentIndex++;
+    }
+    queryProperties.measureDataTypes = dataTypes;
+    // as aggregation will be executed in following order
+    // 1.aggregate dimension expression
+    // 2. expression
+    // 3. query measure
+    // so calculating the index of the expression start index
+    // and measure column start index
+    queryProperties.filterMeasures = new HashSet<>();
+    queryProperties.complexFilterDimension = new HashSet<>();
+    
QueryUtil.getAllFilterDimensions(queryModel.getFilterExpressionResolverTree(),
+        queryProperties.complexFilterDimension, 
queryProperties.filterMeasures);
+
+    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());
+    queryStatistic
+        .addStatistics(QueryStatisticsConstants.LOAD_DICTIONARY, 
System.currentTimeMillis());
+    queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
+    
queryModel.setColumnToDictionaryMapping(queryProperties.columnToDictionayMapping);
+  }
+
+  private List<TableBlockUniqueIdentifier> prepareTableBlockUniqueIdentifier(
+      List<TableBlockInfo> tableBlockInfos, AbsoluteTableIdentifier 
absoluteTableIdentifier) {
+    List<TableBlockUniqueIdentifier> tableBlockUniqueIdentifiers =
+        new ArrayList<>(tableBlockInfos.size());
+    for (TableBlockInfo blockInfo : tableBlockInfos) {
+      tableBlockUniqueIdentifiers
+          .add(new TableBlockUniqueIdentifier(absoluteTableIdentifier, 
blockInfo));
+    }
+    return tableBlockUniqueIdentifiers;
+  }
+
+  /**
+   * Below method will be used to get the key structure info for the query
+   *
+   * @param queryModel   query model
+   * @param keyGenerator
+   * @return key structure info
+   */
+  private KeyStructureInfo getKeyStructureInfo(QueryModel queryModel, 
KeyGenerator keyGenerator) {
+    // getting the masked byte range for dictionary column
+    int[] maskByteRanges =
+        QueryUtil.getMaskedByteRange(queryModel.getQueryDimension(), 
keyGenerator);
+
+    // max key for the dictionary dimension present in the query
+    byte[] maxKey = null;
+    try {
+      // getting the max key which will be used to masked and get the
+      // masked key
+      maxKey = 
QueryUtil.getMaxKeyBasedOnDimensions(queryModel.getQueryDimension(), 
keyGenerator);
+    } catch (KeyGenException e) {
+      LOGGER.error(e, "problem while getting the max key");
+    }
+
+    KeyStructureInfo restructureInfos = new KeyStructureInfo();
+    restructureInfos.setKeyGenerator(keyGenerator);
+    restructureInfos.setMaskByteRanges(maskByteRanges);
+    restructureInfos.setMaxKey(maxKey);
+    return restructureInfos;
+  }
+
+  protected List<BlockExecutionInfo> getBlockExecutionInfos(QueryModel 
queryModel)
+      throws IOException, QueryExecutionException {
+    initQuery(queryModel);
+    List<BlockExecutionInfo> blockExecutionInfoList = new 
ArrayList<BlockExecutionInfo>();
+    // fill all the block execution infos for all the blocks selected in
+    // query
+    // and query will be executed based on that infos
+    for (int i = 0; i < queryProperties.dataBlocks.size(); i++) {
+      blockExecutionInfoList.add(
+          getBlockExecutionInfoForBlock(queryModel, 
queryProperties.dataBlocks.get(i),
+              
queryModel.getTableBlockInfos().get(i).getBlockletInfos().getStartBlockletNumber(),
+              
queryModel.getTableBlockInfos().get(i).getBlockletInfos().getNumberOfBlockletToScan(),
+              queryModel.getTableBlockInfos().get(i).getFilePath()));
+    }
+    if (null != queryModel.getStatisticsRecorder()) {
+      QueryStatistic queryStatistic = new QueryStatistic();
+      
queryStatistic.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKS_NUM,
+          blockExecutionInfoList.size());
+      queryModel.getStatisticsRecorder().recordStatistics(queryStatistic);
+    }
+    return blockExecutionInfoList;
+  }
+
+  /**
+   * Below method will be used to get the block execution info which is
+   * required to execute any block  based on query model
+   *
+   * @param queryModel query model from user query
+   * @param blockIndex block index
+   * @return block execution info
+   * @throws QueryExecutionException any failure during block info creation
+   */
+  protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel 
queryModel,
+      AbstractIndex blockIndex, int startBlockletIndex, int 
numberOfBlockletToScan, String filePath)
+      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> updatedQueryDimension = RestructureUtil
+        .getUpdatedQueryDimension(queryModel.getQueryDimension(), 
tableBlockDimensions,
+            segmentProperties.getComplexDimensions());
+    // TODO add complex dimension children
+    int[] maskByteRangesForBlock =
+        QueryUtil.getMaskedByteRange(updatedQueryDimension, blockKeyGenerator);
+    int[] maksedByte =
+        QueryUtil.getMaskedByte(blockKeyGenerator.getKeySizeInBytes(), 
maskByteRangesForBlock);
+    int tableFactPathLength = CarbonStorePath
+        
.getCarbonTablePath(queryModel.getAbsoluteTableIdentifier().getStorePath(),
+            
queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier()).getFactDir()
+        .length() + 1;
+    blockExecutionInfo.setBlockId(filePath.substring(tableFactPathLength));
+    blockExecutionInfo.setStartBlockletIndex(startBlockletIndex);
+    blockExecutionInfo.setNumberOfBlockletToScan(numberOfBlockletToScan);
+    blockExecutionInfo.setQueryDimensions(
+        updatedQueryDimension.toArray(new 
QueryDimension[updatedQueryDimension.size()]));
+    blockExecutionInfo.setQueryMeasures(queryModel.getQueryMeasures()
+        .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
+    blockExecutionInfo.setDataBlock(blockIndex);
+    blockExecutionInfo.setBlockKeyGenerator(blockKeyGenerator);
+    // adding aggregation info for query
+    blockExecutionInfo.setAggregatorInfo(getAggregatorInfoForBlock(queryModel, 
blockIndex));
+    // setting whether raw record query or not
+    
blockExecutionInfo.setRawRecordDetailQuery(queryModel.isForcedDetailRawQuery());
+    // setting the masked byte of the block which will be
+    // used to update the unpack the older block keys
+    blockExecutionInfo.setMaskedByteForBlock(maksedByte);
+    // total number dimension
+    blockExecutionInfo
+        
.setTotalNumberDimensionBlock(segmentProperties.getDimensionOrdinalToBlockMapping().size());
+    blockExecutionInfo
+        
.setTotalNumberOfMeasureBlock(segmentProperties.getMeasuresOrdinalToBlockMapping().size());
+    
blockExecutionInfo.setAbsoluteTableIdentifier(queryModel.getAbsoluteTableIdentifier());
+    blockExecutionInfo.setComplexDimensionInfoMap(QueryUtil
+        .getComplexDimensionsMap(updatedQueryDimension,
+            segmentProperties.getDimensionOrdinalToBlockMapping(),
+            segmentProperties.getEachComplexDimColumnValueSize(),
+            queryProperties.columnToDictionayMapping, 
queryProperties.complexFilterDimension));
+    // to check whether older block key update is required or not
+    blockExecutionInfo.setFixedKeyUpdateRequired(
+        
!blockKeyGenerator.equals(queryProperties.keyStructureInfo.getKeyGenerator()));
+    IndexKey startIndexKey = null;
+    IndexKey endIndexKey = null;
+    if (null != queryModel.getFilterExpressionResolverTree()) {
+      // loading the filter executer tree for filter evaluation
+      blockExecutionInfo.setFilterExecuterTree(FilterUtil
+          .getFilterExecuterTree(queryModel.getFilterExpressionResolverTree(), 
segmentProperties,
+              blockExecutionInfo.getComlexDimensionInfoMap()));
+      List<IndexKey> listOfStartEndKeys = new ArrayList<IndexKey>(2);
+      FilterUtil.traverseResolverTreeAndGetStartAndEndKey(segmentProperties,
+          queryModel.getFilterExpressionResolverTree(), listOfStartEndKeys);
+      startIndexKey = listOfStartEndKeys.get(0);
+      endIndexKey = listOfStartEndKeys.get(1);
+    } else {
+      try {
+        startIndexKey = 
FilterUtil.prepareDefaultStartIndexKey(segmentProperties);
+        endIndexKey = FilterUtil.prepareDefaultEndIndexKey(segmentProperties);
+      } catch (KeyGenException e) {
+        throw new QueryExecutionException(e);
+      }
+    }
+    //setting the start index key of the block node
+    blockExecutionInfo.setStartKey(startIndexKey);
+    //setting the end index key of the block node
+    blockExecutionInfo.setEndKey(endIndexKey);
+    // expression dimensions
+    List<CarbonDimension> expressionDimensions =
+        new 
ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    // expression measure
+    List<CarbonMeasure> expressionMeasures =
+        new 
ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    // setting all the dimension chunk indexes to be read from file
+    int numberOfElementToConsider = 0;
+    // list of dimensions to be projected
+    List<Integer> allProjectionListDimensionIdexes = new ArrayList<>();
+    int[] dimensionsBlockIndexes = 
QueryUtil.getDimensionsBlockIndexes(updatedQueryDimension,
+        segmentProperties.getDimensionOrdinalToBlockMapping(), 
expressionDimensions,
+        queryProperties.complexFilterDimension, 
allProjectionListDimensionIdexes);
+    if (dimensionsBlockIndexes.length > 0) {
+      numberOfElementToConsider = 
dimensionsBlockIndexes[dimensionsBlockIndexes.length - 1]
+          == segmentProperties.getBlockTodimensionOrdinalMapping().size() - 1 ?
+          dimensionsBlockIndexes.length - 1 :
+          dimensionsBlockIndexes.length;
+      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(CarbonUtil
+          .getRangeIndex(dimensionsBlockIndexes, numberOfElementToConsider,
+              CarbonCommonConstants.NUMBER_OF_COLUMN_READ_IN_IO));
+    } else {
+      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[0][0]);
+    }
+
+    int[] measureBlockIndexes = QueryUtil
+        .getMeasureBlockIndexes(queryModel.getQueryMeasures(), 
expressionMeasures,
+            segmentProperties.getMeasuresOrdinalToBlockMapping(), 
queryProperties.filterMeasures);
+    if (measureBlockIndexes.length > 0) {
+
+      numberOfElementToConsider = 
measureBlockIndexes[measureBlockIndexes.length - 1]
+          == segmentProperties.getMeasures().size() - 1 ?
+          measureBlockIndexes.length - 1 :
+          measureBlockIndexes.length;
+      // setting all the measure chunk indexes to be read from file
+      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(CarbonUtil
+          .getRangeIndex(measureBlockIndexes, numberOfElementToConsider,
+              CarbonCommonConstants.NUMBER_OF_COLUMN_READ_IN_IO));
+    } else {
+      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[0][0]);
+    }
+    // setting the key structure info which will be required
+    // to update the older block key with new key generator
+    blockExecutionInfo.setKeyStructureInfo(queryProperties.keyStructureInfo);
+    // setting the size of fixed key column (dictionary column)
+    blockExecutionInfo.setFixedLengthKeySize(getKeySize(updatedQueryDimension, 
segmentProperties));
+    Set<Integer> dictionaryColumnBlockIndex = new HashSet<Integer>();
+    List<Integer> noDictionaryColumnBlockIndex = 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(updatedQueryDimension,
+        segmentProperties.getDimensionOrdinalToBlockMapping(), 
dictionaryColumnBlockIndex,
+        noDictionaryColumnBlockIndex);
+    int[] queryDictionaryColumnBlockIndexes = ArrayUtils.toPrimitive(
+        dictionaryColumnBlockIndex.toArray(new 
Integer[dictionaryColumnBlockIndex.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);
+    // setting the no dictionary column block indexes
+    blockExecutionInfo.setNoDictionaryBlockIndexes(ArrayUtils.toPrimitive(
+        noDictionaryColumnBlockIndex.toArray(new 
Integer[noDictionaryColumnBlockIndex.size()])));
+    // setting column id to dictionary mapping
+    
blockExecutionInfo.setColumnIdToDcitionaryMapping(queryProperties.columnToDictionayMapping);
+    // setting each column value size
+    
blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize());
+    blockExecutionInfo.setComplexColumnParentBlockIndexes(
+        getComplexDimensionParentBlockIndexes(updatedQueryDimension));
+    blockExecutionInfo.setVectorBatchCollector(queryModel.isVectorReader());
+    try {
+      // to set column group and its key structure info which will be used
+      // to
+      // for getting the column group column data in case of final row
+      // and in case of dimension aggregation
+      blockExecutionInfo.setColumnGroupToKeyStructureInfo(
+          QueryUtil.getColumnGroupKeyStructureInfo(updatedQueryDimension, 
segmentProperties));
+    } catch (KeyGenException e) {
+      throw new QueryExecutionException(e);
+    }
+    return blockExecutionInfo;
+  }
+
+  /**
+   * This method will be used to get fixed key length size this will be used
+   * to create a row from column chunk
+   *
+   * @param queryDimension    query dimension
+   * @param blockMetadataInfo block metadata info
+   * @return key size
+   */
+  private int getKeySize(List<QueryDimension> queryDimension, 
SegmentProperties blockMetadataInfo) {
+    List<Integer> fixedLengthDimensionOrdinal =
+        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    int counter = 0;
+    while (counter < queryDimension.size()) {
+      if (queryDimension.get(counter).getDimension().numberOfChild() > 0) {
+        counter += queryDimension.get(counter).getDimension().numberOfChild();
+        continue;
+      } else if 
(!CarbonUtil.hasEncoding(queryDimension.get(counter).getDimension().getEncoder(),
+          Encoding.DICTIONARY)) {
+        counter++;
+      } else {
+        
fixedLengthDimensionOrdinal.add(queryDimension.get(counter).getDimension().getKeyOrdinal());
+        counter++;
+      }
+    }
+    int[] dictioanryColumnOrdinal = ArrayUtils.toPrimitive(
+        fixedLengthDimensionOrdinal.toArray(new 
Integer[fixedLengthDimensionOrdinal.size()]));
+    if (dictioanryColumnOrdinal.length > 0) {
+      return blockMetadataInfo.getFixedLengthKeySplitter()
+          .getKeySizeByBlock(dictioanryColumnOrdinal);
+    }
+    return 0;
+  }
+
+  /**
+   * Below method will be used to get the aggrgator info for the query
+   *
+   * @param queryModel query model
+   * @param tableBlock table block
+   * @return aggregator info
+   */
+  private AggregatorInfo getAggregatorInfoForBlock(QueryModel queryModel,
+      AbstractIndex tableBlock) {
+    // getting the aggregate infos which will be used during aggregation
+    AggregatorInfo aggregatorInfos = RestructureUtil
+        .getAggregatorInfos(queryModel.getQueryMeasures(),
+            tableBlock.getSegmentProperties().getMeasures());
+    // setting the measure aggregator for all aggregation function selected
+    // in query
+    aggregatorInfos.setMeasureDataTypes(queryProperties.measureDataTypes);
+    return aggregatorInfos;
+  }
+
+  private int[] getComplexDimensionParentBlockIndexes(List<QueryDimension> 
queryDimensions) {
+    List<Integer> parentBlockIndexList = new ArrayList<Integer>();
+    for (QueryDimension queryDimension : queryDimensions) {
+      if (CarbonUtil.hasDataType(queryDimension.getDimension().getDataType(),
+          new DataType[] { DataType.ARRAY, DataType.STRUCT, DataType.MAP })) {
+        parentBlockIndexList.add(queryDimension.getDimension().getOrdinal());
+      }
+    }
+    return ArrayUtils
+        .toPrimitive(parentBlockIndexList.toArray(new 
Integer[parentBlockIndexList.size()]));
+  }
+
+  /**
+   * Below method will be used to finish the execution
+   *
+   * @throws QueryExecutionException
+   */
+  @Override public void finish() throws QueryExecutionException {
+    CarbonUtil.clearBlockCache(queryProperties.dataBlocks);
+    if(null != queryIterator) {
+      queryIterator.close();
+    }
+    if (null != queryProperties.executorService) {
+      queryProperties.executorService.shutdown();
+      try {
+        queryProperties.executorService.awaitTermination(1, TimeUnit.HOURS);
+      } catch (InterruptedException e) {
+        throw new QueryExecutionException(e);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
new file mode 100644
index 0000000..9c5b6a2
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
@@ -0,0 +1,50 @@
+/*
+ * 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.executor.impl;
+
+import java.io.IOException;
+import java.util.List;
+
+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.iterator.DetailQueryResultIterator;
+
+/**
+ * Below class will be used to execute the detail query
+ * 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> {
+
+  @Override
+  public CarbonIterator<BatchResult> execute(QueryModel queryModel)
+      throws QueryExecutionException, IOException {
+    List<BlockExecutionInfo> blockExecutionInfoList = 
getBlockExecutionInfos(queryModel);
+    this.queryIterator = new DetailQueryResultIterator(
+        blockExecutionInfoList,
+        queryModel,
+        queryProperties.executorService
+    );
+    return queryIterator;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/QueryExecutorProperties.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/QueryExecutorProperties.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/QueryExecutorProperties.java
new file mode 100644
index 0000000..ed0ca66
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/QueryExecutorProperties.java
@@ -0,0 +1,73 @@
+/*
+ * 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.executor.impl;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.datastore.block.AbstractIndex;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+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.KeyStructureInfo;
+import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
+
+/**
+ * Holds all the properties required for query execution
+ */
+public class QueryExecutorProperties {
+
+  /**
+   * holds the information required for updating the order block
+   * dictionary key
+   */
+  public KeyStructureInfo keyStructureInfo;
+
+  /**
+   * this will hold the information about the dictionary dimension
+   * which to
+   */
+  public Map<String, Dictionary> columnToDictionayMapping;
+
+  /**
+   * Measure datatypes
+   */
+  public DataType[] measureDataTypes;
+  /**
+   * all the complex dimension which is on filter
+   */
+  public Set<CarbonDimension> complexFilterDimension;
+
+  public Set<CarbonMeasure> filterMeasures;
+  /**
+   * to record the query execution details phase wise
+   */
+  public QueryStatisticsRecorder queryStatisticsRecorder;
+  /**
+   * executor service to execute the query
+   */
+  public ExecutorService executorService;
+  /**
+   * list of blocks in which query will be executed
+   */
+  protected List<AbstractIndex> dataBlocks;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/VectorDetailQueryExecutor.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/VectorDetailQueryExecutor.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/VectorDetailQueryExecutor.java
new file mode 100644
index 0000000..cfea321
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/VectorDetailQueryExecutor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.executor.impl;
+
+import java.io.IOException;
+import java.util.List;
+
+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.iterator.VectorDetailQueryResultIterator;
+
+/**
+ * Below class will be used to execute the detail query and returns columnar 
vectors.
+ */
+public class VectorDetailQueryExecutor extends AbstractQueryExecutor<Object> {
+
+  @Override
+  public CarbonIterator<Object> execute(QueryModel queryModel)
+      throws QueryExecutionException, IOException {
+    List<BlockExecutionInfo> blockExecutionInfoList = 
getBlockExecutionInfos(queryModel);
+    this.queryIterator = new VectorDetailQueryResultIterator(
+        blockExecutionInfoList,
+        queryModel,
+        queryProperties.executorService
+    );
+    return this.queryIterator;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/AggregatorInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/AggregatorInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/AggregatorInfo.java
new file mode 100644
index 0000000..87ec07d
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/AggregatorInfo.java
@@ -0,0 +1,105 @@
+/*
+ * 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.executor.infos;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+
+/**
+ * Info class which store all the details
+ * which is required during aggregation
+ */
+public class AggregatorInfo {
+
+  /**
+   * selected query measure ordinal
+   * which will be used to read the measures chunk data
+   * this will be storing the index of the measure in measures chunk
+   */
+  private int[] measureOrdinals;
+
+  /**
+   * This parameter will be used to
+   * check whether particular measure is present
+   * in the table block, if not then its default value will be used
+   */
+  private boolean[] measureExists;
+
+  /**
+   * this default value will be used to when some measure is not present
+   * in the table block, in case of restructuring of the table if user is 
adding any
+   * measure then in older block that measure wont be present so for measure 
default value
+   * will be used to aggregate in the older table block query execution
+   */
+  private Object[] defaultValues;
+
+  /**
+   * Datatype of each measure;
+   */
+  private DataType[] measureDataTypes;
+
+  /**
+   * @return the measureOrdinal
+   */
+  public int[] getMeasureOrdinals() {
+    return measureOrdinals;
+  }
+
+  /**
+   * @param measureOrdinal the measureOrdinal to set
+   */
+  public void setMeasureOrdinals(int[] measureOrdinal) {
+    this.measureOrdinals = measureOrdinal;
+  }
+
+  /**
+   * @return the measureExists
+   */
+  public boolean[] getMeasureExists() {
+    return measureExists;
+  }
+
+  /**
+   * @param measureExists the measureExists to set
+   */
+  public void setMeasureExists(boolean[] measureExists) {
+    this.measureExists = measureExists;
+  }
+
+  /**
+   * @return the defaultValues
+   */
+  public Object[] getDefaultValues() {
+    return defaultValues;
+  }
+
+  /**
+   * @param defaultValues the defaultValues to set
+   */
+  public void setDefaultValues(Object[] defaultValues) {
+    this.defaultValues = defaultValues;
+  }
+
+  public DataType[] getMeasureDataTypes() {
+    return measureDataTypes;
+  }
+
+  public void setMeasureDataTypes(DataType[] measureDataTypes) {
+    this.measureDataTypes = measureDataTypes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
new file mode 100644
index 0000000..c8f4e7c
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
@@ -0,0 +1,604 @@
+/*
+ * 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.executor.infos;
+
+import java.util.Map;
+
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+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.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;
+
+/**
+ * Below class will have all the properties which needed during query execution
+ * for one block
+ */
+public class BlockExecutionInfo {
+
+  /**
+   * block on which query will be executed
+   */
+  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 aggregation during query
+   * execution
+   */
+  private AggregatorInfo aggregatorInfo;
+
+  /**
+   * this will be used to get the first tentative block from which query
+   * execution start, this will be useful in case of filter query to get the
+   * start block based on filter values
+   */
+  private IndexKey startKey;
+
+  /**
+   * this will be used to get the last tentative block till which scanning
+   * will be done, this will be useful in case of filter query to get the last
+   * block based on filter values
+   */
+  private IndexKey endKey;
+
+  private String blockId;
+
+  /**
+   * masked byte for block which will be used to unpack the fixed length key,
+   * this will be used for updating the older block key with new block key
+   * generator
+   */
+  private int[] maskedByteForBlock;
+
+  /**
+   * total number of dimension in block
+   */
+  private int totalNumberDimensionBlock;
+
+  /**
+   * total number of measure in block
+   */
+  private int totalNumberOfMeasureBlock;
+
+  /**
+   * will be used to read the dimension block from file
+   */
+  private int[][] allSelectedDimensionBlocksIndexes;
+
+  /**
+   * will be used to read the measure block from file
+   */
+  private int[][] allSelectedMeasureBlocksIndexes;
+
+  /**
+   * this will be used to update the older block fixed length keys with the
+   * new block fixed length key
+   */
+  private KeyStructureInfo keyStructureInfo;
+
+  /**
+   * first block from which query execution will start
+   */
+  private DataRefNode firstDataBlock;
+
+  /**
+   * number of block to be scanned in the query
+   */
+  private long numberOfBlockToScan;
+
+  /**
+   * key size of the fixed length dimension column
+   */
+  private int fixedLengthKeySize;
+
+  /**
+   * dictionary column block indexes based on query
+   */
+  private int[] dictionaryColumnBlockIndex;
+  /**
+   * 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;
+
+  /**
+   * each column value size
+   */
+  private int[] eachColumnValueSize;
+
+  /**
+   * column group block index in file to key structure info mapping
+   */
+  private Map<Integer, KeyStructureInfo> columnGroupToKeyStructureInfo;
+
+  /**
+   * mapping of dictionary dimension to its dictionary mapping which will be
+   * used to get the actual data from dictionary for aggregation, sorting
+   */
+  private Map<String, Dictionary> columnIdToDcitionaryMapping;
+
+  /**
+   * filter tree to execute the filter
+   */
+  private FilterExecuter filterExecuterTree;
+
+  /**
+   * whether it needs only raw byte records with out aggregation.
+   */
+  private boolean isRawRecordDetailQuery;
+
+  /**
+   * start index of blocklets
+   */
+  private int startBlockletIndex;
+
+  /**
+   * number of blocklet to be scanned
+   */
+  private int numberOfBlockletToScan;
+
+  /**
+   * complexParentIndexToQueryMap
+   */
+  private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
+
+  /**
+   * complex dimension parent block indexes;
+   */
+  private int[] complexColumnParentBlockIndexes;
+
+  /**
+   * @return the tableBlock
+   */
+  public AbstractIndex getDataBlock() {
+    return blockIndex;
+  }
+
+  /**
+   * list of dimension selected for in query
+   */
+  private QueryDimension[] queryDimensions;
+
+  /**
+   * list of measure selected in query
+   */
+  private QueryMeasure[] queryMeasures;
+
+  /**
+   * whether it needs to read data in vector/columnar format.
+   */
+  private boolean vectorBatchCollector;
+
+  /**
+   * absolute table identifier
+   */
+  private AbsoluteTableIdentifier absoluteTableIdentifier;
+
+  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
+    return absoluteTableIdentifier;
+  }
+
+  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier 
absoluteTableIdentifier) {
+    this.absoluteTableIdentifier = absoluteTableIdentifier;
+  }
+
+  /**
+   * @param blockIndex the tableBlock to set
+   */
+  public void setDataBlock(AbstractIndex blockIndex) {
+    this.blockIndex = blockIndex;
+  }
+
+  /**
+   * @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 AggregatorInfo getAggregatorInfo() {
+    return aggregatorInfo;
+  }
+
+  /**
+   * @param aggregatorInfo the aggregatorInfos to set
+   */
+  public void setAggregatorInfo(AggregatorInfo aggregatorInfo) {
+    this.aggregatorInfo = aggregatorInfo;
+  }
+
+  /**
+   * @return the startKey
+   */
+  public IndexKey getStartKey() {
+    return startKey;
+  }
+
+  /**
+   * @param startKey the startKey to set
+   */
+  public void setStartKey(IndexKey startKey) {
+    this.startKey = startKey;
+  }
+
+  /**
+   * @return the endKey
+   */
+  public IndexKey getEndKey() {
+    return endKey;
+  }
+
+  /**
+   * @param endKey the endKey to set
+   */
+  public void setEndKey(IndexKey endKey) {
+    this.endKey = endKey;
+  }
+
+  /**
+   * @return the maskedByteForBlock
+   */
+  public int[] getMaskedByteForBlock() {
+    return maskedByteForBlock;
+  }
+
+
+
+  /**
+   * @param maskedByteForBlock the maskedByteForBlock to set
+   */
+  public void setMaskedByteForBlock(int[] maskedByteForBlock) {
+    this.maskedByteForBlock = maskedByteForBlock;
+  }
+
+  /**
+   * @return the totalNumberDimensionBlock
+   */
+  public int getTotalNumberDimensionBlock() {
+    return totalNumberDimensionBlock;
+  }
+
+  /**
+   * @param totalNumberDimensionBlock the totalNumberDimensionBlock to set
+   */
+  public void setTotalNumberDimensionBlock(int totalNumberDimensionBlock) {
+    this.totalNumberDimensionBlock = totalNumberDimensionBlock;
+  }
+
+  /**
+   * @return the totalNumberOfMeasureBlock
+   */
+  public int getTotalNumberOfMeasureBlock() {
+    return totalNumberOfMeasureBlock;
+  }
+
+  /**
+   * @param totalNumberOfMeasureBlock the totalNumberOfMeasureBlock to set
+   */
+  public void setTotalNumberOfMeasureBlock(int totalNumberOfMeasureBlock) {
+    this.totalNumberOfMeasureBlock = totalNumberOfMeasureBlock;
+  }
+
+  /**
+   * @return the allSelectedDimensionBlocksIndexes
+   */
+  public int[][] getAllSelectedDimensionBlocksIndexes() {
+    return allSelectedDimensionBlocksIndexes;
+  }
+
+  /**
+   * @param allSelectedDimensionBlocksIndexes the 
allSelectedDimensionBlocksIndexes to set
+   */
+  public void setAllSelectedDimensionBlocksIndexes(int[][] 
allSelectedDimensionBlocksIndexes) {
+    this.allSelectedDimensionBlocksIndexes = allSelectedDimensionBlocksIndexes;
+  }
+
+  /**
+   * @return the allSelectedMeasureBlocksIndexes
+   */
+  public int[][] getAllSelectedMeasureBlocksIndexes() {
+    return allSelectedMeasureBlocksIndexes;
+  }
+
+  /**
+   * @param allSelectedMeasureBlocksIndexes the 
allSelectedMeasureBlocksIndexes to set
+   */
+  public void setAllSelectedMeasureBlocksIndexes(int[][] 
allSelectedMeasureBlocksIndexes) {
+    this.allSelectedMeasureBlocksIndexes = allSelectedMeasureBlocksIndexes;
+  }
+
+  /**
+   * @return the restructureInfos
+   */
+  public KeyStructureInfo getKeyStructureInfo() {
+    return keyStructureInfo;
+  }
+
+  /**
+   * @param keyStructureInfo the restructureInfos to set
+   */
+  public void setKeyStructureInfo(KeyStructureInfo keyStructureInfo) {
+    this.keyStructureInfo = keyStructureInfo;
+  }
+
+  /**
+   * @return the firstDataBlock
+   */
+  public DataRefNode getFirstDataBlock() {
+    return firstDataBlock;
+  }
+
+  /**
+   * @param firstDataBlock the firstDataBlock to set
+   */
+  public void setFirstDataBlock(DataRefNode firstDataBlock) {
+    this.firstDataBlock = firstDataBlock;
+  }
+
+  /**
+   * @return the numberOfBlockToScan
+   */
+  public long getNumberOfBlockToScan() {
+    return numberOfBlockToScan;
+  }
+
+  /**
+   * @param numberOfBlockToScan the numberOfBlockToScan to set
+   */
+  public void setNumberOfBlockToScan(long numberOfBlockToScan) {
+    this.numberOfBlockToScan = numberOfBlockToScan;
+  }
+
+  /**
+   * @return the fixedLengthKeySize
+   */
+  public int getFixedLengthKeySize() {
+    return fixedLengthKeySize;
+  }
+
+  /**
+   * @param fixedLengthKeySize the fixedLengthKeySize to set
+   */
+  public void setFixedLengthKeySize(int fixedLengthKeySize) {
+    this.fixedLengthKeySize = fixedLengthKeySize;
+  }
+
+  /**
+   * @return the filterEvaluatorTree
+   */
+  public FilterExecuter getFilterExecuterTree() {
+    return filterExecuterTree;
+  }
+
+  /**
+   * @param filterExecuterTree the filterEvaluatorTree to set
+   */
+  public void setFilterExecuterTree(FilterExecuter filterExecuterTree) {
+    this.filterExecuterTree = filterExecuterTree;
+  }
+
+  /**
+   * @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() {
+    return eachColumnValueSize;
+  }
+
+  /**
+   * @param eachColumnValueSize the eachColumnValueSize to set
+   */
+  public void setEachColumnValueSize(int[] eachColumnValueSize) {
+    this.eachColumnValueSize = eachColumnValueSize;
+  }
+
+  /**
+   * @return the dictionaryColumnBlockIndex
+   */
+  public int[] getDictionaryColumnBlockIndex() {
+    return dictionaryColumnBlockIndex;
+  }
+
+  /**
+   * @param dictionaryColumnBlockIndex the dictionaryColumnBlockIndex to set
+   */
+  public void setDictionaryColumnBlockIndex(int[] dictionaryColumnBlockIndex) {
+    this.dictionaryColumnBlockIndex = dictionaryColumnBlockIndex;
+  }
+
+  /**
+   * @return the noDictionaryBlockIndexes
+   */
+  public int[] getNoDictionaryBlockIndexes() {
+    return noDictionaryBlockIndexes;
+  }
+
+  /**
+   * @param noDictionaryBlockIndexes the noDictionaryBlockIndexes to set
+   */
+  public void setNoDictionaryBlockIndexes(int[] noDictionaryBlockIndexes) {
+    this.noDictionaryBlockIndexes = noDictionaryBlockIndexes;
+  }
+
+  /**
+   * @return the columnGroupToKeyStructureInfo
+   */
+  public Map<Integer, KeyStructureInfo> getColumnGroupToKeyStructureInfo() {
+    return columnGroupToKeyStructureInfo;
+  }
+
+  /**
+   * @param columnGroupToKeyStructureInfo the columnGroupToKeyStructureInfo to 
set
+   */
+  public void setColumnGroupToKeyStructureInfo(
+      Map<Integer, KeyStructureInfo> columnGroupToKeyStructureInfo) {
+    this.columnGroupToKeyStructureInfo = columnGroupToKeyStructureInfo;
+  }
+
+  /**
+   * @param columnIdToDcitionaryMapping the columnIdToDcitionaryMapping to set
+   */
+  public void setColumnIdToDcitionaryMapping(Map<String, Dictionary> 
columnIdToDcitionaryMapping) {
+    this.columnIdToDcitionaryMapping = columnIdToDcitionaryMapping;
+  }
+
+  public boolean isRawRecordDetailQuery() {
+    return isRawRecordDetailQuery;
+  }
+
+  public void setRawRecordDetailQuery(boolean rawRecordDetailQuery) {
+    isRawRecordDetailQuery = rawRecordDetailQuery;
+  }
+
+  /**
+   * @return the complexParentIndexToQueryMap
+   */
+  public Map<Integer, GenericQueryType> getComlexDimensionInfoMap() {
+    return complexParentIndexToQueryMap;
+  }
+
+  /**
+   * @param complexDimensionInfoMap the complexParentIndexToQueryMap to set
+   */
+  public void setComplexDimensionInfoMap(Map<Integer, GenericQueryType> 
complexDimensionInfoMap) {
+    this.complexParentIndexToQueryMap = complexDimensionInfoMap;
+  }
+
+  /**
+   * @return the complexColumnParentBlockIndexes
+   */
+  public int[] getComplexColumnParentBlockIndexes() {
+    return complexColumnParentBlockIndexes;
+  }
+
+  /**
+   * @param complexColumnParentBlockIndexes the 
complexColumnParentBlockIndexes to set
+   */
+  public void setComplexColumnParentBlockIndexes(int[] 
complexColumnParentBlockIndexes) {
+    this.complexColumnParentBlockIndexes = complexColumnParentBlockIndexes;
+  }
+
+  public QueryDimension[] getQueryDimensions() {
+    return queryDimensions;
+  }
+
+  public void setQueryDimensions(QueryDimension[] queryDimensions) {
+    this.queryDimensions = queryDimensions;
+  }
+
+  public QueryMeasure[] getQueryMeasures() {
+    return queryMeasures;
+  }
+
+  public void setQueryMeasures(QueryMeasure[] queryMeasures) {
+    this.queryMeasures = queryMeasures;
+  }
+
+  /**
+   * The method to set the number of blocklets to be scanned
+   *
+   * @param numberOfBlockletToScan
+   */
+  public void setNumberOfBlockletToScan(int numberOfBlockletToScan) {
+    this.numberOfBlockletToScan = numberOfBlockletToScan;
+  }
+
+  /**
+   * get the no of blocklet  to be scanned
+   *
+   * @return
+   */
+  public int getNumberOfBlockletToScan() {
+    return numberOfBlockletToScan;
+  }
+
+  /**
+   * returns the blocklet index to be scanned
+   *
+   * @return
+   */
+  public int getStartBlockletIndex() {
+    return startBlockletIndex;
+  }
+
+  /**
+   * set the blocklet index to be scanned
+   *
+   * @param startBlockletIndex
+   */
+  public void setStartBlockletIndex(int startBlockletIndex) {
+    this.startBlockletIndex = startBlockletIndex;
+  }
+
+  public boolean isVectorBatchCollector() {
+    return vectorBatchCollector;
+  }
+
+  public void setVectorBatchCollector(boolean vectorBatchCollector) {
+    this.vectorBatchCollector = vectorBatchCollector;
+  }
+
+  public String getBlockId() {
+    return blockId;
+  }
+
+  public void setBlockId(String blockId) {
+    this.blockId = blockId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/KeyStructureInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/KeyStructureInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/KeyStructureInfo.java
new file mode 100644
index 0000000..ad45eb8
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/KeyStructureInfo.java
@@ -0,0 +1,100 @@
+/*
+ * 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.executor.infos;
+
+import org.apache.carbondata.core.keygenerator.KeyGenerator;
+
+/**
+ * Below class will store the structure of the key
+ * used during query execution
+ */
+public class KeyStructureInfo {
+
+  /**
+   * it's actually a latest key generator
+   * last table block as this key generator will be used to
+   * to update the mdkey of the older slice with the new slice
+   */
+  private KeyGenerator keyGenerator;
+
+  /**
+   * mask bytes ranges for the query
+   */
+  private int[] maskByteRanges;
+
+  /**
+   * max key for query execution
+   */
+  private byte[] maxKey;
+
+  /**
+   * dimension ordinals inside the column group
+   */
+  private int[] mdkeyQueryDimensionOrdinal;
+
+  /**
+   * @return the keyGenerator
+   */
+  public KeyGenerator getKeyGenerator() {
+    return keyGenerator;
+  }
+
+  /**
+   * @param keyGenerator the keyGenerator to set
+   */
+  public void setKeyGenerator(KeyGenerator keyGenerator) {
+    this.keyGenerator = keyGenerator;
+  }
+
+  /**
+   * @return the maskByteRanges
+   */
+  public int[] getMaskByteRanges() {
+    return maskByteRanges;
+  }
+
+  /**
+   * @param maskByteRanges the maskByteRanges to set
+   */
+  public void setMaskByteRanges(int[] maskByteRanges) {
+    this.maskByteRanges = maskByteRanges;
+  }
+
+  /**
+   * @return the maxKey
+   */
+  public byte[] getMaxKey() {
+    return maxKey;
+  }
+
+  /**
+   * @param maxKey the maxKey to set
+   */
+  public void setMaxKey(byte[] maxKey) {
+    this.maxKey = maxKey;
+  }
+
+  public int[] getMdkeyQueryDimensionOrdinal() {
+    return mdkeyQueryDimensionOrdinal;
+  }
+
+  public void setMdkeyQueryDimensionOrdinal(int[] mdkeyQueryDimensionOrdinal) {
+    this.mdkeyQueryDimensionOrdinal = mdkeyQueryDimensionOrdinal;
+  }
+}

Reply via email to