You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/08/02 01:11:57 UTC

[2/3] incubator-carbondata git commit: Refactored code to improve performance by using removing unnecessary conversion.

Refactored code to improve performance by using removing unnecessary conversion.

Refactored code

Fixed review comments

support create using select as for other formats

Fixed issues after merge


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

Branch: refs/heads/master
Commit: 9be7451ec7bb5561b1f142ebdd40d6048713760e
Parents: 645eafa
Author: ravipesala <ra...@gmail.com>
Authored: Thu Jul 21 20:50:33 2016 +0530
Committer: chenliang613 <ch...@apache.org>
Committed: Tue Aug 2 09:09:45 2016 +0800

----------------------------------------------------------------------
 .../chunk/DimensionColumnDataChunk.java         |  10 +
 .../impl/ColumnGroupDimensionDataChunk.java     |  22 +-
 .../impl/FixedLengthDimensionDataChunk.java     |  22 +-
 .../impl/VariableLengthDimensionDataChunk.java  |  18 +-
 .../core/keygenerator/KeyGenerator.java         |   9 +
 .../core/keygenerator/mdkey/Bits.java           |   4 +-
 .../mdkey/MultiDimKeyVarLengthGenerator.java    |  11 +-
 .../org/carbondata/core/util/CarbonUtil.java    |  42 +++-
 .../scan/collector/ScannedResultCollector.java  |  11 +-
 .../impl/AbstractScannedResultCollector.java    | 157 ++++++++++++++
 .../impl/DictionaryBasedResultCollector.java    | 130 +++++++++++
 .../impl/ListBasedResultCollector.java          | 216 -------------------
 .../collector/impl/RawBasedResultCollector.java |  67 ++++++
 .../scan/complextypes/PrimitiveQueryType.java   |  21 +-
 .../scan/executor/QueryExecutorFactory.java     |   7 +-
 .../executor/impl/AbstractQueryExecutor.java    |  37 +---
 .../scan/executor/impl/DetailQueryExecutor.java |   6 +-
 .../impl/DetailRawRecordQueryExecutor.java      |  24 ---
 .../scan/executor/infos/BlockExecutionInfo.java |  43 ++--
 .../scan/executor/infos/KeyStructureInfo.java   |  12 ++
 .../scan/executor/util/QueryUtil.java           |   5 +
 .../org/carbondata/scan/model/QueryModel.java   |  14 --
 .../processor/AbstractDataBlockIterator.java    |  18 +-
 .../processor/impl/DataBlockIteratorImpl.java   |  22 +-
 .../scan/result/AbstractScannedResult.java      |  53 +++++
 .../carbondata/scan/result/BatchRawResult.java  |  43 ----
 .../org/carbondata/scan/result/BatchResult.java |  31 ++-
 .../scan/result/ListBasedResultWrapper.java     |  57 -----
 .../result/impl/FilterQueryScannedResult.java   |  19 ++
 .../scan/result/impl/ListBasedResult.java       | 153 -------------
 .../impl/NonFilterQueryScannedResult.java       |  19 ++
 .../iterator/DetailQueryResultIterator.java     |  15 +-
 .../scan/result/iterator/RawResultIterator.java |  10 +-
 .../preparator/QueryResultPreparator.java       |  10 -
 .../impl/AbstractQueryResultPreparator.java     |  87 --------
 .../impl/DetailQueryResultPreparatorImpl.java   | 148 -------------
 .../impl/RawQueryResultPreparatorImpl.java      | 136 ------------
 .../hadoop/util/CarbonInputFormatUtil.java      |   1 -
 .../spark/merger/CarbonCompactionExecutor.java  |   7 +-
 .../org/apache/spark/sql/CarbonOperators.scala  | 120 +++++------
 .../spark/sql/optimizer/CarbonOptimizer.scala   |   4 +
 .../AllDataTypesTestCaseAggregate.scala         |  87 ++++----
 42 files changed, 789 insertions(+), 1139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
index 75c29d2..6a30bc7 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
@@ -36,6 +36,16 @@ public interface DimensionColumnDataChunk<T> {
   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);
+
+  /**
    * Below method to get  the data based in row id
    *
    * @param row id

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
index e85e17d..d8536a1 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
@@ -55,7 +55,7 @@ public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<b
    * @param data             data to filed
    * @param offset           offset from which data need to be filed
    * @param rowId            row id of the chunk
-   * @param keyStructureInfo define the structure of the key
+   * @param restructuringInfo define the structure of the key
    * @return how many bytes was copied
    */
   @Override public int fillChunkData(byte[] data, int offset, int rowId,
@@ -67,10 +67,22 @@ public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<b
   }
 
   /**
-   * Below method to get the data based in row id
+   * Converts to column dictionary integer value
+   */
+  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
+      KeyStructureInfo info) {
+    int start = rowId * chunkAttributes.getColumnValueSize();
+    long[] keyArray = info.getKeyGenerator().getKeyArray(dataChunk, start);
+    int[] ordinal = info.getMdkeyQueryDimensionOrdinal();
+    for (int i = 0; i < ordinal.length; i++) {
+      row[columnIndex++] = (int)keyArray[ordinal[i]];
+    }
+    return columnIndex;
+  }
+
+  /**
+   * Below method masks key
    *
-   * @param row id row id of the data
-   * @return chunk
    */
   public byte[] getMaskedKey(byte[] data, int offset, KeyStructureInfo info) {
     byte[] maskedKey = new byte[info.getMaskByteRanges().length];
@@ -86,7 +98,7 @@ public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<b
   /**
    * Below method to get the data based in row id
    *
-   * @param row id row id of the data
+   * @param rowId row id of the data
    * @return chunk
    */
   @Override public byte[] getChunkData(int rowId) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
index 08ae703..cbe48b2 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
@@ -54,7 +54,7 @@ public class FixedLengthDimensionDataChunk implements DimensionColumnDataChunk<b
    *
    * @param data             data to filed
    * @param offset           offset from which data need to be filed
-   * @param rowId            row id of the chunk
+   * @param index            row id of the chunk
    * @param keyStructureInfo define the structure of the key
    * @return how many bytes was copied
    */
@@ -69,9 +69,27 @@ public class FixedLengthDimensionDataChunk implements DimensionColumnDataChunk<b
   }
 
   /**
+   * Converts to column dictionary integer value
+   */
+  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
+      KeyStructureInfo restructuringInfo) {
+    if (chunkAttributes.getInvertedIndexes() != null) {
+      rowId = chunkAttributes.getInvertedIndexesReverse()[rowId];
+    }
+    int start = rowId * chunkAttributes.getColumnValueSize();
+    int dict = 0;
+    for (int i = start; i < start + chunkAttributes.getColumnValueSize(); i++) {
+      dict <<= 8;
+      dict ^= dataChunk[i] & 0xFF;
+    }
+    row[columnIndex] = dict;
+    return columnIndex + 1;
+  }
+
+  /**
    * Below method to get the data based in row id
    *
-   * @param row id row id of the data
+   * @param index row id of the data
    * @return chunk
    */
   @Override public byte[] getChunkData(int index) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
index 51306b5..0aa2406 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
@@ -39,8 +39,8 @@ public class VariableLengthDimensionDataChunk implements DimensionColumnDataChun
    *
    * @param data             data to filed
    * @param offset           offset from which data need to be filed
-   * @param rowId            row id of the chunk
-   * @param keyStructureInfo define the structure of the key
+   * @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,
@@ -51,9 +51,21 @@ public class VariableLengthDimensionDataChunk implements DimensionColumnDataChun
   }
 
   /**
+   * 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;
+  }
+
+  /**
    * Below method to get the data based in row id
    *
-   * @param row id row id of the data
+   * @param index row id of the data
    * @return chunk
    */
   @Override public byte[] getChunkData(int index) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/keygenerator/KeyGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/KeyGenerator.java b/core/src/main/java/org/carbondata/core/keygenerator/KeyGenerator.java
index 5652d08..f9016f8 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/KeyGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/KeyGenerator.java
@@ -58,6 +58,15 @@ public interface KeyGenerator extends Serializable, Comparator<byte[]> {
    * It gets array of keys out of single key aka byte array
    *
    * @param key
+   * @param offset
+   * @return array of keys.
+   */
+  long[] getKeyArray(byte[] key, int offset);
+
+  /**
+   * It gets array of keys out of single key aka byte array
+   *
+   * @param key
    * @param maskedByteRanges
    * @return array of keys
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/keygenerator/mdkey/Bits.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/mdkey/Bits.java b/core/src/main/java/org/carbondata/core/keygenerator/mdkey/Bits.java
index 2515f0f..2ce64ba 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/mdkey/Bits.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/mdkey/Bits.java
@@ -252,7 +252,7 @@ public class Bits implements Serializable {
     return getBytesVal(words);
   }
 
-  public long[] getKeyArray(byte[] key) {
+  public long[] getKeyArray(byte[] key, int offset) {
 
     int length = 8;
     int ls = byteSize;
@@ -269,7 +269,7 @@ public class Bits implements Serializable {
       }
       for (int j = ls; j < m; j++) {
         l <<= 8;
-        l ^= key[j] & 0xFF;
+        l ^= key[j + offset] & 0xFF;
       }
       words[i] = l;
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java b/core/src/main/java/org/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
index 1cdc847..1aa2b81 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
@@ -52,12 +52,17 @@ public class MultiDimKeyVarLengthGenerator extends AbstractKeyGenerator {
 
   @Override public long[] getKeyArray(byte[] key) {
 
-    return bits.getKeyArray(key);
+    return bits.getKeyArray(key, 0);
+  }
+
+  @Override public long[] getKeyArray(byte[] key, int offset) {
+
+    return bits.getKeyArray(key, offset);
   }
 
   @Override public long getKey(byte[] key, int index) {
 
-    return bits.getKeyArray(key)[index];
+    return bits.getKeyArray(key, 0)[index];
   }
 
   public int getKeySizeInBytes() {
@@ -68,7 +73,7 @@ public class MultiDimKeyVarLengthGenerator extends AbstractKeyGenerator {
     if (index < 0 || size == 0) {
       return null;
     }
-    long[] keys = bits.getKeyArray(key);
+    long[] keys = bits.getKeyArray(key, 0);
     long[] rtn = new long[size];
     System.arraycopy(keys, index, rtn, 0, size);
     return rtn;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 9db8f24..eb8a0fc 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -20,11 +20,21 @@
 
 package org.carbondata.core.util;
 
-import java.io.*;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.security.PrivilegedExceptionAction;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -55,6 +65,7 @@ import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.carbondata.core.metadata.ValueEncoderMeta;
+import org.carbondata.scan.model.QueryDimension;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -1054,6 +1065,33 @@ public final class CarbonUtil {
     }
   }
 
+  public static boolean[] getDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+    boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
+    for (int i = 0; i < queryDimensions.length; i++) {
+      dictionaryEncodingArray[i] =
+          queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY);
+    }
+    return dictionaryEncodingArray;
+  }
+
+  public static boolean[] getDirectDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+    boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
+    for (int i = 0; i < queryDimensions.length; i++) {
+      dictionaryEncodingArray[i] =
+          queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY);
+    }
+    return dictionaryEncodingArray;
+  }
+
+  public static boolean[] getComplexDataTypeArray(QueryDimension[] queryDimensions) {
+    boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
+    for (int i = 0; i < queryDimensions.length; i++) {
+      dictionaryEncodingArray[i] =
+          CarbonUtil.hasComplexDataType(queryDimensions[i].getDimension().getDataType());
+    }
+    return dictionaryEncodingArray;
+  }
+
   /**
    * Below method will be used to read the data file matadata
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java b/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java
index f214a9d..424bd86 100644
--- a/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java
+++ b/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java
@@ -18,8 +18,9 @@
  */
 package org.carbondata.scan.collector;
 
+import java.util.List;
+
 import org.carbondata.scan.result.AbstractScannedResult;
-import org.carbondata.scan.result.Result;
 
 /**
  * Interface which will be used to aggregate the scan result
@@ -32,12 +33,6 @@ public interface ScannedResultCollector {
    * @param scannedResult scanned result
    * @return how many records was aggregated
    */
-  int collectData(AbstractScannedResult scannedResult, int batchSize);
+  List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize);
 
-  /**
-   * Below method will be used to get the aggregated result
-   *
-   * @return
-   */
-  Result getCollectedResult();
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/collector/impl/AbstractScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/collector/impl/AbstractScannedResultCollector.java b/core/src/main/java/org/carbondata/scan/collector/impl/AbstractScannedResultCollector.java
new file mode 100644
index 0000000..81d7b6a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/collector/impl/AbstractScannedResultCollector.java
@@ -0,0 +1,157 @@
+/*
+ * 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.carbondata.scan.collector.impl;
+
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.core.util.DataTypeUtil;
+import org.carbondata.scan.collector.ScannedResultCollector;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.util.QueryUtil;
+import org.carbondata.scan.result.AbstractScannedResult;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * It is not a collector it is just a scanned result holder.
+ */
+public abstract class AbstractScannedResultCollector implements ScannedResultCollector {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractScannedResultCollector.class.getName());
+
+  /**
+   * restructuring info
+   */
+  private KeyStructureInfo restructureInfos;
+
+  /**
+   * table block execution infos
+   */
+  protected BlockExecutionInfo tableBlockExecutionInfos;
+
+  /**
+   * Measure ordinals
+   */
+  protected int[] measuresOrdinal;
+
+  /**
+   * to check whether measure exists in current table block or not this to
+   * handle restructuring scenario
+   */
+  protected boolean[] isMeasureExistsInCurrentBlock;
+
+  /**
+   * default value of the measures in case of restructuring some measure wont
+   * be present in the table so in that default value will be used to
+   * aggregate the data for that measure columns
+   */
+  private Object[] measureDefaultValue;
+
+  /**
+   * measure datatypes.
+   */
+  protected DataType[] measureDatatypes;
+
+  public AbstractScannedResultCollector(BlockExecutionInfo blockExecutionInfos) {
+    this.tableBlockExecutionInfos = blockExecutionInfos;
+    restructureInfos = blockExecutionInfos.getKeyStructureInfo();
+    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
+    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
+    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
+    this.measureDatatypes = tableBlockExecutionInfos.getAggregatorInfo().getMeasureDataTypes();
+  }
+
+  protected void fillMeasureData(Object[] msrValues, int offset,
+      AbstractScannedResult scannedResult) {
+    for (short i = 0; i < measuresOrdinal.length; i++) {
+      // if measure exists is block then pass measure column
+      // data chunk to the collector
+      if (isMeasureExistsInCurrentBlock[i]) {
+        msrValues[i + offset] = getMeasureData(scannedResult.getMeasureChunk(measuresOrdinal[i]),
+            scannedResult.getCurrenrRowId(), measureDatatypes[i]);
+      } else {
+        // if not then get the default value and use that value in aggregation
+        msrValues[i + offset] = measureDefaultValue[i];
+      }
+    }
+  }
+
+  private Object getMeasureData(MeasureColumnDataChunk dataChunk, int index, DataType dataType) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      Object msrVal;
+      switch (dataType) {
+        case INT:
+        case LONG:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+          break;
+        case DECIMAL:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+          break;
+        default:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+      }
+      return DataTypeUtil.getMeasureDataBasedOnDataType(msrVal, dataType);
+    }
+    return null;
+  }
+
+  /**
+   * Below method will used to get the result
+   */
+  protected void updateData(List<Object[]> listBasedResult) {
+    if (tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
+      updateKeyWithLatestBlockKeygenerator(listBasedResult);
+    }
+  }
+
+  /**
+   * Below method will be used to update the fixed length key with the
+   * latest block key generator
+   *
+   * @return updated block
+   */
+  private void updateKeyWithLatestBlockKeygenerator(List<Object[]> listBasedResult) {
+    try {
+      long[] data = null;
+      ByteArrayWrapper key = null;
+      for (int i = 0; i < listBasedResult.size(); i++) {
+        // get the key
+        key = (ByteArrayWrapper)listBasedResult.get(i)[0];
+        // unpack the key with table block key generator
+        data = tableBlockExecutionInfos.getBlockKeyGenerator()
+            .getKeyArray(key.getDictionaryKey(), tableBlockExecutionInfos.getMaskedByteForBlock());
+        // packed the key with latest block key generator
+        // and generate the masked key for that key
+        key.setDictionaryKey(QueryUtil
+            .getMaskedKey(restructureInfos.getKeyGenerator().generateKey(data),
+                restructureInfos.getMaxKey(), restructureInfos.getMaskByteRanges(),
+                restructureInfos.getMaskByteRanges().length));
+      }
+    } catch (KeyGenException e) {
+      LOGGER.error(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/collector/impl/DictionaryBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/collector/impl/DictionaryBasedResultCollector.java b/core/src/main/java/org/carbondata/scan/collector/impl/DictionaryBasedResultCollector.java
new file mode 100644
index 0000000..89a7d9d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/collector/impl/DictionaryBasedResultCollector.java
@@ -0,0 +1,130 @@
+/*
+ * 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.carbondata.scan.collector.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.core.util.DataTypeUtil;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.filter.GenericQueryType;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.result.AbstractScannedResult;
+
+/**
+ * It is not a collector it is just a scanned result holder.
+ */
+public class DictionaryBasedResultCollector extends AbstractScannedResultCollector {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DictionaryBasedResultCollector.class.getName());
+
+  public DictionaryBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
+    super(blockExecutionInfos);
+  }
+
+  /**
+   * 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) {
+    List<Object[]> listBasedResult = new ArrayList<>(batchSize);
+    boolean isMsrsPresent = measureDatatypes.length > 0;
+    QueryDimension[] queryDimensions = tableBlockExecutionInfos.getQueryDimensions();
+    QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+    Map<Integer, GenericQueryType> comlexDimensionInfoMap =
+        tableBlockExecutionInfos.getComlexDimensionInfoMap();
+    boolean[] dictionaryEncodingArray = CarbonUtil.getDictionaryEncodingArray(queryDimensions);
+    boolean[] directDictionaryEncodingArray =
+        CarbonUtil.getDirectDictionaryEncodingArray(queryDimensions);
+    boolean[] complexDataTypeArray = CarbonUtil.getComplexDataTypeArray(queryDimensions);
+    int dimSize = queryDimensions.length;
+    boolean isDimensionsExist = dimSize > 0;
+    int[] order = new int[dimSize + queryMeasures.length];
+    for (int i = 0; i < dimSize; i++) {
+      order[i] = queryDimensions[i].getQueryOrder();
+    }
+    for (int i = 0; i < queryMeasures.length; i++) {
+      order[i + dimSize] = queryMeasures[i].getQueryOrder();
+    }
+    // scan the record and add to list
+    int rowCounter = 0;
+    int dictionaryColumnIndex = 0;
+    int noDictionaryColumnIndex = 0;
+    int complexTypeColumnIndex = 0;
+    int[] surrogateResult;
+    String[] noDictionaryKeys;
+    byte[][] complexTypeKeyArray;
+    while (scannedResult.hasNext() && rowCounter < batchSize) {
+      Object[] row = new Object[dimSize + queryMeasures.length];
+      if (isDimensionsExist) {
+        surrogateResult = scannedResult.getDictionaryKeyIntegerArray();
+        noDictionaryKeys = scannedResult.getNoDictionaryKeyStringArray();
+        complexTypeKeyArray = scannedResult.getComplexTypeKeyArray();
+        dictionaryColumnIndex = 0;
+        noDictionaryColumnIndex = 0;
+        complexTypeColumnIndex = 0;
+        for (int i = 0; i < dimSize; i++) {
+          if (!dictionaryEncodingArray[i]) {
+            row[order[i]] = DataTypeUtil
+                .getDataBasedOnDataType(noDictionaryKeys[noDictionaryColumnIndex++],
+                    queryDimensions[i].getDimension().getDataType());
+          } else if (directDictionaryEncodingArray[i]) {
+            DirectDictionaryGenerator directDictionaryGenerator =
+                DirectDictionaryKeyGeneratorFactory
+                    .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
+            if (directDictionaryGenerator != null) {
+              row[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
+                  surrogateResult[dictionaryColumnIndex++]);
+            }
+          } else if (complexDataTypeArray[i]) {
+            row[order[i]] = comlexDimensionInfoMap
+                .get(queryDimensions[i].getDimension().getOrdinal())
+                .getDataBasedOnDataTypeFromSurrogates(
+                    ByteBuffer.wrap(complexTypeKeyArray[complexTypeColumnIndex++]));
+          } else {
+            row[order[i]] = surrogateResult[dictionaryColumnIndex++];
+          }
+        }
+
+      } else {
+        scannedResult.incrementCounter();
+      }
+      if (isMsrsPresent) {
+        Object[] msrValues = new Object[measureDatatypes.length];
+        fillMeasureData(msrValues, 0, scannedResult);
+        for (int i = 0; i < msrValues.length; i++) {
+          row[order[i + dimSize]] = msrValues[i];
+        }
+      }
+      listBasedResult.add(row);
+      rowCounter++;
+    }
+    return listBasedResult;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java b/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java
deleted file mode 100644
index a199cad..0000000
--- a/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java
+++ /dev/null
@@ -1,216 +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.carbondata.scan.collector.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.scan.collector.ScannedResultCollector;
-import org.carbondata.scan.executor.infos.BlockExecutionInfo;
-import org.carbondata.scan.executor.infos.KeyStructureInfo;
-import org.carbondata.scan.executor.util.QueryUtil;
-import org.carbondata.scan.result.AbstractScannedResult;
-import org.carbondata.scan.result.ListBasedResultWrapper;
-import org.carbondata.scan.result.Result;
-import org.carbondata.scan.result.impl.ListBasedResult;
-import org.carbondata.scan.wrappers.ByteArrayWrapper;
-
-/**
- * It is not a collector it is just a scanned result holder.
- *
- */
-public class ListBasedResultCollector implements ScannedResultCollector {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ListBasedResultCollector.class.getName());
-
-  /**
-   * to keep a track of number of row processed to handle limit push down in
-   * case of detail query scenario
-   */
-  private int rowCounter;
-
-  /**
-   * dimension values list
-   */
-  private List<ListBasedResultWrapper> listBasedResult;
-
-  /**
-   * restructuring info
-   */
-  private KeyStructureInfo restructureInfos;
-
-  /**
-   * table block execution infos
-   */
-  private BlockExecutionInfo tableBlockExecutionInfos;
-
-  private int[] measuresOrdinal;
-
-  /**
-   * to check whether measure exists in current table block or not this to
-   * handle restructuring scenario
-   */
-  private boolean[] isMeasureExistsInCurrentBlock;
-
-  /**
-   * default value of the measures in case of restructuring some measure wont
-   * be present in the table so in that default value will be used to
-   * aggregate the data for that measure columns
-   */
-  private Object[] measureDefaultValue;
-
-  /**
-   * measure datatypes.
-   */
-  private DataType[] measureDatatypes;
-
-  public ListBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
-    this.tableBlockExecutionInfos = blockExecutionInfos;
-    restructureInfos = blockExecutionInfos.getKeyStructureInfo();
-    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
-    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
-    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
-    this.measureDatatypes = tableBlockExecutionInfos.getAggregatorInfo().getMeasureDataTypes();
-  }
-
-  @Override
-  /**
-   * 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
-   * @param scanned result
-   *
-   */
-  public int collectData(AbstractScannedResult scannedResult, int batchSize) {
-    this.listBasedResult =
-        new ArrayList<>(batchSize);
-    boolean isMsrsPresent = measureDatatypes.length > 0;
-    ByteArrayWrapper wrapper = null;
-    // scan the record and add to list
-    ListBasedResultWrapper resultWrapper;
-    int rowCounter = 0;
-    while (scannedResult.hasNext() && rowCounter < batchSize) {
-      resultWrapper = new ListBasedResultWrapper();
-      if(tableBlockExecutionInfos.isDimensionsExistInQuery()) {
-        wrapper = new ByteArrayWrapper();
-        wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
-        wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
-        wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
-        resultWrapper.setKey(wrapper);
-      } else {
-        scannedResult.incrementCounter();
-      }
-      if(isMsrsPresent) {
-        Object[] msrValues = new Object[measureDatatypes.length];
-        fillMeasureData(msrValues, scannedResult);
-        resultWrapper.setValue(msrValues);
-      }
-      listBasedResult.add(resultWrapper);
-      rowCounter++;
-    }
-    return rowCounter;
-  }
-
-  private void fillMeasureData(Object[] msrValues, AbstractScannedResult scannedResult) {
-    for (short i = 0; i < measuresOrdinal.length; i++) {
-      // if measure exists is block then pass measure column
-      // data chunk to the collector
-      if (isMeasureExistsInCurrentBlock[i]) {
-        msrValues[i] =
-            getMeasureData(scannedResult.getMeasureChunk(measuresOrdinal[i]),
-                scannedResult.getCurrenrRowId(),measureDatatypes[i]);
-      } else {
-        // if not then get the default value and use that value in aggregation
-        msrValues[i] = measureDefaultValue[i];
-      }
-    }
-  }
-
-  private Object getMeasureData(MeasureColumnDataChunk dataChunk, int index, DataType dataType) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      Object msrVal;
-      switch (dataType) {
-        case INT:
-        case LONG:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-          break;
-        case DECIMAL:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-          break;
-        default:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      }
-      return DataTypeUtil.getMeasureDataBasedOnDataType(msrVal, dataType);
-    }
-    return null;
-  }
-
-  /**
-   * Below method will used to get the result
-   */
-  @Override public Result getCollectedResult() {
-    Result<List<ListBasedResultWrapper>, Object> result = new ListBasedResult();
-    if (tableBlockExecutionInfos.isFixedKeyUpdateRequired() && tableBlockExecutionInfos
-        .isDimensionsExistInQuery()) {
-      updateKeyWithLatestBlockKeygenerator();
-      result.addScannedResult(listBasedResult);
-    } else {
-      result.addScannedResult(listBasedResult);
-    }
-    return result;
-  }
-
-
-
-  /**
-   * Below method will be used to update the fixed length key with the
-   * latest block key generator
-   *
-   * @return updated block
-   */
-  private void updateKeyWithLatestBlockKeygenerator() {
-    try {
-      long[] data = null;
-      ByteArrayWrapper key = null;
-      for (int i = 0; i < listBasedResult.size(); i++) {
-        // get the key
-        key = listBasedResult.get(i).getKey();
-        // unpack the key with table block key generator
-        data = tableBlockExecutionInfos.getBlockKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(), tableBlockExecutionInfos.getMaskedByteForBlock());
-        // packed the key with latest block key generator
-        // and generate the masked key for that key
-        key.setDictionaryKey(QueryUtil
-            .getMaskedKey(restructureInfos.getKeyGenerator().generateKey(data),
-                restructureInfos.getMaxKey(), restructureInfos.getMaskByteRanges(),
-                restructureInfos.getMaskByteRanges().length));
-        listBasedResult.get(i).setKey(key);
-      }
-    } catch (KeyGenException e) {
-      LOGGER.error(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/collector/impl/RawBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/collector/impl/RawBasedResultCollector.java b/core/src/main/java/org/carbondata/scan/collector/impl/RawBasedResultCollector.java
new file mode 100644
index 0000000..5bb0567
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/collector/impl/RawBasedResultCollector.java
@@ -0,0 +1,67 @@
+/*
+ * 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.carbondata.scan.collector.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.result.AbstractScannedResult;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * It is not a collector it is just a scanned result holder.
+ */
+public class RawBasedResultCollector extends AbstractScannedResultCollector {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(RawBasedResultCollector.class.getName());
+
+  public RawBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
+    super(blockExecutionInfos);
+  }
+
+  /**
+   * 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) {
+    List<Object[]> listBasedResult = new ArrayList<>(batchSize);
+    QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+    ByteArrayWrapper wrapper = null;
+    // scan the record and add to list
+    int rowCounter = 0;
+    while (scannedResult.hasNext() && rowCounter < batchSize) {
+      Object[] row = new Object[1 + queryMeasures.length];
+      wrapper = new ByteArrayWrapper();
+      wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
+      wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
+      wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
+      row[0] = wrapper;
+      fillMeasureData(row, 1, scannedResult);
+      listBasedResult.add(row);
+      rowCounter++;
+    }
+    updateData(listBasedResult);
+    return listBasedResult;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/complextypes/PrimitiveQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/complextypes/PrimitiveQueryType.java b/core/src/main/java/org/carbondata/scan/complextypes/PrimitiveQueryType.java
index 21a1d02..33a22c7 100644
--- a/core/src/main/java/org/carbondata/scan/complextypes/PrimitiveQueryType.java
+++ b/core/src/main/java/org/carbondata/scan/complextypes/PrimitiveQueryType.java
@@ -33,12 +33,7 @@ import org.carbondata.core.util.DataTypeUtil;
 import org.carbondata.scan.filter.GenericQueryType;
 import org.carbondata.scan.processor.BlocksChunkHolder;
 
-import org.apache.spark.sql.types.BooleanType;
-import org.apache.spark.sql.types.DataType;
-import org.apache.spark.sql.types.DoubleType;
-import org.apache.spark.sql.types.IntegerType;
-import org.apache.spark.sql.types.LongType;
-import org.apache.spark.sql.types.TimestampType;
+import org.apache.spark.sql.types.*;
 
 public class PrimitiveQueryType extends ComplexQueryType implements GenericQueryType {
 
@@ -135,17 +130,17 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
   @Override public DataType getSchemaType() {
     switch (dataType) {
       case INT:
-        return new IntegerType();
+        return IntegerType$.MODULE$;
       case DOUBLE:
-        return new DoubleType();
+        return DoubleType$.MODULE$;
       case LONG:
-        return new LongType();
+        return LongType$.MODULE$;
       case BOOLEAN:
-        return new BooleanType();
+        return BooleanType$.MODULE$;
       case TIMESTAMP:
-        return new TimestampType();
+        return TimestampType$.MODULE$;
       default:
-        return new IntegerType();
+        return IntegerType$.MODULE$;
     }
   }
 
@@ -165,7 +160,7 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
     byte[] data = new byte[keySize];
     surrogateData.get(data);
     Bits bit = new Bits(new int[]{keySize * 8});
-    int surrgateValue = (int)bit.getKeyArray(data)[0];
+    int surrgateValue = (int)bit.getKeyArray(data, 0)[0];
     Object actualData = null;
     if (isDirectDictionary) {
       DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java b/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java
index 724b8b6..7f7203c 100644
--- a/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java
+++ b/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java
@@ -19,7 +19,6 @@
 package org.carbondata.scan.executor;
 
 import org.carbondata.scan.executor.impl.DetailQueryExecutor;
-import org.carbondata.scan.executor.impl.DetailRawRecordQueryExecutor;
 import org.carbondata.scan.model.QueryModel;
 
 /**
@@ -29,10 +28,6 @@ import org.carbondata.scan.model.QueryModel;
 public class QueryExecutorFactory {
 
   public static QueryExecutor getQueryExecutor(QueryModel queryModel) {
-    if (queryModel.isForcedDetailRawQuery()) {
-      return new DetailRawRecordQueryExecutor();
-    } else {
-      return new DetailQueryExecutor();
-    }
+    return new DetailQueryExecutor();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java
index dd23119..519d4e6 100644
--- a/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java
@@ -48,7 +48,6 @@ import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.AggregatorInfo;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.executor.infos.KeyStructureInfo;
-import org.carbondata.scan.executor.infos.SortInfo;
 import org.carbondata.scan.executor.util.QueryUtil;
 import org.carbondata.scan.executor.util.RestructureUtil;
 import org.carbondata.scan.filter.FilterUtil;
@@ -229,7 +228,10 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
         QueryUtil.getMaskedByteRange(updatedQueryDimension, blockKeyGenerator);
     int[] maksedByte =
         QueryUtil.getMaskedByte(blockKeyGenerator.getKeySizeInBytes(), maskByteRangesForBlock);
-    blockExecutionInfo.setDimensionsExistInQuery(updatedQueryDimension.size() > 0);
+    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
@@ -373,37 +375,6 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
   }
 
   /**
-   * Below method will be used to get the sort information which will be
-   * required during sorting the data on dimension column
-   *
-   * @param queryModel query model
-   * @return Sort infos
-   * @throws QueryExecutionException if problem while
-   */
-  protected SortInfo getSortInfos(QueryModel queryModel) throws QueryExecutionException {
-
-    // get the masked by range for order by dimension
-    int[][] maskedByteRangeForSorting = QueryUtil
-        .getMaskedByteRangeForSorting(queryModel.getSortDimension(),
-            queryProperties.keyStructureInfo.getKeyGenerator(),
-            queryProperties.keyStructureInfo.getMaskByteRanges());
-    // get masked key for sorting
-    byte[][] maksedKeyForSorting = QueryUtil.getMaksedKeyForSorting(queryModel.getSortDimension(),
-        queryProperties.keyStructureInfo.getKeyGenerator(), maskedByteRangeForSorting,
-        queryProperties.keyStructureInfo.getMaskByteRanges());
-    // fill sort dimension indexes
-    queryProperties.sortDimIndexes = QueryUtil
-        .getSortDimensionIndexes(queryModel.getSortDimension(), queryModel.getQueryDimension());
-    SortInfo sortInfos = new SortInfo();
-    sortInfos.setDimensionMaskKeyForSorting(maksedKeyForSorting);
-    sortInfos.setDimensionSortOrder(queryModel.getSortOrder());
-    sortInfos.setMaskedByteRangeForSorting(maskedByteRangeForSorting);
-    sortInfos.setSortDimensionIndex(queryProperties.sortDimIndexes);
-    sortInfos.setSortDimension(queryModel.getSortDimension());
-    return sortInfos;
-  }
-
-  /**
    * Below method will be used to get the aggrgator info for the query
    *
    * @param queryModel query model

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
index edb7675..30eb2d2 100644
--- a/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
@@ -24,9 +24,7 @@ import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;
-import org.carbondata.scan.result.iterator.ChunkRowIterator;
 import org.carbondata.scan.result.iterator.DetailQueryResultIterator;
-import org.carbondata.scan.result.preparator.impl.DetailQueryResultPreparatorImpl;
 
 /**
  * Below class will be used to execute the detail query
@@ -38,9 +36,7 @@ public class DetailQueryExecutor extends AbstractQueryExecutor {
   @Override public CarbonIterator<Object[]> execute(QueryModel queryModel)
       throws QueryExecutionException {
     List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    return new ChunkRowIterator(
-        new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
-            new DetailQueryResultPreparatorImpl(queryProperties, queryModel)));
+    return new DetailQueryResultIterator(blockExecutionInfoList, queryModel);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
deleted file mode 100644
index 8659e82..0000000
--- a/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
+++ /dev/null
@@ -1,24 +0,0 @@
-package org.carbondata.scan.executor.impl;
-
-import java.util.List;
-
-import org.carbondata.common.CarbonIterator;
-import org.carbondata.scan.executor.exception.QueryExecutionException;
-import org.carbondata.scan.executor.infos.BlockExecutionInfo;
-import org.carbondata.scan.model.QueryModel;
-import org.carbondata.scan.result.BatchResult;
-import org.carbondata.scan.result.iterator.DetailQueryResultIterator;
-import org.carbondata.scan.result.preparator.impl.RawQueryResultPreparatorImpl;
-
-/**
- * Executor for raw records, it does not parse to actual data
- */
-public class DetailRawRecordQueryExecutor extends AbstractQueryExecutor<BatchResult> {
-
-  @Override public CarbonIterator<BatchResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    return new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
-        new RawQueryResultPreparatorImpl(queryProperties, queryModel));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java
index 7b62681..883357f 100644
--- a/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java
@@ -29,6 +29,8 @@ import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.scan.filter.GenericQueryType;
 import org.carbondata.scan.filter.executer.FilterExecuter;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
 
 /**
  * Below class will have all the properties which needed during query execution
@@ -198,11 +200,6 @@ public class BlockExecutionInfo {
   private boolean isRawRecordDetailQuery;
 
   /**
-   * whether dimensions exist in query.
-   */
-  private boolean isDimensionsExistInQuery;
-
-  /**
    * complexParentIndexToQueryMap
    */
   private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
@@ -225,6 +222,16 @@ public class BlockExecutionInfo {
   }
 
   /**
+   * list of dimension selected for in query
+   */
+  private QueryDimension[] queryDimensions;
+
+  /**
+   * list of measure selected in query
+   */
+  private QueryMeasure[] queryMeasures;
+
+  /**
    * @param blockIndex the tableBlock to set
    */
   public void setDataBlock(AbstractIndex blockIndex) {
@@ -336,6 +343,8 @@ public class BlockExecutionInfo {
     return maskedByteForBlock;
   }
 
+
+
   /**
    * @param maskedByteForBlock the maskedByteForBlock to set
    */
@@ -618,14 +627,6 @@ public class BlockExecutionInfo {
     isRawRecordDetailQuery = rawRecordDetailQuery;
   }
 
-  public boolean isDimensionsExistInQuery() {
-    return isDimensionsExistInQuery;
-  }
-
-  public void setDimensionsExistInQuery(boolean dimensionsExistInQuery) {
-    isDimensionsExistInQuery = dimensionsExistInQuery;
-  }
-
   /**
    * @return the complexParentIndexToQueryMap
    */
@@ -661,4 +662,20 @@ public class BlockExecutionInfo {
   public void setStatisticsRecorder(QueryStatisticsRecorder statisticsRecorder) {
     this.statisticsRecorder = statisticsRecorder;
   }
+
+  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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java b/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
index e19a3e9..a595f33 100644
--- a/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
+++ b/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
@@ -49,6 +49,11 @@ public class KeyStructureInfo {
   private byte[] maxKey;
 
   /**
+   * dimension ordinals inside the column group
+   */
+  private int[] mdkeyQueryDimensionOrdinal;
+
+  /**
    * @return the keyGenerator
    */
   public KeyGenerator getKeyGenerator() {
@@ -104,4 +109,11 @@ public class KeyStructureInfo {
     this.maxKey = maxKey;
   }
 
+  public int[] getMdkeyQueryDimensionOrdinal() {
+    return mdkeyQueryDimensionOrdinal;
+  }
+
+  public void setMdkeyQueryDimensionOrdinal(int[] mdkeyQueryDimensionOrdinal) {
+    this.mdkeyQueryDimensionOrdinal = mdkeyQueryDimensionOrdinal;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
index 00e0f48..2b26ba0 100644
--- a/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
@@ -546,8 +546,11 @@ public class QueryUtil {
       // sort the ordinal
       List<Integer> ordinal = next.getValue();
       List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+      //Un sorted
+      List<Integer> mdKeyOrdinalForQuery = new ArrayList<Integer>();
       for (Integer ord : ordinal) {
         mdKeyOrdinal.add(segmentProperties.getColumnGroupMdKeyOrdinal(next.getKey(), ord));
+        mdKeyOrdinalForQuery.add(segmentProperties.getColumnGroupMdKeyOrdinal(next.getKey(), ord));
       }
       Collections.sort(mdKeyOrdinal);
       // get the masked byte range for column group
@@ -560,6 +563,8 @@ public class QueryUtil {
       restructureInfos.setMaskByteRanges(maskByteRanges);
       restructureInfos.setMaxKey(maxKey);
       restructureInfos.setMaskedBytes(maksedByte);
+      restructureInfos.setMdkeyQueryDimensionOrdinal(ArrayUtils
+          .toPrimitive(mdKeyOrdinalForQuery.toArray(new Integer[mdKeyOrdinalForQuery.size()])));
       rowGroupToItsRSInfo
           .put(segmentProperties.getDimensionOrdinalToBlockMapping().get(ordinal.get(0)),
               restructureInfos);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryModel.java b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
index e02c56e..e299e7b 100644
--- a/core/src/main/java/org/carbondata/scan/model/QueryModel.java
+++ b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
@@ -130,12 +130,6 @@ public class QueryModel implements Serializable {
    */
   private CarbonTable table;
 
-  /**
-   * This is used only whne [forcedDetailRawQuery = true]. By default forcedDetailRawQuery returns
-   * dictionary values. But user wants in detail raw bytes the user set this field to true.
-   */
-  private boolean rawBytesDetailQuery;
-
   private QueryStatisticsRecorder statisticsRecorder;
 
   public QueryModel() {
@@ -495,14 +489,6 @@ public class QueryModel implements Serializable {
     this.columnToDictionaryMapping = columnToDictionaryMapping;
   }
 
-  public boolean isRawBytesDetailQuery() {
-    return rawBytesDetailQuery;
-  }
-
-  public void setRawBytesDetailQuery(boolean rawBytesDetailQuery) {
-    this.rawBytesDetailQuery = rawBytesDetailQuery;
-  }
-
   public int getInMemoryRecordSize() {
     return inMemoryRecordSize;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
index e385bec..cdf4f07 100644
--- a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
+++ b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
@@ -18,17 +18,19 @@
  */
 package org.carbondata.scan.processor;
 
+import java.util.List;
+
 import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.DataRefNode;
 import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.scan.collector.ScannedResultCollector;
-import org.carbondata.scan.collector.impl.ListBasedResultCollector;
+import org.carbondata.scan.collector.impl.DictionaryBasedResultCollector;
+import org.carbondata.scan.collector.impl.RawBasedResultCollector;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.result.AbstractScannedResult;
-import org.carbondata.scan.result.Result;
 import org.carbondata.scan.scanner.BlockletScanner;
 import org.carbondata.scan.scanner.impl.FilterScanner;
 import org.carbondata.scan.scanner.impl.NonFilterScanner;
@@ -37,7 +39,7 @@ import org.carbondata.scan.scanner.impl.NonFilterScanner;
  * This abstract class provides a skeletal implementation of the
  * Block iterator.
  */
-public abstract class AbstractDataBlockIterator extends CarbonIterator<Result> {
+public abstract class AbstractDataBlockIterator extends CarbonIterator<List<Object[]>> {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName());
@@ -88,9 +90,13 @@ public abstract class AbstractDataBlockIterator extends CarbonIterator<Result> {
     } else {
       blockletScanner = new NonFilterScanner(blockExecutionInfo);
     }
-
-    this.scannerResultAggregator =
-        new ListBasedResultCollector(blockExecutionInfo);
+    if (blockExecutionInfo.isRawRecordDetailQuery()) {
+      this.scannerResultAggregator =
+          new RawBasedResultCollector(blockExecutionInfo);
+    } else {
+      this.scannerResultAggregator =
+          new DictionaryBasedResultCollector(blockExecutionInfo);
+    }
     this.batchSize = batchSize;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java b/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java
index 0c61947..ab9c665 100644
--- a/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java
+++ b/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java
@@ -18,10 +18,11 @@
  */
 package org.carbondata.scan.processor.impl;
 
+import java.util.List;
+
 import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.processor.AbstractDataBlockIterator;
-import org.carbondata.scan.result.Result;
 
 /**
  * Below class will be used to process the block for detail query
@@ -33,8 +34,8 @@ public class DataBlockIteratorImpl extends AbstractDataBlockIterator {
    *
    * @param blockExecutionInfo execution information
    */
-  public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo,
-      FileHolder fileReader, int batchSize) {
+  public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader,
+      int batchSize) {
     super(blockExecutionInfo, fileReader, batchSize);
   }
 
@@ -43,14 +44,15 @@ public class DataBlockIteratorImpl extends AbstractDataBlockIterator {
    *
    * @return Result of @batchSize
    */
-  public Result next() {
-    this.scannerResultAggregator.collectData(scannedResult, batchSize);
-    Result result = this.scannerResultAggregator.getCollectedResult();
-    while (result.size() < batchSize && hasNext()) {
-      this.scannerResultAggregator.collectData(scannedResult, batchSize-result.size());
-      result.merge(this.scannerResultAggregator.getCollectedResult());
+  public List<Object[]> next() {
+    List<Object[]> collectedResult =
+        this.scannerResultAggregator.collectData(scannedResult, batchSize);
+    while (collectedResult.size() < batchSize && hasNext()) {
+      List<Object[]> data = this.scannerResultAggregator
+          .collectData(scannedResult, batchSize - collectedResult.size());
+      collectedResult.addAll(data);
     }
-    return result;
+    return collectedResult;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java b/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java
index 9e7f007..d917c2a 100644
--- a/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java
+++ b/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java
@@ -93,6 +93,8 @@ public abstract class AbstractScannedResult {
    */
   private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
 
+  private int totalDimensionsSize;
+
   /**
    * parent block indexes
    */
@@ -105,6 +107,7 @@ public abstract class AbstractScannedResult {
     this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo();
     this.complexParentIndexToQueryMap = blockExecutionInfo.getComlexDimensionInfoMap();
     this.complexParentBlockIndexes = blockExecutionInfo.getComplexColumnParentBlockIndexes();
+    this.totalDimensionsSize = blockExecutionInfo.getQueryDimensions().length;
   }
 
   /**
@@ -156,6 +159,25 @@ public abstract class AbstractScannedResult {
   }
 
   /**
+   * 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 = dataChunks[dictionaryColumnBlockIndexes[i]]
+          .fillConvertedChunkData(rowId, column, completeKey,
+              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
+    }
+    rowCounter++;
+    return completeKey;
+  }
+
+  /**
    * Just increment the counter incase of query only on measures.
    */
   public void incrementCounter() {
@@ -193,6 +215,23 @@ public abstract class AbstractScannedResult {
   }
 
   /**
+   * 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(dataChunks[noDictionaryColumnBlockIndexes[i]].getChunkData(rowId));
+    }
+    return noDictionaryColumnsKeys;
+  }
+
+  /**
    * 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
    *
@@ -324,6 +363,12 @@ public abstract class AbstractScannedResult {
   public abstract byte[] getDictionaryKeyArray();
 
   /**
+   * @return dictionary key array for all the dictionary dimension in integer array forat
+   * selected in query
+   */
+  public abstract int[] getDictionaryKeyIntegerArray();
+
+  /**
    * Return the dimension data based on dimension ordinal
    *
    * @param dimensionOrdinal dimension ordinal
@@ -347,6 +392,14 @@ public abstract class AbstractScannedResult {
   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();
+
+  /**
    * Below method will be used to to check whether measure value
    * is null or for a measure
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java b/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java
deleted file mode 100644
index c13b0f7..0000000
--- a/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java
+++ /dev/null
@@ -1,43 +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.carbondata.scan.result;
-
-/**
- * Below class holds the query result of batches.
- */
-public class BatchRawResult extends BatchResult {
-
-  /**
-   * This method will return one row at a time based on the counter given.
-   * @param counter
-   * @return
-   */
-  public Object[] getRawRow(int counter) {
-    return rows[counter];
-  }
-
-  /**
-   * For getting the total size.
-   * @return
-   */
-  public int getSize() {
-    return rows.length;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/BatchResult.java b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
index c5e1182..456717e 100644
--- a/core/src/main/java/org/carbondata/scan/result/BatchResult.java
+++ b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
@@ -19,6 +19,8 @@
 
 package org.carbondata.scan.result;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.NoSuchElementException;
 
 import org.carbondata.common.CarbonIterator;
@@ -31,7 +33,7 @@ public class BatchResult extends CarbonIterator<Object[]> {
   /**
    * list of keys
    */
-  protected Object[][] rows;
+  protected List<Object[]> rows;
 
   /**
    * counter to check whether all the records are processed or not
@@ -39,7 +41,7 @@ public class BatchResult extends CarbonIterator<Object[]> {
   protected int counter;
 
   public BatchResult() {
-    this.rows = new Object[0][];
+    this.rows = new ArrayList<>();
   }
 
   /**
@@ -47,7 +49,7 @@ public class BatchResult extends CarbonIterator<Object[]> {
    *
    * @return
    */
-  public Object[][] getRows() {
+  public List<Object[]> getRows() {
     return rows;
   }
 
@@ -56,10 +58,27 @@ public class BatchResult extends CarbonIterator<Object[]> {
    *
    * @param rows
    */
-  public void setRows(Object[][] 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.
@@ -67,7 +86,7 @@ public class BatchResult extends CarbonIterator<Object[]> {
    * @return {@code true} if the iteration has more elements
    */
   @Override public boolean hasNext() {
-    return counter < rows.length;
+    return counter < rows.size();
   }
 
   /**
@@ -79,7 +98,7 @@ public class BatchResult extends CarbonIterator<Object[]> {
     if (!hasNext()) {
       throw new NoSuchElementException();
     }
-    Object[] row = rows[counter];
+    Object[] row = rows.get(counter);
     counter++;
     return row;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.java b/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.java
deleted file mode 100644
index f3085ce..0000000
--- a/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.java
+++ /dev/null
@@ -1,57 +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.carbondata.scan.result;
-
-import org.carbondata.scan.wrappers.ByteArrayWrapper;
-
-public class ListBasedResultWrapper {
-
-  private ByteArrayWrapper key;
-
-  private Object[] value;
-
-  /**
-   * @return the key
-   */
-  public ByteArrayWrapper getKey() {
-    return key;
-  }
-
-  /**
-   * @param key the key to set
-   */
-  public void setKey(ByteArrayWrapper key) {
-    this.key = key;
-  }
-
-  /**
-   * @return the value
-   */
-  public Object[] getValue() {
-    return value;
-  }
-
-  /**
-   * @param value the value to set
-   */
-  public void setValue(Object[] value) {
-    this.value = value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java b/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java
index 962d9a3..e519f9e 100644
--- a/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java
+++ b/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java
@@ -44,6 +44,15 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
   }
 
   /**
+   * @return dictionary key integer array for all the dictionary dimension
+   * selected in query
+   */
+  @Override public int[] getDictionaryKeyIntegerArray() {
+    ++currentRow;
+    return getDictionaryKeyIntegerArray(rowMapping[currentRow]);
+  }
+
+  /**
    * Below method will be used to get the complex type key array
    *
    * @return complex type key array
@@ -63,6 +72,16 @@ public class FilterQueryScannedResult 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(rowMapping[currentRow]);
+  }
+
+  /**
    * will return the current valid row id
    *
    * @return valid row id

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java b/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java
deleted file mode 100644
index 24ebf5b..0000000
--- a/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java
+++ /dev/null
@@ -1,153 +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.carbondata.scan.result.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.scan.result.ListBasedResultWrapper;
-import org.carbondata.scan.result.Result;
-import org.carbondata.scan.wrappers.ByteArrayWrapper;
-
-/**
- * Below class is a holder over list based result wrapper
- */
-public class ListBasedResult implements Result<List<ListBasedResultWrapper>, Object> {
-
-  /**
-   * current result list
-   */
-  private List<ListBasedResultWrapper> currentRowPointer;
-
-  /**
-   * all result list , this is required because if we merger all the scanned
-   * result from all the blocks in one list, that list creation will take more
-   * time as every time list will create a big array and then it will do copy
-   * the older element to new array, and creation of big array will also be a
-   * problem if memory is fragmented then jvm in to do defragmentation to
-   * create a big space, but if divide the data in multiple list than it avoid
-   * copy and defragmentation
-   */
-  private List<List<ListBasedResultWrapper>> allRowsResult;
-
-  /**
-   * counter to check how many result processed
-   */
-  private int totalRecordCounter = -1;
-
-  /**
-   * number of records
-   */
-  private int totalNumberOfRecords;
-
-  /**
-   * current counter of the record in list
-   */
-  private int listRecordCounter = -1;
-
-  /**
-   * current list counter
-   */
-  private int currentListCounter;
-
-  public ListBasedResult() {
-    currentRowPointer =
-        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    allRowsResult =
-        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * below method will be used to add the scan result
-   */
-  @Override public void addScannedResult(List<ListBasedResultWrapper> listBasedResult) {
-    this.currentRowPointer = listBasedResult;
-    totalNumberOfRecords = listBasedResult.size();
-    allRowsResult.add(listBasedResult);
-  }
-
-  /**
-   * Method to check more result is present
-   * or not
-   */
-  @Override public boolean hasNext() {
-    if (allRowsResult.size() == 0) {
-      return false;
-    }
-    // As we are storing data in list of list, below code is to check whether
-    // any more result is present
-    // in the result.
-    // first it will check list counter is zero if it is zero
-    // than it will check list counter to check how many list has been processed
-    // if more list are present and all the list of current list is processed
-    // than it will take a new list from all row result list
-    totalRecordCounter++;
-    listRecordCounter++;
-    if (listRecordCounter == 0 || (listRecordCounter >= currentRowPointer.size()
-        && currentListCounter < allRowsResult.size())) {
-      listRecordCounter = 0;
-      currentRowPointer = allRowsResult.get(currentListCounter);
-      currentListCounter++;
-    }
-    return totalRecordCounter < totalNumberOfRecords;
-  }
-
-  /**
-   * @return key
-   */
-  @Override public ByteArrayWrapper getKey() {
-    return currentRowPointer.get(listRecordCounter).getKey();
-  }
-
-  /**
-   * @return will return the value
-   */
-  @Override public Object[] getValue() {
-    return currentRowPointer.get(listRecordCounter).getValue();
-  }
-
-  /***
-   * below method will be used to merge the
-   * scanned result
-   *
-   * @param otherResult return to be merged
-   */
-  @Override public void merge(Result<List<ListBasedResultWrapper>, Object> otherResult) {
-    if (otherResult.size() > 0) {
-      totalNumberOfRecords += otherResult.size();
-      this.allRowsResult.add(otherResult.getResult());
-    }
-  }
-
-  /**
-   * Return the size of the result
-   */
-  @Override public int size() {
-    return totalNumberOfRecords;
-  }
-
-  /**
-   * @return the complete result
-   */
-  @Override public List<ListBasedResultWrapper> getResult() {
-    return currentRowPointer;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java b/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java
index 9782099..00b5563 100644
--- a/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java
+++ b/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java
@@ -26,6 +26,15 @@ public class NonFilterQueryScannedResult extends AbstractScannedResult {
   }
 
   /**
+   * @return dictionary key integer array for all the dictionary dimension
+   * selected in query
+   */
+  @Override public int[] getDictionaryKeyIntegerArray() {
+    ++currentRow;
+    return getDictionaryKeyIntegerArray(currentRow);
+  }
+
+  /**
    * Below method will be used to get the complex type key array
    *
    * @return complex type key array
@@ -45,6 +54,16 @@ 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/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java
index 4b2d777..0539cfa 100644
--- a/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java
@@ -29,8 +29,6 @@ import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;
 import org.carbondata.scan.result.BatchResult;
-import org.carbondata.scan.result.ListBasedResultWrapper;
-import org.carbondata.scan.result.preparator.QueryResultPreparator;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
@@ -39,19 +37,12 @@ import org.carbondata.scan.result.preparator.QueryResultPreparator;
  */
 public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator {
 
-  /**
-   * to prepare the result
-   */
-  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
-
   private ExecutorService execService = Executors.newFixedThreadPool(1);
 
   private Future<BatchResult> future;
 
-  public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
-      QueryResultPreparator queryResultPreparator) {
+  public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
     super(infos, queryModel);
-    this.queryResultPreparator = queryResultPreparator;
   }
 
   @Override public BatchResult next() {
@@ -81,7 +72,9 @@ public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator
   private Future<BatchResult> execute() {
     return execService.submit(new Callable<BatchResult>() {
       @Override public BatchResult call() throws QueryExecutionException {
-        return queryResultPreparator.prepareQueryResult(dataBlockIterator.next());
+        BatchResult batchResult = new BatchResult();
+        batchResult.setRows(dataBlockIterator.next());
+        return batchResult;
       }
     });
   }