http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
 
b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
deleted file mode 100644
index 458a4c8..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
+++ /dev/null
@@ -1,84 +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.iterator;
-
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-
-/**
- * In case of detail query we cannot keep all the records in memory so for
- * executing that query are returning a iterator over block and every time next
- * call will come it will execute the block and return the result
- */
-public class DetailQueryResultIterator extends 
AbstractDetailQueryResultIterator {
-
-  /**
-   * 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) {
-    super(infos, queryModel);
-    this.queryResultPreparator = queryResultPreparator;
-  }
-
-  @Override public BatchResult next() {
-    BatchResult result;
-    try {
-      if (future == null) {
-        future = execute();
-      }
-      result = future.get();
-      nextBatch = false;
-      if (hasNext()) {
-        nextBatch = true;
-        future = execute();
-      } else {
-        fileReader.finish();
-      }
-    } catch (Exception ex) {
-      fileReader.finish();
-      throw new RuntimeException(ex.getCause().getMessage());
-    }
-    return result;
-  }
-
-  private Future<BatchResult> execute() {
-    return execService.submit(new Callable<BatchResult>() {
-      @Override public BatchResult call() throws QueryExecutionException {
-        return 
queryResultPreparator.prepareQueryResult(dataBlockIterator.next());
-      }
-    });
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java
 
b/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java
deleted file mode 100644
index c99ee15..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.carbondata.query.carbon.result.iterator;
-
-import java.util.NoSuchElementException;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Memory based result iterator
- */
-public class MemoryBasedResultIterator extends CarbonIterator<Result> {
-
-  /**
-   * query result
-   */
-  private Result result;
-
-  /**
-   * to check any more
-   */
-  private boolean hasNext = true;
-
-  public MemoryBasedResultIterator(Result result) {
-    this.result = result;
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements. (In other words,
-   * returns {@code true} if {@link #next} would return an element rather than
-   * throwing an exception.)
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return hasNext;
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Result next() {
-    if (hasNext) {
-      hasNext = false;
-      return result;
-    }
-    throw new NoSuchElementException();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java
 
b/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java
deleted file mode 100644
index 27114f2..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java
+++ /dev/null
@@ -1,167 +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.iterator;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.carbon.result.BatchRawResult;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * This is a wrapper iterator over the detail raw query iterator.
- * This iterator will handle the processing of the raw rows.
- * This will handle the batch results and will iterate on the batches and give 
single row.
- */
-public class RawResultIterator extends CarbonIterator<Object[]> {
-
-  private final SegmentProperties sourceSegProperties;
-
-  private final SegmentProperties destinationSegProperties;
-  /**
-   * Iterator of the Batch raw result.
-   */
-  private CarbonIterator<BatchRawResult> detailRawQueryResultIterator;
-
-  /**
-   * Counter to maintain the row counter.
-   */
-  private int counter = 0;
-
-  private Object[] currentConveretedRawRow = null;
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(RawResultIterator.class.getName());
-
-  /**
-   * batch of the result.
-   */
-  private BatchRawResult batch;
-
-  public RawResultIterator(CarbonIterator<BatchRawResult> 
detailRawQueryResultIterator,
-      SegmentProperties sourceSegProperties, SegmentProperties 
destinationSegProperties) {
-    this.detailRawQueryResultIterator = detailRawQueryResultIterator;
-    this.sourceSegProperties = sourceSegProperties;
-    this.destinationSegProperties = destinationSegProperties;
-  }
-
-  @Override public boolean hasNext() {
-
-    if (null == batch || checkIfBatchIsProcessedCompletely(batch)) {
-      if (detailRawQueryResultIterator.hasNext()) {
-        batch = detailRawQueryResultIterator.next();
-        counter = 0; // batch changed so reset the counter.
-      } else {
-        return false;
-      }
-    }
-
-    if (!checkIfBatchIsProcessedCompletely(batch)) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override public Object[] next() {
-    if (null == batch) { // for 1st time
-      batch = detailRawQueryResultIterator.next();
-    }
-    if (!checkIfBatchIsProcessedCompletely(batch)) {
-      try {
-        if(null != currentConveretedRawRow){
-          counter++;
-          Object[] currentConveretedRawRowTemp = this.currentConveretedRawRow;
-          currentConveretedRawRow = null;
-          return currentConveretedRawRowTemp;
-        }
-        return convertRow(batch.getRawRow(counter++));
-      } catch (KeyGenException e) {
-        LOGGER.error(e.getMessage());
-        return null;
-      }
-    } else { // completed one batch.
-      batch = detailRawQueryResultIterator.next();
-      counter = 0;
-    }
-    try {
-      if(null != currentConveretedRawRow){
-        counter++;
-        Object[] currentConveretedRawRowTemp = this.currentConveretedRawRow;
-        currentConveretedRawRow = null;
-        return currentConveretedRawRowTemp;
-      }
-
-      return convertRow(batch.getRawRow(counter++));
-    } catch (KeyGenException e) {
-      LOGGER.error(e.getMessage());
-      return null;
-    }
-
-  }
-
-  /**
-   * for fetching the row with out incrementing counter.
-   * @return
-   */
-  public Object[] fetchConverted() throws KeyGenException {
-    if(null != currentConveretedRawRow){
-      return currentConveretedRawRow;
-    }
-    if(hasNext())
-    {
-      Object[] rawRow = batch.getRawRow(counter);
-      currentConveretedRawRow = convertRow(rawRow);;
-      return currentConveretedRawRow;
-    }
-    else
-    {
-      return null;
-    }
-  }
-
-  private Object[] convertRow(Object[] rawRow) throws KeyGenException {
-    byte[] dims = ((ByteArrayWrapper) rawRow[0]).getDictionaryKey();
-    long[] keyArray = 
sourceSegProperties.getDimensionKeyGenerator().getKeyArray(dims);
-    byte[] covertedBytes =
-        
destinationSegProperties.getDimensionKeyGenerator().generateKey(keyArray);
-    ((ByteArrayWrapper) rawRow[0]).setDictionaryKey(covertedBytes);
-    return rawRow;
-  }
-
-  /**
-   * To check if the batch is processed completely
-   * @param batch
-   * @return
-   */
-  private boolean checkIfBatchIsProcessedCompletely(BatchRawResult batch){
-    if(counter < batch.getSize())
-    {
-      return false;
-    }
-    else{
-      return true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
 
b/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
deleted file mode 100644
index fbf3074..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
+++ /dev/null
@@ -1,10 +0,0 @@
-package org.carbondata.query.carbon.result.preparator;
-
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.Result;
-
-public interface QueryResultPreparator<K, V> {
-
-  public BatchResult prepareQueryResult(Result<K, V> scannedResult);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
 
b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
deleted file mode 100644
index ad5b4c5..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
+++ /dev/null
@@ -1,87 +0,0 @@
-package org.carbondata.query.carbon.result.preparator.impl;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-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.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-
-public abstract class AbstractQueryResultPreparator<K, V> implements 
QueryResultPreparator<K, V> {
-
-  /**
-   * query properties
-   */
-  protected QueryExecutorProperties queryExecuterProperties;
-
-  /**
-   * query model
-   */
-  protected QueryModel queryModel;
-
-  public AbstractQueryResultPreparator(QueryExecutorProperties 
executerProperties,
-      QueryModel queryModel) {
-    this.queryExecuterProperties = executerProperties;
-    this.queryModel = queryModel;
-  }
-
-  protected void fillDimensionData(Object[][] convertedResult, 
List<QueryDimension> queryDimensions,
-      int dimensionCount, Object[] row, int rowIndex) {
-    QueryDimension queryDimension;
-    for (int i = 0; i < dimensionCount; i++) {
-      queryDimension = queryDimensions.get(i);
-      if (!CarbonUtil
-          .hasEncoding(queryDimension.getDimension().getEncoder(), 
Encoding.DICTIONARY)) {
-        row[queryDimension.getQueryOrder()] = convertedResult[i][rowIndex];
-      } else if (CarbonUtil
-          .hasEncoding(queryDimension.getDimension().getEncoder(), 
Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator directDictionaryGenerator = 
DirectDictionaryKeyGeneratorFactory
-            
.getDirectDictionaryGenerator(queryDimension.getDimension().getDataType());
-        row[queryDimension.getQueryOrder()] = directDictionaryGenerator
-            .getValueFromSurrogate((Integer) convertedResult[i][rowIndex]);
-      } else {
-        if (queryExecuterProperties.sortDimIndexes[i] == 1) {
-          row[queryDimension.getQueryOrder()] = 
DataTypeUtil.getDataBasedOnDataType(
-              queryExecuterProperties.columnToDictionayMapping
-                  .get(queryDimension.getDimension().getColumnId())
-                  .getDictionaryValueFromSortedIndex((Integer) 
convertedResult[i][rowIndex]),
-              queryDimension.getDimension().getDataType());
-        } else {
-          row[queryDimension.getQueryOrder()] = 
DataTypeUtil.getDataBasedOnDataType(
-              queryExecuterProperties.columnToDictionayMapping
-                  .get(queryDimension.getDimension().getColumnId())
-                  .getDictionaryValueForKey((Integer) 
convertedResult[i][rowIndex]),
-              queryDimension.getDimension().getDataType());
-        }
-      }
-    }
-  }
-
-  protected Object[][] encodeToRows(Object[][] data) {
-    if (data.length == 0) {
-      return data;
-    }
-    Object[][] rData = new Object[data[0].length][data.length];
-    int len = data.length;
-    for (int i = 0; i < rData.length; i++) {
-      for (int j = 0; j < len; j++) {
-        rData[i][j] = data[j][i];
-      }
-    }
-    return rData;
-  }
-
-  protected BatchResult getEmptyChunkResult(int size) {
-    Object[][] row = new Object[size][1];
-    BatchResult chunkResult = new BatchResult();
-    chunkResult.setRows(row);
-    return chunkResult;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
 
b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
deleted file mode 100644
index 712894a..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
+++ /dev/null
@@ -1,139 +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.preparator.impl;
-
-import java.nio.charset.Charset;
-import java.util.List;
-
-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.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * Below class will be used to get the result by converting to actual data
- * Actual data conversion can be converting the surrogate key to actual data
- *
- * @TODO there are many things in class which is very confusing, need to check
- * why it was handled like that and how we can handle that in a better
- * way.Need to revisit this class. IF aggregation is push down to spark
- * layer and if we can process the data in byte array format then this
- * class wont be useful so in future we can delete this class.
- * @TODO need to expose one interface which will return the result based on 
required type
- * for example its implementation case return converted result or directly 
result with out
- * converting to actual value
- */
-public class DetailQueryResultPreparatorImpl
-    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, 
Object> {
-
-  private static final LogService LOGGER =
-      
LogServiceFactory.getLogService(DetailQueryResultPreparatorImpl.class.getName());
-
-  public DetailQueryResultPreparatorImpl(QueryExecutorProperties 
executerProperties,
-      QueryModel queryModel) {
-    super(executerProperties, queryModel);
-  }
-
-  @Override public BatchResult prepareQueryResult(
-      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
-    if ((null == scannedResult || scannedResult.size() < 1)) {
-      return new BatchResult();
-    }
-    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
-    int dimensionCount = queryDimension.size();
-    int totalNumberOfColumn = dimensionCount + 
queryExecuterProperties.measureDataTypes.length;
-    Object[][] resultData = new 
Object[scannedResult.size()][totalNumberOfColumn];
-    if (!queryExecuterProperties.isFunctionQuery && totalNumberOfColumn == 0
-        && scannedResult.size() > 0) {
-      return getEmptyChunkResult(scannedResult.size());
-    }
-    int currentRow = 0;
-    long[] surrogateResult = null;
-    int noDictionaryColumnIndex = 0;
-    ByteArrayWrapper key = null;
-    Object[] value = null;
-    while (scannedResult.hasNext()) {
-      key = scannedResult.getKey();
-      value = scannedResult.getValue();
-      if (key != null) {
-        surrogateResult = 
queryExecuterProperties.keyStructureInfo.getKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(),
-                queryExecuterProperties.keyStructureInfo.getMaskedBytes());
-        for (int i = 0; i < dimensionCount; i++) {
-          if 
(!CarbonUtil.hasEncoding(queryDimension.get(i).getDimension().getEncoder(),
-              Encoding.DICTIONARY)) {
-            resultData[currentRow][i] = DataTypeUtil.getDataBasedOnDataType(
-                new 
String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++),
-                    Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
-                queryDimension.get(i).getDimension().getDataType());
-          } else {
-            resultData[currentRow][i] =
-                (int) 
surrogateResult[queryDimension.get(i).getDimension().getKeyOrdinal()];
-          }
-        }
-      }
-      if (value != null) {
-        System.arraycopy(value, 0, resultData[currentRow], dimensionCount,
-            queryExecuterProperties.measureDataTypes.length);
-      }
-      currentRow++;
-      noDictionaryColumnIndex = 0;
-    }
-    if (resultData.length > 0) {
-      resultData = encodeToRows(resultData);
-    }
-    return getResult(queryModel, resultData);
-  }
-
-  private BatchResult getResult(QueryModel queryModel, Object[][] 
convertedResult) {
-
-    int rowSize = convertedResult[0].length;
-    Object[][] rows = new Object[rowSize][];
-    List<QueryDimension> queryDimensions = queryModel.getQueryDimension();
-    int dimensionCount = queryDimensions.size();
-    int msrCount = queryExecuterProperties.measureDataTypes.length;
-    Object[] row;
-    for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) {
-      row = new Object[dimensionCount + msrCount];
-      fillDimensionData(convertedResult, queryDimensions, dimensionCount, row, 
rowIndex);
-
-      QueryMeasure msr;
-      for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) {
-        msr = queryModel.getQueryMeasures().get(i);
-        row[msr.getQueryOrder()] = convertedResult[dimensionCount + 
i][rowIndex];
-      }
-      rows[rowIndex] = row;
-    }
-    LOGGER.info(
-        "###########################################------ Total Number of 
records" + rowSize);
-    BatchResult chunkResult = new BatchResult();
-    chunkResult.setRows(rows);
-    return chunkResult;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
 
b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
deleted file mode 100644
index a6a8fbc..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
+++ /dev/null
@@ -1,127 +0,0 @@
-package org.carbondata.query.carbon.result.preparator.impl;
-
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import 
org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import 
org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.model.QuerySchemaInfo;
-import org.carbondata.query.carbon.result.BatchRawResult;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * It does not decode the dictionary.
- */
-public class RawQueryResultPreparatorImpl
-    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, 
Object> {
-
-  private static final LogService LOGGER =
-      
LogServiceFactory.getLogService(RawQueryResultPreparatorImpl.class.getName());
-
-  private QuerySchemaInfo querySchemaInfo;
-
-  public RawQueryResultPreparatorImpl(QueryExecutorProperties 
executerProperties,
-      QueryModel queryModel) {
-    super(executerProperties, queryModel);
-    querySchemaInfo = new QuerySchemaInfo();
-    
querySchemaInfo.setKeyGenerator(queryExecuterProperties.keyStructureInfo.getKeyGenerator());
-    
querySchemaInfo.setMaskedByteIndexes(queryExecuterProperties.keyStructureInfo.getMaskedBytes());
-    querySchemaInfo.setQueryDimensions(queryModel.getQueryDimension()
-        .toArray(new QueryDimension[queryModel.getQueryDimension().size()]));
-    querySchemaInfo.setQueryMeasures(queryModel.getQueryMeasures()
-        .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
-    int msrSize = queryExecuterProperties.measureDataTypes.length;
-    int dimensionCount = queryModel.getQueryDimension().size();
-    int[] queryOrder = new int[dimensionCount + msrSize];
-    int[] queryReverseOrder = new int[dimensionCount + msrSize];
-    for (int i = 0; i < dimensionCount; i++) {
-      queryOrder[queryModel.getQueryDimension().get(i).getQueryOrder()] = i;
-      queryReverseOrder[i] = 
queryModel.getQueryDimension().get(i).getQueryOrder();
-    }
-    for (int i = 0; i < msrSize; i++) {
-      queryOrder[queryModel.getQueryMeasures().get(i).getQueryOrder()] = i + 
dimensionCount;
-      queryReverseOrder[i + dimensionCount] = 
queryModel.getQueryMeasures().get(i).getQueryOrder();
-    }
-    querySchemaInfo.setQueryOrder(queryOrder);
-    querySchemaInfo.setQueryReverseOrder(queryReverseOrder);
-  }
-
-  @Override public BatchResult prepareQueryResult(
-      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
-    if ((null == scannedResult || scannedResult.size() < 1)) {
-      return new BatchRawResult();
-    }
-    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
-    int msrSize = queryExecuterProperties.measureDataTypes.length;
-    int dimSize = queryDimensions.length;
-    int[] order = querySchemaInfo.getQueryReverseOrder();
-    Object[][] resultData = new Object[scannedResult.size()][];
-    Object[] value;
-    Object[] row;
-    int counter = 0;
-    if (queryModel.isRawBytesDetailQuery()) {
-      while (scannedResult.hasNext()) {
-        value = scannedResult.getValue();
-        row = new Object[msrSize + 1];
-        row[0] = scannedResult.getKey();
-        if (value != null) {
-          assert (value.length == msrSize);
-          System.arraycopy(value, 0, row, 1, msrSize);
-        }
-        resultData[counter] = row;
-        counter++;
-      }
-    } else {
-      while (scannedResult.hasNext()) {
-        value = scannedResult.getValue();
-        row = new Object[msrSize + dimSize];
-        ByteArrayWrapper key = scannedResult.getKey();
-        if (key != null) {
-          long[] surrogateResult = querySchemaInfo.getKeyGenerator()
-              .getKeyArray(key.getDictionaryKey(), 
querySchemaInfo.getMaskedByteIndexes());
-          int noDictionaryColumnIndex = 0;
-          for (int i = 0; i < dimSize; i++) {
-            if 
(!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
-              row[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());
-              if (directDictionaryGenerator != null) {
-                row[order[i]] = 
directDictionaryGenerator.getValueFromSurrogate(
-                    (int) 
surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
-              }
-            } else {
-              row[order[i]] =
-                  (int) 
surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
-            }
-          }
-        }
-        for (int i = 0; i < msrSize; i++) {
-          row[order[i + queryDimensions.length]] = value[i];
-        }
-        resultData[counter] = row;
-        counter++;
-      }
-    }
-
-    LOGGER.info("###########################---- Total Number of records" + 
scannedResult.size());
-    BatchRawResult result = new BatchRawResult();
-    result.setRows(resultData);
-    return result;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java
 
b/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java
deleted file mode 100644
index 34738c8..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java
+++ /dev/null
@@ -1,62 +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.scanner;
-
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Blocklet scanner class to process the block
- */
-public abstract class AbstractBlockletScanner implements BlockletScanner {
-
-  /**
-   * scanner result
-   */
-  protected AbstractScannedResult scannedResult;
-
-  /**
-   * block execution info
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) {
-    this.blockExecutionInfo = tableBlockExecutionInfos;
-  }
-
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder 
blocksChunkHolder)
-      throws QueryExecutionException {
-    fillKeyValue(blocksChunkHolder);
-    return scannedResult;
-  }
-
-  protected void fillKeyValue(BlocksChunkHolder blocksChunkHolder) {
-    scannedResult.reset();
-    scannedResult.setMeasureChunks(blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedMeasureBlocksIndexes()));
-    scannedResult.setNumberOfRows(blocksChunkHolder.getDataBlock().nodeSize());
-
-    scannedResult.setDimensionChunks(blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedDimensionBlocksIndexes()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java 
b/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java
deleted file mode 100644
index 39b2568..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java
+++ /dev/null
@@ -1,41 +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.scanner;
-
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Interface for processing the block
- * Processing can be filter based processing or non filter based processing
- */
-public interface BlockletScanner {
-
-  /**
-   * Below method will used to process the block data and get the scanned 
result
-   *
-   * @param blocksChunkHolder block chunk which holds the block data
-   * @return scannerResult
-   * result after processing
-   * @throws QueryExecutionException
-   */
-  AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws QueryExecutionException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java
 
b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java
deleted file mode 100644
index 5225b41..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java
+++ /dev/null
@@ -1,174 +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.scanner.impl;
-
-import java.util.BitSet;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.impl.FilterQueryScannedResult;
-import org.carbondata.query.carbon.scanner.AbstractBlockletScanner;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.executer.FilterExecuter;
-
-/**
- * Below class will be used for filter query processing
- * this class will be first apply the filter then it will read the block if
- * required and return the scanned result
- */
-public class FilterScanner extends AbstractBlockletScanner {
-
-  /**
-   * filter tree
-   */
-  private FilterExecuter filterExecuter;
-
-  /**
-   * this will be used to apply min max
-   * this will be useful for dimension column which is on the right side
-   * as node finder will always give tentative blocks, if column data stored 
individually
-   * and data is in sorted order then we can check whether filter is in the 
range of min max or not
-   * if it present then only we can apply filter on complete data.
-   * this will be very useful in case of sparse data when rows are
-   * repeating.
-   */
-  private boolean isMinMaxEnabled;
-
-  public FilterScanner(BlockExecutionInfo blockExecutionInfo) {
-    super(blockExecutionInfo);
-    scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
-    // to check whether min max is enabled or not
-    String minMaxEnableValue = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
-            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
-    if (null != minMaxEnableValue) {
-      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
-    }
-    // get the filter tree
-    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
-  }
-
-  /**
-   * Below method will be used to process the block
-   *
-   * @param blocksChunkHolder block chunk holder which holds the data
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder 
blocksChunkHolder)
-      throws QueryExecutionException {
-    try {
-      fillScannedResult(blocksChunkHolder);
-    } catch (FilterUnsupportedException e) {
-      throw new QueryExecutionException(e.getMessage());
-    }
-    return scannedResult;
-  }
-
-  /**
-   * This method will process the data in below order
-   * 1. first apply min max on the filter tree and check whether any of the 
filter
-   * is fall on the range of min max, if not then return empty result
-   * 2. If filter falls on min max range then apply filter on actual
-   * data and get the filtered row index
-   * 3. if row index is empty then return the empty result
-   * 4. if row indexes is not empty then read only those blocks(measure or 
dimension)
-   * which was present in the query but not present in the filter, as while 
applying filter
-   * some of the blocks where already read and present in chunk holder so not 
need to
-   * read those blocks again, this is to avoid reading of same blocks which 
was already read
-   * 5. Set the blocks and filter indexes to result
-   *
-   * @param blocksChunkHolder
-   * @throws FilterUnsupportedException
-   */
-  private void fillScannedResult(BlocksChunkHolder blocksChunkHolder)
-      throws FilterUnsupportedException {
-
-    scannedResult.reset();
-    // apply min max
-    if (isMinMaxEnabled) {
-      BitSet bitSet = this.filterExecuter
-          
.isScanRequired(blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
-              blocksChunkHolder.getDataBlock().getColumnsMinValue());
-      if (bitSet.isEmpty()) {
-        scannedResult.setNumberOfRows(0);
-        scannedResult.setIndexes(new int[0]);
-        return;
-      }
-    }
-    // apply filter on actual data
-    BitSet bitSet = this.filterExecuter.applyFilter(blocksChunkHolder);
-    // if indexes is empty then return with empty result
-    if (bitSet.isEmpty()) {
-      scannedResult.setNumberOfRows(0);
-      scannedResult.setIndexes(new int[0]);
-      return;
-    }
-    // get the row indexes from bot set
-    int[] indexes = new int[bitSet.cardinality()];
-    int index = 0;
-    for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-      indexes[index++] = i;
-    }
-
-    FileHolder fileReader = blocksChunkHolder.getFileReader();
-    int[] allSelectedDimensionBlocksIndexes =
-        blockExecutionInfo.getAllSelectedDimensionBlocksIndexes();
-    DimensionColumnDataChunk[] dimensionColumnDataChunk =
-        new 
DimensionColumnDataChunk[blockExecutionInfo.getTotalNumberDimensionBlock()];
-    // read dimension chunk blocks from file which is not present
-    for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) {
-      if (null == 
blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]])
 {
-        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getDimensionChunk(fileReader, 
allSelectedDimensionBlocksIndexes[i]);
-      } else {
-        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
-            
blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]];
-      }
-    }
-    MeasureColumnDataChunk[] measureColumnDataChunk =
-        new 
MeasureColumnDataChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()];
-    int[] allSelectedMeasureBlocksIndexes = 
blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
-
-    // read the measure chunk blocks which is not present
-    for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) {
-
-      if (null == 
blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]]) {
-        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getMeasureChunk(fileReader, 
allSelectedMeasureBlocksIndexes[i]);
-      } else {
-        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
-            
blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]];
-      }
-    }
-    scannedResult.setDimensionChunks(dimensionColumnDataChunk);
-    scannedResult.setIndexes(indexes);
-    scannedResult.setMeasureChunks(measureColumnDataChunk);
-    scannedResult.setNumberOfRows(indexes.length);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.java
 
b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.java
deleted file mode 100644
index c42d8b8..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.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.scanner.impl;
-
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.impl.NonFilterQueryScannedResult;
-import org.carbondata.query.carbon.scanner.AbstractBlockletScanner;
-
-/**
- * Non filter processor which will be used for non filter query
- * In case of non filter query we just need to read all the blocks requested 
in the
- * query and pass it to scanned result
- */
-public class NonFilterScanner extends AbstractBlockletScanner {
-
-  public NonFilterScanner(BlockExecutionInfo blockExecutionInfo) {
-    super(blockExecutionInfo);
-    // as its a non filter query creating a non filter query scanned result 
object
-    scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java 
b/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
deleted file mode 100644
index 2a7c5ae..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
+++ /dev/null
@@ -1,175 +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.util;
-
-import java.math.BigDecimal;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-import org.apache.spark.unsafe.types.UTF8String;
-
-/**
- * Utility for data type
- */
-public class DataTypeUtil {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataTypeUtil.class.getName());
-
-  /**
-   * Below method will be used to convert the data passed to its actual data
-   * type
-   *
-   * @param data           data
-   * @param actualDataType actual data type
-   * @return actual data after conversion
-   */
-  public static Object getDataBasedOnDataType(String data, DataType 
actualDataType) {
-
-    if (null == data) {
-      return null;
-    }
-    try {
-      switch (actualDataType) {
-        case INT:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Integer.parseInt(data);
-        case DOUBLE:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Double.parseDouble(data);
-        case LONG:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Long.parseLong(data);
-        case TIMESTAMP:
-          if (data.isEmpty()) {
-            return null;
-          }
-          SimpleDateFormat parser = new 
SimpleDateFormat(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-          Date dateToStr = null;
-          try {
-            dateToStr = parser.parse(data);
-            return dateToStr.getTime() * 1000;
-          } catch (ParseException e) {
-            LOGGER.error("Cannot convert" + data + " to Time/Long type value" 
+ e.getMessage());
-            return null;
-          }
-        case DECIMAL:
-          if (data.isEmpty()) {
-            return null;
-          }
-          java.math.BigDecimal javaDecVal = new java.math.BigDecimal(data);
-          scala.math.BigDecimal scalaDecVal = new 
scala.math.BigDecimal(javaDecVal);
-          org.apache.spark.sql.types.Decimal decConverter =
-              new org.apache.spark.sql.types.Decimal();
-          return decConverter.set(scalaDecVal);
-        default:
-          return UTF8String.fromString(data);
-      }
-    } catch (NumberFormatException ex) {
-      LOGGER.error("Problem while converting data type" + data);
-      return null;
-    }
-
-  }
-
-  public static Object getMeasureDataBasedOnDataType(Object data, DataType 
dataType) {
-
-    if (null == data) {
-      return null;
-    }
-    try {
-      switch (dataType) {
-        case DOUBLE:
-
-          return (Double) data;
-        case LONG:
-
-          return (Long) data;
-
-        case DECIMAL:
-
-          java.math.BigDecimal javaDecVal = new 
java.math.BigDecimal(data.toString());
-          scala.math.BigDecimal scalaDecVal = new 
scala.math.BigDecimal(javaDecVal);
-          org.apache.spark.sql.types.Decimal decConverter =
-              new org.apache.spark.sql.types.Decimal();
-          return decConverter.set(scalaDecVal);
-        default:
-
-          return data;
-      }
-    } catch (NumberFormatException ex) {
-      LOGGER.error("Problem while converting data type" + data);
-      return null;
-    }
-
-  }
-
-  public static int compareBasedOnDatatYpe(Object data1, Object data2, 
DataType dataType) {
-    switch (dataType) {
-      case INT:
-        return ((Integer) data1).compareTo((Integer) data2);
-      case LONG:
-      case TIMESTAMP:
-        return ((Long) data1).compareTo((Long) data2);
-      case DOUBLE:
-        return ((Double) data1).compareTo((Double) data2);
-      case DECIMAL:
-        return ((BigDecimal) data1).compareTo((BigDecimal) data2);
-      default:
-        return ((String) data1).compareTo((String) data2);
-    }
-  }
-
-  /**
-   * below method is to check whether data type is of numeric type or not
-   *
-   * @param dataType data type
-   * @return true if numeric data type
-   */
-  public boolean isNumericDatatype(DataType dataType) {
-    switch (dataType) {
-      case INT:
-      case LONG:
-      case DOUBLE:
-      case DECIMAL:
-        return true;
-      default:
-        return false;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java 
b/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java
deleted file mode 100644
index 6050b97..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java
+++ /dev/null
@@ -1,202 +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.wrappers;
-
-import org.carbondata.core.util.ByteUtil.UnsafeComparer;
-
-/**
- * This class will store the dimension column data when query is executed
- * This can be used as a key for aggregation
- */
-public class ByteArrayWrapper implements Comparable<ByteArrayWrapper> {
-
-  /**
-   * to store key which is generated using
-   * key generator
-   */
-  protected byte[] dictionaryKey;
-
-  /**
-   * to store no dictionary column data
-   */
-  protected byte[][] complexTypesKeys;
-
-  /**
-   * to store no dictionary column data
-   */
-  protected byte[][] noDictionaryKeys;
-
-  public ByteArrayWrapper() {
-  }
-
-  /**
-   * @return the dictionaryKey
-   */
-  public byte[] getDictionaryKey() {
-    return dictionaryKey;
-  }
-
-  /**
-   * @param dictionaryKey the dictionaryKey to set
-   */
-  public void setDictionaryKey(byte[] dictionaryKey) {
-    this.dictionaryKey = dictionaryKey;
-  }
-
-  /**
-   * @param noDictionaryKeys the noDictionaryKeys to set
-   */
-  public void setNoDictionaryKeys(byte[][] noDictionaryKeys) {
-    this.noDictionaryKeys = noDictionaryKeys;
-  }
-
-  /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getNoDictionaryKeyByIndex(int index) {
-    return this.noDictionaryKeys[index];
-  }
-
-  /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getComplexTypeByIndex(int index) {
-    return this.complexTypesKeys[index];
-  }
-
-  /**
-   * to generate the hash code
-   */
-  @Override public int hashCode() {
-    // first generate the has code of the dictionary column
-    int len = dictionaryKey.length;
-    int result = 1;
-    for (int j = 0; j < len; j++) {
-      result = 31 * result + dictionaryKey[j];
-    }
-    // then no dictionary column
-    for (byte[] directSurrogateValue : noDictionaryKeys) {
-      for (int i = 0; i < directSurrogateValue.length; i++) {
-        result = 31 * result + directSurrogateValue[i];
-      }
-    }
-    // then for complex type
-    for (byte[] complexTypeKey : complexTypesKeys) {
-      for (int i = 0; i < complexTypeKey.length; i++) {
-        result = 31 * result + complexTypeKey[i];
-      }
-    }
-    return result;
-  }
-
-  /**
-   * to validate the two
-   *
-   * @param other object
-   */
-  @Override public boolean equals(Object other) {
-    if (null == other || !(other instanceof ByteArrayWrapper)) {
-      return false;
-    }
-    boolean result = false;
-    // Comparison will be as follows
-    // first compare the no dictionary column
-    // if it is not equal then return false
-    // if it is equal then compare the complex column
-    // if it is also equal then compare dictionary column
-    byte[][] noDictionaryKeysOther = ((ByteArrayWrapper) 
other).noDictionaryKeys;
-    if (noDictionaryKeysOther.length != noDictionaryKeys.length) {
-      return false;
-    } else {
-      for (int i = 0; i < noDictionaryKeys.length; i++) {
-        result = UnsafeComparer.INSTANCE.equals(noDictionaryKeys[i], 
noDictionaryKeysOther[i]);
-        if (!result) {
-          return false;
-        }
-      }
-    }
-
-    byte[][] complexTypesKeysOther = ((ByteArrayWrapper) 
other).complexTypesKeys;
-    if (complexTypesKeysOther.length != complexTypesKeys.length) {
-      return false;
-    } else {
-      for (int i = 0; i < complexTypesKeys.length; i++) {
-        result = UnsafeComparer.INSTANCE.equals(complexTypesKeys[i], 
complexTypesKeysOther[i]);
-        if (!result) {
-          return false;
-        }
-      }
-    }
-
-    return UnsafeComparer.INSTANCE.equals(dictionaryKey, ((ByteArrayWrapper) 
other).dictionaryKey);
-  }
-
-  /**
-   * Compare method for ByteArrayWrapper class this will used to compare Two
-   * ByteArrayWrapper data object, basically it will compare two byte array
-   *
-   * @param other ArrayWrapper Object
-   */
-  @Override public int compareTo(ByteArrayWrapper other) {
-    // compare will be as follows
-    //compare dictionary column
-    // then no dictionary column
-    // then complex type column data
-    int compareTo = UnsafeComparer.INSTANCE.compareTo(dictionaryKey, 
other.dictionaryKey);
-    if (compareTo == 0) {
-      for (int i = 0; i < noDictionaryKeys.length; i++) {
-        compareTo =
-            UnsafeComparer.INSTANCE.compareTo(noDictionaryKeys[i], 
other.noDictionaryKeys[i]);
-        if (compareTo != 0) {
-          return compareTo;
-        }
-      }
-    }
-    if (compareTo == 0) {
-      for (int i = 0; i < complexTypesKeys.length; i++) {
-        compareTo =
-            UnsafeComparer.INSTANCE.compareTo(complexTypesKeys[i], 
other.complexTypesKeys[i]);
-        if (compareTo != 0) {
-          return compareTo;
-        }
-      }
-    }
-    return compareTo;
-  }
-
-  /**
-   * @return the complexTypesKeys
-   */
-  public byte[][] getComplexTypesKeys() {
-    return complexTypesKeys;
-  }
-
-  /**
-   * @param complexTypesKeys the complexTypesKeys to set
-   */
-  public void setComplexTypesKeys(byte[][] complexTypesKeys) {
-    this.complexTypesKeys = complexTypesKeys;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.java
 
b/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.java
deleted file mode 100644
index c54ce84..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.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.carbonfilterinterface;
-
-public enum ExpressionType {
-
-  AND,
-  OR,
-  NOT,
-  EQUALS,
-  NOT_EQUALS,
-  LESSTHAN,
-  LESSTHAN_EQUALTO,
-  GREATERTHAN,
-  GREATERTHAN_EQUALTO,
-  ADD,
-  SUBSTRACT,
-  DIVIDE,
-  MULTIPLY,
-  IN,
-  LIST,
-  NOT_IN,
-  UNKNOWN,
-  LITERAL,
-  RANGE
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java
 
b/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java
deleted file mode 100644
index b037c38..0000000
--- 
a/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java
+++ /dev/null
@@ -1,28 +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.carbonfilterinterface;
-
-import java.io.Serializable;
-
-public enum FilterExecuterType implements Serializable {
-
-  INCLUDE, EXCLUDE, OR, AND, RESTRUCTURE, ROWLEVEL, RANGE, 
ROWLEVEL_GREATERTHAN,
-  ROWLEVEL_GREATERTHAN_EQUALTO, ROWLEVEL_LESSTHAN_EQUALTO, ROWLEVEL_LESSTHAN
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.java 
b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.java
deleted file mode 100644
index 2133aca..0000000
--- a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.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.carbonfilterinterface;
-
-public class RowImpl implements RowIntf {
-  private Object[] row;
-
-  public RowImpl() {
-    row = new Object[0];
-  }
-
-  @Override public Object getVal(int index) {
-    return row[index];
-  }
-
-  @Override public Object[] getValues() {
-    return row;
-  }
-
-  @Override public void setValues(final Object[] row) {
-    this.row = row;
-  }
-
-  @Override public int size() {
-    return this.row.length;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java 
b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java
deleted file mode 100644
index 1984eed..0000000
--- a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java
+++ /dev/null
@@ -1,31 +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.carbonfilterinterface;
-
-public interface RowIntf {
-  Object getVal(int index);
-
-  Object[] getValues();
-
-  void setValues(Object[] setValues);
-
-  int size();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java
 
b/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java
deleted file mode 100644
index bfcc0f8..0000000
--- 
a/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java
+++ /dev/null
@@ -1,73 +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.complex.querytypes;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import 
org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-
-import org.apache.spark.sql.types.DataType;
-
-public interface GenericQueryType {
-
-  String getName();
-
-  void setName(String name);
-
-  String getParentname();
-
-  void setParentname(String parentname);
-
-  int getBlockIndex();
-
-  void setBlockIndex(int blockIndex);
-
-  void addChildren(GenericQueryType children);
-
-  void getAllPrimitiveChildren(List<GenericQueryType> primitiveChild);
-
-  int getSurrogateIndex();
-
-  void setSurrogateIndex(int surrIndex);
-
-  int getColsCount();
-
-  void setKeySize(int[] keyBlockSize);
-
-  int getKeyOrdinalForQuery();
-
-  void setKeyOrdinalForQuery(int keyOrdinalForQuery);
-
-  void parseBlocksAndReturnComplexColumnByteArray(
-      ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolder, int rowNumber,
-      DataOutputStream dataOutputStream) throws IOException;
-
-  DataType getSchemaType();
-
-  void parseAndGetResultBytes(ByteBuffer complexData, DataOutputStream 
dataOutput)
-      throws IOException;
-
-  void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java
 
b/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java
deleted file mode 100644
index dd2a656..0000000
--- 
a/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java
+++ /dev/null
@@ -1,32 +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.evaluators;
-
-public class DimColumnExecuterFilterInfo {
-
-  byte[][] filterKeys;
-
-  public void setFilterKeys(byte[][] filterKeys) {
-    this.filterKeys = filterKeys;
-  }
-
-  public byte[][] getFilterKeys() {
-    return filterKeys;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.java
 
b/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.java
deleted file mode 100644
index 174336b..0000000
--- 
a/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.java
+++ /dev/null
@@ -1,24 +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.evaluators;
-
-public interface FilterProcessorPlaceHolder {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java 
b/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java
deleted file mode 100644
index c85d991..0000000
--- a/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.expression;
-
-public abstract class BinaryExpression extends Expression {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-  /**
-   *
-   */
-
-  protected Expression left;
-  protected Expression right;
-  protected boolean isRangeExpression;
-
-  public BinaryExpression(Expression left, Expression right) {
-    this.left = left;
-    this.right = right;
-    children.add(left);
-    children.add(right);
-  }
-
-  public Expression getLeft() {
-    return left;
-  }
-
-  public Expression getRight() {
-    return right;
-  }
-
-  public boolean isRangeExpression() {
-    return isRangeExpression;
-  }
-
-  public void setRangeExpression(boolean isRangeExpression) {
-    this.isRangeExpression = isRangeExpression;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java 
b/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
deleted file mode 100644
index aa0de43..0000000
--- a/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
+++ /dev/null
@@ -1,114 +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.expression;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-
-public class ColumnExpression extends LeafExpression {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-
-  private String columnName;
-
-  private boolean isDimension;
-
-  private int colIndex = -1;
-
-  private DataType dataType;
-
-  private CarbonDimension dimension;
-
-  private CarbonColumn carbonColumn;
-
-  public ColumnExpression(String columnName, DataType dataType) {
-    this.columnName = columnName;
-    this.dataType = dataType;
-
-  }
-
-  public CarbonDimension getDimension() {
-    return dimension;
-  }
-
-  public void setDimension(CarbonDimension dimension) {
-    this.dimension = dimension;
-  }
-
-  public String getColumnName() {
-    return columnName;
-  }
-
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
-
-  public boolean isDimension() {
-    return isDimension;
-  }
-
-  public void setDimension(boolean isDimension) {
-    this.isDimension = isDimension;
-  }
-
-  public int getColIndex() {
-    return colIndex;
-  }
-
-  public void setColIndex(int colIndex) {
-    this.colIndex = colIndex;
-  }
-
-  public DataType getDataType() {
-    return dataType;
-  }
-
-  public void setDataType(DataType dataType) {
-    this.dataType = dataType;
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) {
-    ExpressionResult expressionResult = new ExpressionResult(dataType, 
value.getVal(colIndex));
-    return expressionResult;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override public String getString() {
-    // TODO Auto-generated method stub
-    return "ColumnExpression(" + columnName + ')';
-  }
-
-  public CarbonColumn getCarbonColumn() {
-    return carbonColumn;
-  }
-
-  public void setCarbonColumn(CarbonColumn carbonColumn) {
-    this.carbonColumn = carbonColumn;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/DataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/DataType.java 
b/core/src/main/java/org/carbondata/query/expression/DataType.java
deleted file mode 100644
index fb2b277..0000000
--- a/core/src/main/java/org/carbondata/query/expression/DataType.java
+++ /dev/null
@@ -1,34 +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.expression;
-
-public enum DataType {
-  StringType(0), DateType(1), TimestampType(2), BooleanType(1), 
IntegerType(3), FloatType(
-      4), LongType(5), DoubleType(6), NullType(7), DecimalType(8), 
ArrayType(9), StructType(10);
-  private int presedenceOrder;
-
-  private DataType(int value) {
-    this.presedenceOrder = value;
-  }
-
-  public int getPresedenceOrder() {
-    return presedenceOrder;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/Expression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/Expression.java 
b/core/src/main/java/org/carbondata/query/expression/Expression.java
deleted file mode 100644
index c033020..0000000
--- a/core/src/main/java/org/carbondata/query/expression/Expression.java
+++ /dev/null
@@ -1,51 +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.expression;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public abstract class Expression implements Serializable {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = -7568676723039530713L;
-  protected List<Expression> children =
-      new ArrayList<Expression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  public abstract ExpressionResult evaluate(RowIntf value) throws 
FilterUnsupportedException;
-
-  public abstract ExpressionType getFilterExpressionType();
-
-  public List<Expression> getChildren() {
-    return children;
-  }
-
-  public abstract String getString();
-
-  // public abstract void  accept(ExpressionVisitor visitor);
-}

Reply via email to