carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbono...@apache.org
Subject [17/56] [abbrv] incubator-carbondata git commit: Refactored core package and fixed all testcases (#684)
Date Thu, 23 Jun 2016 14:16:05 GMT
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java
deleted file mode 100644
index d3af870..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java
+++ /dev/null
@@ -1,53 +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.query.carbon.executor.impl;
-
-import java.util.List;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.executor.internal.impl.InternalDetailWithOrderQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.RowResult;
-import org.carbondata.query.carbon.result.iterator.ChunkBasedResultIterator;
-import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
-
-/**
- * Below method will be used to execute the detail query with order by
- */
-public class DetailWithOrderByQueryExecutor extends AbstractQueryExecutor {
-
-  @Override public CarbonIterator<RowResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    // get the execution info
-    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    // in case of sorting we need to add sort information only for last block as
-    // all the previous block data will be updated based last block and after
-    // processing of all the block sorting will be applied
-    blockExecutionInfoList.get(blockExecutionInfoList.size() - 1)
-        .setSortInfo(getSortInfos(queryModel));
-    InternalQueryExecutor internalQueryExecutor = new InternalDetailWithOrderQueryExecutor();
-    return new ChunkRowIterator(new ChunkBasedResultIterator(
-        internalQueryExecutor.executeQuery(blockExecutionInfoList, null), queryProperties,
-        queryModel));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java
deleted file mode 100644
index 563201d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java
+++ /dev/null
@@ -1,37 +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.query.carbon.executor.impl;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.RowResult;
-
-/**
- * Below class will be used execute the function query
- * for example:count(1)
- */
-public class FunctionQueryExecutor extends CountStarQueryExecutor {
-
-  @Override public CarbonIterator<RowResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    queryProperties.isFunctionQuery = true;
-    return super.execute(queryModel);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
index 1a30120..ff42fbc 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
@@ -23,7 +23,7 @@ import java.util.Map;
 
 import org.carbondata.core.cache.dictionary.Dictionary;
 import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
 
 /**
@@ -66,13 +66,13 @@ public class QueryExecutorProperties {
   public byte[] sortDimIndexes;
 
   /**
-   * aggregator class selected for all aggregation function selected in query
-   */
-  public MeasureAggregator[] measureAggregators;
-
-  /**
    * this will hold the information about the dictionary dimension
    * which to
    */
   public Map<String, Dictionary> columnToDictionayMapping;
+
+  /**
+   * Measure datatypes
+   */
+  public DataType[] measureDataTypes;
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
index fa44437..c9e9732 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
@@ -18,7 +18,7 @@
  */
 package org.carbondata.query.carbon.executor.infos;
 
-import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
 
 /**
  * Info class which store all the details
@@ -27,13 +27,6 @@ import org.carbondata.query.aggregator.MeasureAggregator;
 public class AggregatorInfo {
 
   /**
-   * measure aggregator array instance which will be used to
-   * aggregate the aggregate function columns
-   * it can be any dimension or measure column
-   */
-  private MeasureAggregator[] measuresAggreagators;
-
-  /**
    * 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
@@ -72,18 +65,9 @@ public class AggregatorInfo {
   private int measureAggregatorStartIndex;
 
   /**
-   * @return the measuresAggreagators
-   */
-  public MeasureAggregator[] getMeasuresAggreagators() {
-    return measuresAggreagators;
-  }
-
-  /**
-   * @param measuresAggreagators the measuresAggreagators to set
+   * Datatype of each measure;
    */
-  public void setMeasuresAggreagators(MeasureAggregator[] measuresAggreagators) {
-    this.measuresAggreagators = measuresAggreagators;
-  }
+  private DataType[] measureDataTypes;
 
   /**
    * @return the measureOrdinal
@@ -154,4 +138,12 @@ public class AggregatorInfo {
   public void setMeasureAggregatorStartIndex(int measureAggregatorStartIndex) {
     this.measureAggregatorStartIndex = measureAggregatorStartIndex;
   }
+
+  public DataType[] getMeasureDataTypes() {
+    return measureDataTypes;
+  }
+
+  public void setMeasureDataTypes(DataType[] measureDataTypes) {
+    this.measureDataTypes = measureDataTypes;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
index 894be0d..7bed33d 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
@@ -18,7 +18,6 @@
  */
 package org.carbondata.query.carbon.executor.infos;
 
-import java.util.List;
 import java.util.Map;
 
 import org.carbondata.core.cache.dictionary.Dictionary;
@@ -27,9 +26,7 @@ import org.carbondata.core.carbon.datastore.IndexKey;
 import org.carbondata.core.carbon.datastore.block.AbstractIndex;
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
 import org.carbondata.query.carbon.merger.ScannedResultMerger;
-import org.carbondata.query.carbon.model.CustomAggregateExpression;
 import org.carbondata.query.filter.executer.FilterExecuter;
 
 /**
@@ -92,12 +89,6 @@ public class BlockExecutionInfo {
   private IndexKey endKey;
 
   /**
-   * holder of custom aggregation details which will be used to aggregate the
-   * custome function UDAF
-   */
-  private List<CustomAggregateExpression> customAggregateExpressions;
-
-  /**
    * 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
@@ -173,11 +164,6 @@ public class BlockExecutionInfo {
    * key generator used for generating the table block fixed length key
    */
   private KeyGenerator blockKeyGenerator;
-  /**
-   * dimension aggregator list which will be used to aggregate the dimension
-   * data
-   */
-  private List<DimensionDataAggregator> dimensionAggregator;
 
   /**
    * each column value size
@@ -216,6 +202,11 @@ public class BlockExecutionInfo {
   private boolean isRawRecordDetailQuery;
 
   /**
+   * whether dimensions exist in query.
+   */
+  private boolean isDimensionsExistInQuery;
+
+  /**
    * @return the tableBlock
    */
   public AbstractIndex getDataBlock() {
@@ -328,21 +319,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the customAggregateExpression
-   */
-  public List<CustomAggregateExpression> getCustomAggregateExpressions() {
-    return customAggregateExpressions;
-  }
-
-  /**
-   * @param customAggregateExpression the customAggregateExpression to set
-   */
-  public void setCustomAggregateExpressions(
-      List<CustomAggregateExpression> customAggregateExpressions) {
-    this.customAggregateExpressions = customAggregateExpressions;
-  }
-
-  /**
    * @return the maskedByteForBlock
    */
   public int[] getMaskedByteForBlock() {
@@ -539,20 +515,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the dimensionAggregator
-   */
-  public List<DimensionDataAggregator> getDimensionAggregator() {
-    return dimensionAggregator;
-  }
-
-  /**
-   * @param dimensionAggregator the dimensionAggregator to set
-   */
-  public void setDimensionAggregator(List<DimensionDataAggregator> dimensionAggregator) {
-    this.dimensionAggregator = dimensionAggregator;
-  }
-
-  /**
    * @return the eachColumnValueSize
    */
   public int[] getEachColumnValueSize() {
@@ -658,4 +620,12 @@ public class BlockExecutionInfo {
   public void setRawRecordDetailQuery(boolean rawRecordDetailQuery) {
     isRawRecordDetailQuery = rawRecordDetailQuery;
   }
+
+  public boolean isDimensionsExistInQuery() {
+    return isDimensionsExistInQuery;
+  }
+
+  public void setDimensionsExistInQuery(boolean dimensionsExistInQuery) {
+    isDimensionsExistInQuery = dimensionsExistInQuery;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
index 8a6268c..a8c2c3a 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
@@ -36,7 +36,6 @@ import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.merger.ScannedResultMerger;
-import org.carbondata.query.carbon.merger.impl.SortedScannedResultMerger;
 import org.carbondata.query.carbon.merger.impl.UnSortedScannedResultMerger;
 import org.carbondata.query.carbon.result.Result;
 
@@ -66,9 +65,9 @@ public abstract class InternalAbstractQueryExecutor implements InternalQueryExec
    * Below method will be used to used to execute the detail query
    * and it will return iterator over result
    *
-   * @param executionInfos block execution info which will have all the properties
+   * @param tableBlockExecutionInfosList block execution info which will have all the properties
    *                       required for query execution
-   * @param sliceIndexes   slice indexes to be executed in this case it w
+   * @param sliceIndex   slice indexes to be executed in this case it w
    * @return query result
    */
   @Override public CarbonIterator<Result> executeQuery(
@@ -80,12 +79,7 @@ public abstract class InternalAbstractQueryExecutor implements InternalQueryExec
         tableBlockExecutionInfosList.get(tableBlockExecutionInfosList.size() - 1);
     execService = Executors.newFixedThreadPool(numberOfCores);
     ScannedResultMerger scannedResultProcessor = null;
-    if (null != latestInfo.getSortInfo()
-        && latestInfo.getSortInfo().getSortDimensionIndex().length > 0) {
-      scannedResultProcessor = new SortedScannedResultMerger(latestInfo, numberOfCores);
-    } else {
-      scannedResultProcessor = new UnSortedScannedResultMerger(latestInfo, numberOfCores);
-    }
+    scannedResultProcessor = new UnSortedScannedResultMerger(latestInfo, numberOfCores);
     try {
       List<Future> listFutureObjects = new ArrayList<Future>();
       for (BlockExecutionInfo blockInfo : tableBlockExecutionInfosList) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java
deleted file mode 100644
index 4c2a1a3..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java
+++ /dev/null
@@ -1,44 +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.query.carbon.executor.internal.impl;
-
-import java.util.concurrent.Executors;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-/**
- * Below class will be used to execute the aggregated query
- */
-public class InternalAggregationQueryExecutor extends InternalAbstractQueryExecutor {
-
-  public InternalAggregationQueryExecutor() {
-
-    // get the number of core can be used to execute the query
-    try {
-      numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException e) {
-      numberOfCores = 1;
-    }
-    // create thread pool
-    execService = Executors.newFixedThreadPool(numberOfCores);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
deleted file mode 100644
index 14c336d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
+++ /dev/null
@@ -1,83 +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.query.carbon.executor.internal.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.count.CountAggregator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.impl.ListBasedResult;
-import org.carbondata.query.carbon.result.iterator.MemoryBasedResultIterator;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * Below class will be used to execute the count star query or any function query
- * like count(1) , in this case block scanning is not required.
- */
-public class InternalCountStartQueryExecutor implements InternalQueryExecutor {
-
-  /**
-   * data block available for query execution
-   */
-  private List<AbstractIndex> blockList;
-
-  public InternalCountStartQueryExecutor(List<AbstractIndex> blockList) {
-    this.blockList = blockList;
-  }
-
-  /**
-   * Method to execute the count start query
-   *
-   * @param block execution info
-   * @param slice indexes
-   */
-  public CarbonIterator<Result> executeQuery(List<BlockExecutionInfo> infos, int[] sliceIndex)
-      throws QueryExecutionException {
-    long count = 0;
-    // for each block get the total number of rows
-    for (AbstractIndex tableBlock : this.blockList) {
-      count += tableBlock.getTotalNumberOfRows();
-    }
-    // as this is a count start need to create counter star aggregator
-    MeasureAggregator[] countAgg = new MeasureAggregator[1];
-    countAgg[0] = new CountAggregator();
-    countAgg[0].setNewValue(count);
-
-    ListBasedResultWrapper resultWrapper = new ListBasedResultWrapper();
-    Result<List<ListBasedResultWrapper>> result = new ListBasedResult();
-    ByteArrayWrapper wrapper = new ByteArrayWrapper();
-    wrapper.setDictionaryKey(new byte[0]);
-    resultWrapper.setKey(wrapper);
-    resultWrapper.setValue(countAgg);
-    List<ListBasedResultWrapper> wrapperList = new ArrayList<ListBasedResultWrapper>(1);
-    wrapperList.add(resultWrapper);
-    result.addScannedResult(wrapperList);
-    // returning the iterator over the result
-    return new MemoryBasedResultIterator(result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
index c7bfa6b..866596f 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
@@ -70,7 +70,7 @@ public class InternalDetailQueryExecutor implements InternalQueryExecutor {
         recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
       }
     }
-    this.numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
+    numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
             CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
     if (numberOfCores == 0) {
@@ -87,10 +87,11 @@ public class InternalDetailQueryExecutor implements InternalQueryExecutor {
    * @param sliceIndexes   slice indexes to be executed
    * @return query result
    */
-  @Override public CarbonIterator<Result> executeQuery(List<BlockExecutionInfo> executionInfos,
+  @Override public CarbonIterator<Result> executeQuery(
+      List<BlockExecutionInfo> executionInfos,
       int[] sliceIndexes) throws QueryExecutionException {
     long startTime = System.currentTimeMillis();
-    QueryRunner task = null;
+    QueryRunner task;
     ScannedResultMerger scannedResultProcessor =
         new UnSortedScannedResultMerger(executionInfos.get(executionInfos.size() - 1),
             sliceIndexes.length);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java
deleted file mode 100644
index c20a265..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java
+++ /dev/null
@@ -1,74 +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.query.carbon.executor.internal.impl;
-
-import java.util.concurrent.Executors;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-/**
- * Below class will be used to execute the detail query with order by
- */
-public class InternalDetailWithOrderQueryExecutor extends InternalAbstractQueryExecutor {
-
-  /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(InternalDetailWithOrderQueryExecutor.class.getName());
-
-  public InternalDetailWithOrderQueryExecutor() {
-    // numer of cores
-    try {
-      numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException e) {
-      numberOfCores = 1;
-    }
-    // get the number of records which can be kept in memory
-    // in case of detail query with order by we cannot keep all the
-    // record, so based on number of records which can be kept
-    // update the number of cores, this is required,
-    // if number of cores is 4 and number of records in memory is 120000
-    // then only one blocklet can be data can be kept in memory
-    int recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
-    String defaultInMemoryRecordsSize =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.INMEMORY_REOCRD_SIZE);
-    if (null != defaultInMemoryRecordsSize) {
-      try {
-        recordSize = Integer.parseInt(defaultInMemoryRecordsSize);
-      } catch (NumberFormatException ne) {
-        LOGGER.error("Invalid inmemory records size. Using default value");
-        recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
-      }
-    }
-    this.numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    if (numberOfCores == 0) {
-      numberOfCores++;
-    }
-    execService = Executors.newFixedThreadPool(numberOfCores);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java
deleted file mode 100644
index 4fffaa0..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java
+++ /dev/null
@@ -1,36 +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.query.carbon.executor.internal.impl;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-
-/**
- * This is to handle function query like
- * count(1), as function query will be executed like
- * count start query so it is extending the count star executor
- */
-public class InternalFunctionQueryExecutor extends InternalCountStartQueryExecutor {
-
-  public InternalFunctionQueryExecutor(List<AbstractIndex> blockList) {
-    super(blockList);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
index bd5ab00..f341fa9 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
@@ -27,7 +27,6 @@ import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.processor.BlockProcessor;
-import org.carbondata.query.carbon.processor.impl.AggregateQueryBlockProcessor;
 import org.carbondata.query.carbon.processor.impl.DetailQueryBlockProcessor;
 
 /**
@@ -58,11 +57,7 @@ public class QueryRunner implements Callable<Void> {
     this.fileReader = FileFactory.getFileHolder(executionInfo.getFileType());
     // if detail query detail query processor will be used to process the
     // block
-    if (executionInfo.isDetailQuery() || executionInfo.isRawRecordDetailQuery()) {
-      dataBlockProcessor = new DetailQueryBlockProcessor(executionInfo, fileReader);
-    } else {
-      dataBlockProcessor = new AggregateQueryBlockProcessor(executionInfo, fileReader);
-    }
+    dataBlockProcessor = new DetailQueryBlockProcessor(executionInfo, fileReader);
   }
 
   @Override public Void call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
index 789f77e..516230d 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
@@ -25,7 +25,6 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -43,7 +42,6 @@ import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -51,15 +49,8 @@ import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.ColumnGroupDimensionsAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.DirectDictionaryDimensionAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.FixedLengthDimensionAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.VariableLengthDimensionAggregator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.model.CustomAggregateExpression;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
 import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QueryMeasure;
 import org.carbondata.query.carbon.model.QueryModel;
@@ -161,7 +152,6 @@ public class QueryUtil {
    *
    * @param queryDimensions dimension selected in query
    * @param generator       key generator
-   * @param allDimension    all dimension present in the table
    * @return max key for dimension
    * @throws KeyGenException if any problem while generating the key
    */
@@ -203,29 +193,6 @@ public class QueryUtil {
   }
 
   /**
-   * Below method will be used to extract the dimension and measure from the
-   * expression
-   *
-   * @param expressions aggregate expression
-   * @param dims        extracted dimensions
-   * @param msrs        extracted measures
-   */
-  public static void extractDimensionsAndMeasuresFromExpression(
-      List<CustomAggregateExpression> expressions, List<CarbonDimension> dims,
-      List<CarbonMeasure> msrs) {
-    for (CustomAggregateExpression expression : expressions) {
-      List<CarbonColumn> dimsFromExpr = expression.getReferredColumns();
-      for (CarbonColumn dimFromExpr : dimsFromExpr) {
-        if (!dimFromExpr.isDimesion()) {
-          msrs.add((CarbonMeasure) dimFromExpr);
-        } else {
-          dims.add((CarbonDimension) dimFromExpr);
-        }
-      }
-    }
-  }
-
-  /**
    * Below method will be used to get the dimension block index in file based
    * on query dimension
    *
@@ -235,17 +202,13 @@ public class QueryUtil {
    */
   public static int[] getDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
       Map<Integer, Integer> dimensionOrdinalToBlockMapping,
-      List<DimensionAggregatorInfo> dimAggInfo, List<CarbonDimension> customAggregationDimension) {
+      List<CarbonDimension> customAggregationDimension) {
     // using set as in row group columns will point to same block
     Set<Integer> dimensionBlockIndex = new HashSet<Integer>();
     for (int i = 0; i < queryDimensions.size(); i++) {
       dimensionBlockIndex.add(
           dimensionOrdinalToBlockMapping.get(queryDimensions.get(i).getDimension().getOrdinal()));
     }
-    for (int i = 0; i < dimAggInfo.size(); i++) {
-      dimensionBlockIndex
-          .add(dimensionOrdinalToBlockMapping.get(dimAggInfo.get(i).getDim().getOrdinal()));
-    }
     for (int i = 0; i < customAggregationDimension.size(); i++) {
       dimensionBlockIndex
           .add(dimensionOrdinalToBlockMapping.get(customAggregationDimension.get(i).getOrdinal()));
@@ -260,18 +223,12 @@ public class QueryUtil {
    *
    * @param queryDimensions            query dimension present in the query this will be used to
    *                                   convert the result from surrogate key to actual data
-   * @param dimAggInfo                 dimension present in the dimension aggregation
-   *                                   dictionary will be used to convert to actual data
-   *                                   for aggregation
-   * @param customAggregationDimension dimension which is present in the expression for aggregation
-   *                                   we need dictionary data
    * @param absoluteTableIdentifier    absolute table identifier
    * @return dimension unique id to its dictionary map
    * @throws QueryExecutionException
    */
   public static Map<String, Dictionary> getDimensionDictionaryDetail(
-      List<QueryDimension> queryDimensions, List<DimensionAggregatorInfo> dimAggInfo,
-      List<CustomAggregateExpression> customAggExpression,
+      List<QueryDimension> queryDimensions,
       AbsoluteTableIdentifier absoluteTableIdentifier) throws QueryExecutionException {
     // to store dimension unique column id list, this is required as
     // dimension can be present in
@@ -287,23 +244,6 @@ public class QueryUtil {
         dictionaryDimensionFromQuery.add(queryDimensions.get(i).getDimension().getColumnId());
       }
     }
-    for (int i = 0; i < dimAggInfo.size(); i++) {
-      List<Encoding> encodingList = dimAggInfo.get(i).getDim().getEncoder();
-      if (CarbonUtil.hasEncoding(encodingList, Encoding.DICTIONARY) && !CarbonUtil
-          .hasEncoding(encodingList, Encoding.DIRECT_DICTIONARY)) {
-        dictionaryDimensionFromQuery.add(dimAggInfo.get(i).getDim().getColumnId());
-      }
-    }
-    for (int i = 0; i < customAggExpression.size(); i++) {
-      List<CarbonColumn> referredColumns = customAggExpression.get(i).getReferredColumns();
-      for (CarbonColumn column : referredColumns) {
-        if (CarbonUtil.hasEncoding(column.getEncoder(), Encoding.DICTIONARY) && !CarbonUtil
-            .hasEncoding(column.getEncoder(), Encoding.DIRECT_DICTIONARY)) {
-          dictionaryDimensionFromQuery.add(column.getColumnId());
-        }
-      }
-
-    }
     // converting to list as api exposed needed list which i think
     // is not correct
     List<String> dictionaryColumnIdList =
@@ -672,127 +612,6 @@ public class QueryUtil {
   }
 
   /**
-   * Below method will be used to get the dimension data aggregator list,
-   * which will be used to aggregate the dimension data, aggregate function
-   * like count(dimension) or any other function on dimension will be handled
-   *
-   * @param dimensionAggInfoList           dimension aggregation from query model
-   * @param dimensionToBlockIndexMapping   dimension block to its index mapping this will be
-   *                                       used to read  the block from file
-   * @param columnGroupIdToKeyGeneratorMap column group it to its key generator which will
-   *                                       be used to unpack the row group columns
-   * @param columnUniqueIdToDictionaryMap  this will dictionary column to its dictionary mapping
-   * @return list dimension data aggregator objects
-   */
-
-  public static List<DimensionDataAggregator> getDimensionDataAggregatorList1(
-      List<DimensionAggregatorInfo> dimensionAggInfoList,
-      Map<Integer, Integer> dimensionToBlockIndexMapping,
-      Map<Integer, KeyGenerator> columnGroupIdToKeyGeneratorMap,
-      Map<String, Dictionary> columnUniqueIdToDictionaryMap) {
-
-    Map<Integer, List<DimensionAggregatorInfo>> arrangeDimensionAggregationInfo =
-        arrangeDimensionAggregationInfo(dimensionAggInfoList, dimensionToBlockIndexMapping);
-    List<DimensionDataAggregator> dimensionDataAggregators =
-        new ArrayList<DimensionDataAggregator>();
-    int aggregatorStartIndex = 0;
-
-    for (Entry<Integer, List<DimensionAggregatorInfo>> entry : arrangeDimensionAggregationInfo
-        .entrySet()) {
-      // if number of dimension aggregation is info is more than 2 than it
-      // is a column group dimension
-      // so only one aggregator instance will be created to handle the
-      // aggregation
-      // as in case of column group unpacking of the bit packed mdkey will
-      // be done
-      // only once
-      CarbonDimension dim = entry.getValue().get(0).getDim();
-      if (entry.getValue().size() > 1) {
-        // how many aggregator will be used for column group
-        int numberOfAggregatorForColumnGroup = 0;
-        List<Dictionary> dictionaryList = new ArrayList<Dictionary>();
-        // below code is to create a dictionary list of all the column
-        // group dimension present in the query
-        for (DimensionAggregatorInfo dimensionAggregatorInfo : entry.getValue()) {
-          dictionaryList.add(
-              columnUniqueIdToDictionaryMap.get(dimensionAggregatorInfo.getDim().getColumnId()));
-          numberOfAggregatorForColumnGroup += dimensionAggregatorInfo.getAggList().size();
-        }
-        dimensionDataAggregators.add(new ColumnGroupDimensionsAggregator(entry.getValue(),
-            columnGroupIdToKeyGeneratorMap.get(dim.columnGroupId()),
-            dimensionToBlockIndexMapping.get(dim.getOrdinal()), dictionaryList,
-            aggregatorStartIndex));
-        aggregatorStartIndex += numberOfAggregatorForColumnGroup;
-        continue;
-      } else {
-        if(CarbonUtil.hasEncoding(dim.getEncoder(), Encoding.DIRECT_DICTIONARY)){
-          dimensionDataAggregators.add(
-              new DirectDictionaryDimensionAggregator(entry.getValue().get(0),
-                  aggregatorStartIndex,
-                  dimensionToBlockIndexMapping.get(dim.getOrdinal())));
-        }
-        // if it is a dictionary column than create a fixed length
-        // aggeragtor
-        else if (CarbonUtil
-            .hasEncoding(dim.getEncoder(), Encoding.DICTIONARY)) {
-          dimensionDataAggregators.add(
-              new FixedLengthDimensionAggregator(entry.getValue().get(0), null,
-                  columnUniqueIdToDictionaryMap.get(dim.getColumnId()),
-                  aggregatorStartIndex,
-                  dimensionToBlockIndexMapping.get(dim.getOrdinal())));
-        } else {
-          // else for not dictionary column create a
-          // variable length aggregator
-          dimensionDataAggregators.add(
-              new VariableLengthDimensionAggregator(entry.getValue().get(0), null,
-                  aggregatorStartIndex,
-                  dimensionToBlockIndexMapping.get(dim.getOrdinal())));
-        }
-        aggregatorStartIndex += entry.getValue().get(0).getAggList().size();
-      }
-    }
-    return dimensionDataAggregators;
-  }
-
-  /**
-   * Below method will be used to group the dimension aggregation infos This
-   * grouping will be based on block index of the file to dimension mapping
-   * Basically it will group all the dimension aggregation of same column
-   * group dimension This is done to avoid the column group dimension mdkey
-   * unpacking multiple times. If all the dimension of column group is handled
-   * separately then unpacking of mdkey for same column group will be done
-   * multiple times and it will impact the query performance, so to avoid this
-   * if we group the dimension together and they will point same block in the
-   * physical file so reading will be done only once and unpacking of each row
-   * will be also done only once
-   *
-   * @param queryDimensionAggregationInfos      query dimension aggregation infos
-   * @param dimensionOrdinaltoBlockIndexMapping dimension to file block index mapping
-   * @return block index in file to list of dimension pointing to that block
-   * mapping
-   */
-  private static Map<Integer, List<DimensionAggregatorInfo>> arrangeDimensionAggregationInfo(
-      List<DimensionAggregatorInfo> queryDimensionAggregationInfos,
-      Map<Integer, Integer> dimensionOrdinaltoBlockIndexMapping) {
-    Map<Integer, List<DimensionAggregatorInfo>> groupedDimensionAggregationInfo =
-        new LinkedHashMap<Integer, List<DimensionAggregatorInfo>>();
-    for (DimensionAggregatorInfo queryDimensionAggregatorInfo : queryDimensionAggregationInfos) {
-      List<DimensionAggregatorInfo> list = groupedDimensionAggregationInfo.get(
-          dimensionOrdinaltoBlockIndexMapping
-              .get(queryDimensionAggregatorInfo.getDim().getOrdinal()));
-
-      if (null == list) {
-        list =
-            new ArrayList<DimensionAggregatorInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        groupedDimensionAggregationInfo.put(dimensionOrdinaltoBlockIndexMapping
-            .get(queryDimensionAggregatorInfo.getDim().getOrdinal()), list);
-      }
-      list.add(queryDimensionAggregatorInfo);
-    }
-    return groupedDimensionAggregationInfo;
-  }
-
-  /**
    * Below method will be used to fill block indexes of the query dimension
    * which will be used in creating a output row Here is method we are passing
    * two list which store the indexes one for dictionary column other for not
@@ -860,10 +679,6 @@ public class QueryUtil {
             .setMeasure(carbonTable.getMeasureByName(tableName, queryMeasure.getColumnName()));
       }
     }
-    // resolve dimension aggregation info
-    for (DimensionAggregatorInfo dimAggInfo : queryModel.getDimAggregationInfo()) {
-      dimAggInfo.setDim(carbonTable.getDimensionByName(tableName, dimAggInfo.getColumnName()));
-    }
     //TODO need to handle expression
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java b/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
index 8b77e45..9358f42 100644
--- a/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
+++ b/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
@@ -33,7 +33,6 @@ import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.result.impl.ListBasedResult;
-import org.carbondata.query.carbon.result.impl.MapBasedResult;
 import org.carbondata.query.carbon.result.iterator.MemoryBasedResultIterator;
 
 /**
@@ -93,11 +92,7 @@ public abstract class AbstractScannedResultMerger implements ScannedResultMerger
    * for initializing the map based or list based result.
    */
   protected void initialiseResult() {
-    if (!blockExecutionInfo.isDetailQuery() && !blockExecutionInfo.isRawRecordDetailQuery()) {
-      mergedScannedResult = new MapBasedResult();
-    } else {
-      mergedScannedResult = new ListBasedResult();
-    }
+    mergedScannedResult = new ListBasedResult();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java b/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java
deleted file mode 100644
index 2ed565e..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java
+++ /dev/null
@@ -1,176 +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.query.carbon.merger.impl;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.executor.infos.SortInfo;
-import org.carbondata.query.carbon.merger.AbstractScannedResultMerger;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.comparator.FixedLengthKeyResultComparator;
-import org.carbondata.query.carbon.result.comparator.VariableLengthKeyResultComparator;
-import org.carbondata.query.carbon.result.iterator.MemoryBasedResultIterator;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-import org.apache.commons.collections.comparators.ComparatorChain;
-
-/**
- * Below class will be used to sort and merge the scanned result
- */
-public class SortedScannedResultMerger extends AbstractScannedResultMerger {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SortedScannedResultMerger.class.getName());
-
-  public SortedScannedResultMerger(BlockExecutionInfo blockExecutionInfo,
-      int maxNumberOfScannedresultList) {
-    super(blockExecutionInfo, maxNumberOfScannedresultList);
-  }
-
-  /**
-   * Below method will be used to get the comparator for sorting the
-   * result
-   *
-   * @param sortInfo sort info
-   * @return comparator
-   */
-  public static ComparatorChain getMergerChainComparator(SortInfo sortInfo) {
-    List<Comparator> compratorList =
-        new ArrayList<Comparator>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    int length = sortInfo.getSortDimension().size();
-    int noDictionaryIndex = 0;
-    for (int i = 0; i < length; i++) {
-      if (!CarbonUtil.hasEncoding(sortInfo.getSortDimension().get(i).getDimension().getEncoder(),
-          Encoding.DICTIONARY)) {
-        compratorList.add(new VariableLengthKeyResultComparator(sortInfo.getDimensionSortOrder()[i],
-            noDictionaryIndex++, sortInfo.getSortDimension().get(i).getDimension().getDataType()));
-      } else {
-        compratorList.add(
-            new FixedLengthKeyResultComparator(sortInfo.getMaskedByteRangeForSorting()[i],
-                sortInfo.getDimensionSortOrder()[i], sortInfo.getDimensionMaskKeyForSorting()[i]));
-      }
-    }
-    return new ComparatorChain(compratorList);
-  }
-
-  /**
-   * Below method will be used to get the final query
-   * return
-   *
-   * @return iterator over result
-   */
-  @Override public CarbonIterator<Result> getQueryResultIterator() throws QueryExecutionException {
-    execService.shutdown();
-    try {
-      execService.awaitTermination(1, TimeUnit.DAYS);
-    } catch (InterruptedException e1) {
-      LOGGER.error("Problem in thread termination" + e1.getMessage());
-    }
-    if (scannedResultList.size() > 0) {
-      mergeScannedResults(scannedResultList);
-      scannedResultList = null;
-    }
-    LOGGER.debug("Finished result merging from all slices");
-    sortResult();
-    return new MemoryBasedResultIterator(mergedScannedResult);
-  }
-
-  /**
-   * Below method will be used to sort the query result
-   * for dictionary dimension it will unpack the key array a
-   * and then it will get the sort index based on which new dictionary
-   * key will be created
-   *
-   * @throws QueryExecutionException
-   */
-  private void sortResult() throws QueryExecutionException {
-    List<ListBasedResultWrapper> result =
-        new ArrayList<ListBasedResultWrapper>(mergedScannedResult.size());
-    ListBasedResultWrapper wrapper = null;
-    SortInfo sortInfo = blockExecutionInfo.getSortInfo();
-    KeyStructureInfo keyStructureInfo = blockExecutionInfo.getKeyStructureInfo();
-    long[] keyArray = null;
-    try {
-      while (mergedScannedResult.hasNext()) {
-        wrapper = new ListBasedResultWrapper();
-        ByteArrayWrapper key = mergedScannedResult.getKey();
-        keyArray = keyStructureInfo.getKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(), keyStructureInfo.getMaskedBytes());
-        for (int i = 0; i < sortInfo.getSortDimension().size(); i++) {
-          if (CarbonUtil.hasEncoding(sortInfo.getSortDimension().get(i).getDimension().getEncoder(),
-              Encoding.DICTIONARY)) {
-            keyArray[sortInfo.getSortDimension().get(i).getDimension().getKeyOrdinal()] =
-                blockExecutionInfo.getColumnIdToDcitionaryMapping()
-                    .get(sortInfo.getSortDimension().get(i).getDimension().getColumnId())
-                    .getSortedIndex((int) keyArray[sortInfo.getSortDimension().get(i).getDimension()
-                        .getKeyOrdinal()]);
-          }
-        }
-        key.setDictionaryKey(getMaskedKey(keyStructureInfo.getKeyGenerator().generateKey(keyArray),
-            keyStructureInfo));
-        wrapper.setKey(key);
-        wrapper.setValue(mergedScannedResult.getValue());
-        result.add(wrapper);
-      }
-    } catch (KeyGenException e) {
-      throw new QueryExecutionException(e);
-    }
-    initialiseResult();
-    Collections.sort(result, getMergerChainComparator(sortInfo));
-    mergedScannedResult.addScannedResult(result);
-  }
-
-  /**
-   * Below method will be used to get the masked key
-   *
-   * @param data
-   * @return keyStructureInfo
-   */
-  private byte[] getMaskedKey(byte[] data, KeyStructureInfo keyStructureInfo) {
-    int keySize = blockExecutionInfo.getFixedLengthKeySize();
-    int[] actualMaskByteRanges = keyStructureInfo.getMaskByteRanges();
-    byte[] maxKey = keyStructureInfo.getMaxKey();
-    byte[] maskedKey = new byte[keySize];
-    int counter = 0;
-    int byteRange = 0;
-    for (int i = 0; i < keySize; i++) {
-      byteRange = actualMaskByteRanges[i];
-      maskedKey[counter++] = (byte) (data[byteRange] & maxKey[byteRange]);
-    }
-    return maskedKey;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java b/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
index dbff0c3..3684f4a 100644
--- a/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
+++ b/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
@@ -24,9 +24,7 @@ package org.carbondata.query.carbon.model;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.query.expression.Expression;
@@ -99,12 +97,6 @@ public class CarbonQueryPlan implements Serializable {
   private String outLocationPath;
 
   /**
-   * dimAggregatorInfoList
-   */
-  private Map<String, DimensionAggregatorInfo> dimAggregatorInfos =
-      new LinkedHashMap<String, DimensionAggregatorInfo>();
-
-  /**
    * isCountStarQuery
    */
   private boolean isCountStartQuery;
@@ -225,24 +217,6 @@ public class CarbonQueryPlan implements Serializable {
     this.outLocationPath = outLocationPath;
   }
 
-  public void addAggDimAggInfo(String columnName, String aggType, int queryOrder) {
-    DimensionAggregatorInfo dimensionAggregatorInfo = dimAggregatorInfos.get(columnName);
-    if (null == dimensionAggregatorInfo) {
-      dimensionAggregatorInfo = new DimensionAggregatorInfo();
-      dimensionAggregatorInfo.setColumnName(columnName);
-      List<Integer> queryOrderList= new ArrayList<Integer>();
-      queryOrderList.add(queryOrder);
-      List<String> aggTypeList= new ArrayList<String>();
-      aggTypeList.add(aggType);
-      dimensionAggregatorInfo.setOrderList(queryOrderList);
-      dimensionAggregatorInfo.setAggList(aggTypeList);
-      dimAggregatorInfos.put(columnName, dimensionAggregatorInfo);
-    } else {
-      dimensionAggregatorInfo.getOrderList().add(queryOrder);
-      dimensionAggregatorInfo.getAggList().add(aggType);
-    }
-  }
-
   public boolean isCountStarQuery() {
     return isCountStartQuery;
   }
@@ -251,10 +225,6 @@ public class CarbonQueryPlan implements Serializable {
     this.isCountStartQuery = isCountStartQuery;
   }
 
-  public Map<String, DimensionAggregatorInfo> getDimAggregatorInfos() {
-    return dimAggregatorInfos;
-  }
-
   public List<QueryDimension> getSortedDimemsions() {
     return sortedDimensions;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java b/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java
deleted file mode 100644
index 9890f7f..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.CustomMeasureAggregator;
-
-/**
- * Holds the information about expression present in the query
- */
-public class CustomAggregateExpression implements Serializable {
-
-  /**
-   * serialization version
-   */
-  private static final long serialVersionUID = 4831882661337567735L;
-
-  /**
-   * Identified and delegated from Spark Layer for UDAF in Carbon
-   */
-  private CustomMeasureAggregator aggregator;
-
-  /**
-   * Aggregate column name may not be a measure or dimension. Can be a column
-   * name given in query
-   */
-  private String name;
-
-  /**
-   * Columns used in the expression where column can be a dimension or a
-   * measure.
-   */
-  private List<CarbonColumn> referredColumns;
-
-  /**
-   * Actual expression in query to use in the comparison with other Aggregate
-   * expressions.
-   */
-  private String expression;
-
-  /**
-   * Position in the query
-   */
-  private int queryOrder;
-
-  public CustomAggregateExpression() {
-    referredColumns = new ArrayList<CarbonColumn>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  public String getExpression() {
-    return expression;
-  }
-
-  public void setExpression(String expression) {
-    this.expression = expression;
-  }
-
-  public List<CarbonColumn> getReferredColumns() {
-    return referredColumns;
-  }
-
-  public void setReferredColumns(List<CarbonColumn> referredColumns) {
-    this.referredColumns = referredColumns;
-  }
-
-  public int getQueryOrder() {
-    return queryOrder;
-  }
-
-  public void setQueryOrder(int queryOrder) {
-    this.queryOrder = queryOrder;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /**
-   * @return MeasureAggregator
-   */
-  public CustomMeasureAggregator getAggregator() {
-    return aggregator;
-  }
-
-  /**
-   * @param aggregator
-   */
-  public void setAggregator(CustomMeasureAggregator aggregator) {
-    this.aggregator = aggregator;
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((expression == null) ? 0 : expression.hashCode());
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj == this) {
-      return true;
-    }
-
-    if (!(obj instanceof CustomAggregateExpression)) {
-      return false;
-    }
-
-    CustomAggregateExpression other = ((CustomAggregateExpression) obj);
-
-    if ((expression != null) && (expression.equals(other.expression))) {
-      return true;
-    }
-
-    if (expression != null) {
-      return expression.equalsIgnoreCase(other.expression);
-    }
-
-    if (other.expression != null) {
-      return other.expression.equalsIgnoreCase(expression);
-    }
-
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
index 57d96d0..64b519a 100644
--- a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
+++ b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
@@ -31,7 +31,6 @@ import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.expression.ColumnExpression;
 import org.carbondata.query.expression.Expression;
@@ -88,16 +87,6 @@ public class QueryModel implements Serializable {
   private int limit;
 
   /**
-   * for applying aggregation on dimension
-   */
-  private List<DimensionAggregatorInfo> dimAggregationInfo;
-
-  /**
-   * custom aggregate expression
-   */
-  private List<CustomAggregateExpression> expressions;
-
-  /**
    * to check if it is a count star query , so processing will be different
    */
   private boolean isCountStarQuery;
@@ -153,10 +142,6 @@ public class QueryModel implements Serializable {
 
   public QueryModel() {
     tableBlockInfos = new ArrayList<TableBlockInfo>();
-    dimAggregationInfo =
-        new ArrayList<DimensionAggregatorInfo>();
-    expressions =
-        new ArrayList<CustomAggregateExpression>();
     queryDimension = new ArrayList<QueryDimension>();
     queryMeasures = new ArrayList<QueryMeasure>();
     sortDimension = new ArrayList<QueryDimension>();
@@ -173,7 +158,6 @@ public class QueryModel implements Serializable {
 
     fillQueryModel(queryPlan, carbonTable, queryModel, factTableName);
 
-    fillDimensionAggregator(queryPlan, queryModel);
     queryModel.setLimit(queryPlan.getLimit());
     queryModel.setDetailQuery(queryPlan.isDetailQuery());
     queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery());
@@ -217,17 +201,6 @@ public class QueryModel implements Serializable {
 
   }
 
-  private static void fillDimensionAggregator(CarbonQueryPlan logicalPlan,
-      QueryModel executorModel) {
-    Map<String, DimensionAggregatorInfo> dimAggregatorInfos = logicalPlan.getDimAggregatorInfos();
-    List<DimensionAggregatorInfo> dimensionAggregatorInfos =
-        new ArrayList<DimensionAggregatorInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (Map.Entry<String, DimensionAggregatorInfo> entry : dimAggregatorInfos.entrySet()) {
-      dimensionAggregatorInfos.add(entry.getValue());
-    }
-    executorModel.setDimAggregationInfo(dimensionAggregatorInfos);
-  }
-
   public static void processFilterExpression(
       Expression filterExpression, List<CarbonDimension> dimensions, List<CarbonMeasure> measures) {
     if (null != filterExpression) {
@@ -397,20 +370,6 @@ public class QueryModel implements Serializable {
   }
 
   /**
-   * @return the dimAggregationInfo
-   */
-  public List<DimensionAggregatorInfo> getDimAggregationInfo() {
-    return dimAggregationInfo;
-  }
-
-  /**
-   * @param dimAggregationInfo the dimAggregationInfo to set
-   */
-  public void setDimAggregationInfo(List<DimensionAggregatorInfo> dimAggregationInfo) {
-    this.dimAggregationInfo = dimAggregationInfo;
-  }
-
-  /**
    * @return the tableBlockInfos
    */
   public List<TableBlockInfo> getTableBlockInfos() {
@@ -425,20 +384,6 @@ public class QueryModel implements Serializable {
   }
 
   /**
-   * @return the expressions
-   */
-  public List<CustomAggregateExpression> getExpressions() {
-    return expressions;
-  }
-
-  /**
-   * @param expressions the expressions to set
-   */
-  public void setExpressions(List<CustomAggregateExpression> expressions) {
-    this.expressions = expressions;
-  }
-
-  /**
    * @return the queryTempLocation
    */
   public String getQueryTempLocation() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
index 392fcd7..fdcf6f1 100644
--- a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
@@ -23,10 +23,8 @@ import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.DataRefNode;
 import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.aggregator.DataAggregator;
 import org.carbondata.query.carbon.aggregator.ScannedResultAggregator;
 import org.carbondata.query.carbon.aggregator.impl.ListBasedResultAggregator;
-import org.carbondata.query.carbon.aggregator.impl.MapBasedResultAggregator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.scanner.BlockletScanner;
@@ -82,13 +80,8 @@ public abstract class AbstractDataBlockProcessor implements BlockProcessor {
       blockletScanner = new NonFilterScanner(blockExecutionInfo);
     }
 
-    if (blockExecutionInfo.isDetailQuery() || blockExecutionInfo.isRawRecordDetailQuery()) {
-      this.scannerResultAggregator =
-          new ListBasedResultAggregator(blockExecutionInfo, new DataAggregator(blockExecutionInfo));
-    } else {
-      this.scannerResultAggregator =
-          new MapBasedResultAggregator(blockExecutionInfo, new DataAggregator(blockExecutionInfo));
-    }
+    this.scannerResultAggregator =
+        new ListBasedResultAggregator(blockExecutionInfo);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
index 81d09ae..444b2bd 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
@@ -135,6 +135,14 @@ public abstract class AbstractScannedResult {
   }
 
   /**
+   * Just increment the counter incase of query only on measures.
+   */
+  public void incrementCounter() {
+    rowCounter ++;
+    currentRow ++;
+  }
+
+  /**
    * Below method will be used to get the dimension data based on dimension
    * ordinal and index
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
index 26588e6..4aa2adf 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
@@ -20,10 +20,8 @@
 package org.carbondata.query.carbon.result;
 
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QuerySchemaInfo;
 import org.carbondata.query.carbon.util.DataTypeUtil;
@@ -32,38 +30,9 @@ import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 /**
  * Below class holds the query result of batches.
  */
-public class BatchRawResult extends CarbonIterator<Object[]> {
-
-  /**
-   * list of keys
-   */
-  private Object[][] rows;
+public class BatchRawResult extends BatchResult {
 
   private QuerySchemaInfo querySchemaInfo;
-  /**
-   * counter to check whether all the records are processed or not
-   */
-  private int counter;
-
-  /**
-   * size of the batches.
-   */
-  private int size;
-
-  public BatchRawResult(Object[][] rows) {
-    this.rows = rows;
-    if (rows.length > 0) {
-      this.size = rows[0].length;
-    }
-  }
-
-  /**
-   * This will return all the raw records.
-   * @return
-   */
-  public Object[][] getAllRows() {
-    return rows;
-  }
 
   /**
    * This method will return one row at a time based on the counter given.
@@ -71,22 +40,8 @@ public class BatchRawResult extends CarbonIterator<Object[]> {
    * @return
    */
   public Object[] getRawRow(int counter) {
-    Object[] outputRow = new Object[rows.length];
-    for(int col = 0 ; col < rows.length ; col++) {
-      outputRow[col] = rows[col][counter];
-    }
-    return outputRow;
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements.
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return counter < size;
+    return rows[counter];
   }
-
   /**
    * Returns the next element in the iteration.
    *
@@ -97,71 +52,38 @@ public class BatchRawResult extends CarbonIterator<Object[]> {
   }
 
   private Object[] parseData() {
-    ByteArrayWrapper key = (ByteArrayWrapper) rows[0][counter];
     int[] order = querySchemaInfo.getQueryReverseOrder();
-    long[] surrogateResult = querySchemaInfo.getKeyGenerator()
-        .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
+    Object[] row = rows[counter];
+    ByteArrayWrapper key = (ByteArrayWrapper) row[0];
     QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
-    Object[] parsedData = new Object[queryDimensions.length + rows.length - 1];
-    int noDictionaryColumnIndex = 0;
-    for (int i = 0; i < queryDimensions.length; i++) {
-      if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        parsedData[order[i]] = DataTypeUtil.getDataBasedOnDataType(
-            new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
-            queryDimensions[i].getDimension().getDataType());
-      } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
-        parsedData[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
-            (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
-      } else {
-        parsedData[order[i]] =
-            (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
+    Object[] parsedData = new Object[queryDimensions.length + row.length - 1];
+    if(key != null) {
+      long[] surrogateResult = querySchemaInfo.getKeyGenerator()
+          .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
+      int noDictionaryColumnIndex = 0;
+      for (int i = 0; i < queryDimensions.length; i++) {
+        if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
+          parsedData[order[i]] = DataTypeUtil.getDataBasedOnDataType(
+              new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
+              queryDimensions[i].getDimension().getDataType());
+        } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+              .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
+          parsedData[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
+              (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
+        } else {
+          parsedData[order[i]] =
+              (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
+        }
       }
     }
-    for (int i = 0; i < rows.length - 1; i++) {
-      parsedData[order[i + queryDimensions.length]] = rows[i + 1][counter];
+    for (int i = 0; i < row.length - 1; i++) {
+      parsedData[order[i + queryDimensions.length]] = row[i + 1];
     }
     counter++;
     return parsedData;
   }
 
-  public static Object[] parseData(ByteArrayWrapper key, Object[] aggData,
-      QuerySchemaInfo querySchemaInfo, int[] aggOrder) {
-    long[] surrogateResult = querySchemaInfo.getKeyGenerator()
-        .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
-    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
-    Object[] parsedData = new Object[queryDimensions.length + aggData.length];
-    int noDictionaryColumnIndex = 0;
-    for (int i = 0; i < queryDimensions.length; i++) {
-      if (!CarbonUtil
-          .hasEncoding(queryDimensions[i].getDimension().getEncoder(), Encoding.DICTIONARY)) {
-        parsedData[i] = DataTypeUtil.getDataBasedOnDataType(
-            new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
-            queryDimensions[i].getDimension().getDataType());
-      } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
-        parsedData[i] = directDictionaryGenerator.getValueFromSurrogate(
-            (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
-      } else {
-        parsedData[i] = (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
-      }
-    }
-    for (int i = 0; i < aggData.length; i++) {
-      parsedData[i + queryDimensions.length] = aggData[i];
-    }
-    Object[] orderData = new Object[parsedData.length];
-    for (int i = 0; i < parsedData.length; i++) {
-      orderData[i] = parsedData[aggOrder[i]];
-    }
-    return orderData;
-  }
-
-  public QuerySchemaInfo getQuerySchemaInfo() {
-    return querySchemaInfo;
-  }
-
   public void setQuerySchemaInfo(QuerySchemaInfo querySchemaInfo) {
     this.querySchemaInfo = querySchemaInfo;
   }
@@ -171,6 +93,6 @@ public class BatchRawResult extends CarbonIterator<Object[]> {
    * @return
    */
   public int getSize() {
-    return size;
+    return rows.length;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java b/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
index f8e2a54..220336b 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
@@ -19,82 +19,55 @@
 
 package org.carbondata.query.carbon.result;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.NoSuchElementException;
 
-import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.scanner.impl.CarbonKey;
-import org.carbondata.query.scanner.impl.CarbonValue;
 
 /**
  * Below class holds the query result
  */
-public class BatchResult extends CarbonIterator<RowResult> {
+public class BatchResult extends CarbonIterator<Object[]> {
 
   /**
    * list of keys
    */
-  private List<CarbonKey> keys;
-
-  /**
-   * list of values
-   */
-  private List<CarbonValue> values;
+  protected Object[][] rows;
 
   /**
    * counter to check whether all the records are processed or not
    */
-  private int counter;
+  protected int counter;
 
   public BatchResult() {
-    keys = new ArrayList<CarbonKey>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    values = new ArrayList<CarbonValue>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.rows = new Object[0][];
   }
 
   /**
-   * Below method will be used to get the key
+   * Below method will be used to get the rows
    *
    * @return
    */
-  public List<CarbonKey> getKeys() {
-    return keys;
-  }
-
-  /**
-   * below method will be used to set the key
-   *
-   * @param keys
-   */
-  public void setKeys(List<CarbonKey> keys) {
-    this.keys = keys;
-  }
-
-  /**
-   * Below method will be used to get the values
-   *
-   * @return
-   */
-  public List<CarbonValue> getValues() {
-    return values;
+  public Object[][] getRows() {
+    return rows;
   }
 
   /**
    * Below method will be used to get the set the values
    *
-   * @param values
+   * @param rows
    */
-  public void setValues(List<CarbonValue> values) {
-    this.values = values;
+  public void setRows(Object[][] rows) {
+    this.rows = rows;
   }
 
+
   /**
    * Returns {@code true} if the iteration has more elements.
    *
    * @return {@code true} if the iteration has more elements
    */
   @Override public boolean hasNext() {
-    return counter < keys.size();
+    return counter < rows.length;
   }
 
   /**
@@ -102,11 +75,12 @@ public class BatchResult extends CarbonIterator<RowResult> {
    *
    * @return the next element in the iteration
    */
-  @Override public RowResult next() {
-    RowResult rowResult = new RowResult();
-    rowResult.setKey(keys.get(counter));
-    rowResult.setValue(values.get(counter));
+  @Override public Object[] next() {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
+    Object[] row = rows[counter];
     counter++;
-    return rowResult;
+    return row;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java b/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
index ca72d2d..1a40fc3 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
@@ -1,13 +1,12 @@
 package org.carbondata.query.carbon.result;
 
-import org.carbondata.query.aggregator.MeasureAggregator;
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 
 public class ListBasedResultWrapper {
 
   private ByteArrayWrapper key;
 
-  private MeasureAggregator[] value;
+  private Object[] value;
 
   /**
    * @return the key
@@ -26,14 +25,14 @@ public class ListBasedResultWrapper {
   /**
    * @return the value
    */
-  public MeasureAggregator[] getValue() {
+  public Object[] getValue() {
     return value;
   }
 
   /**
    * @param value the value to set
    */
-  public void setValue(MeasureAggregator[] value) {
+  public void setValue(Object[] value) {
     this.value = value;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/Result.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/Result.java b/core/src/main/java/org/carbondata/query/carbon/result/Result.java
index d4ba14f..8526b07 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/Result.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/Result.java
@@ -19,13 +19,12 @@
 
 package org.carbondata.query.carbon.result;
 
-import org.carbondata.query.aggregator.MeasureAggregator;
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 
 /**
  * Result interface for storing the result
  */
-public interface Result<K> {
+public interface Result<K, V> {
   /**
    * Below method will be used to
    * add the sccaed result
@@ -53,9 +52,9 @@ public interface Result<K> {
    *
    * @return value
    */
-  MeasureAggregator[] getValue();
+  V[] getValue();
 
-  void merge(Result<K> otherResult);
+  void merge(Result<K, V> otherResult);
 
   /**
    * Below method will be used to get the result

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java b/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java
deleted file mode 100644
index 93a9102..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java
+++ /dev/null
@@ -1,45 +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.query.carbon.result;
-
-import org.carbondata.query.scanner.impl.CarbonKey;
-import org.carbondata.query.scanner.impl.CarbonValue;
-
-public class RowResult {
-  private CarbonKey key;
-
-  private CarbonValue value;
-
-  public CarbonKey getKey() {
-    return key;
-  }
-
-  public void setKey(CarbonKey key) {
-    this.key = key;
-  }
-
-  public CarbonValue getValue() {
-    return value;
-  }
-
-  public void setValue(CarbonValue value) {
-    this.value = value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
index 8b7904a..3f31028 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
 import org.carbondata.query.carbon.result.ListBasedResultWrapper;
 import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
@@ -31,7 +30,7 @@ import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 /**
  * Below class is a holder over list based result wrapper
  */
-public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
+public class ListBasedResult implements Result<List<ListBasedResultWrapper>, Object> {
 
   /**
    * current result list
@@ -71,9 +70,9 @@ public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
 
   public ListBasedResult() {
     currentRowPointer =
-        new ArrayList<ListBasedResultWrapper>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     allRowsResult =
-        new ArrayList<List<ListBasedResultWrapper>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
   }
 
   /**
@@ -121,7 +120,7 @@ public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
   /**
    * @return will return the value
    */
-  @Override public MeasureAggregator[] getValue() {
+  @Override public Object[] getValue() {
     return currentRowPointer.get(listRecordCounter).getValue();
   }
 
@@ -131,7 +130,7 @@ public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
    *
    * @param otherResult return to be merged
    */
-  @Override public void merge(Result<List<ListBasedResultWrapper>> otherResult) {
+  @Override public void merge(Result<List<ListBasedResultWrapper>, Object> otherResult) {
     if (otherResult.size() > 0) {
       totalNumberOfRecords += otherResult.size();
       this.allRowsResult.add(otherResult.getResult());



Mime
View raw message