You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by jb...@apache.org on 2016/06/23 14:16:11 UTC

[23/56] [abbrv] incubator-carbondata git commit: Optimized detail query flow and cleanup (#691)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
index 09fa50c..d2aa014 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
@@ -18,7 +18,6 @@
  */
 package org.carbondata.query.carbon.result.iterator;
 
-import java.util.Arrays;
 import java.util.List;
 
 import org.carbondata.common.logging.LogService;
@@ -27,12 +26,15 @@ import org.carbondata.core.carbon.datastore.DataRefNode;
 import org.carbondata.core.carbon.datastore.DataRefNodeFinder;
 import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
+import org.carbondata.query.carbon.processor.AbstractDataBlockIterator;
+import org.carbondata.query.carbon.processor.impl.DataBlockIteratorImpl;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
@@ -60,101 +62,73 @@ public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
   /**
    * number of cores which can be used
    */
-  private long numberOfCores;
+  private int batchSize;
 
   /**
-   * keep track of number of blocklet per block
+   * file reader which will be used to execute the query
    */
-  private long[] totalNumberBlockletPerSlice;
+  protected FileHolder fileReader;
 
-  /**
-   * total number of blocklet to be executed
-   */
-  private long totalNumberOfNode;
+  protected AbstractDataBlockIterator dataBlockIterator;
 
-  /**
-   * current counter to check how blocklet has been executed
-   */
-  protected long currentCounter;
+  protected boolean nextBatch = false;
 
-  /**
-   * keep the track of number of blocklet of a block has been executed
-   */
-  private long[] numberOfBlockletExecutedPerBlock;
-
-  /**
-   * block index to be executed
-   */
-  protected int[] blockIndexToBeExecuted;
-
-  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos,
-      QueryExecutorProperties executerProperties, QueryModel queryModel,
-      InternalQueryExecutor queryExecutor) {
-    int recordSize = 0;
-    String defaultInMemoryRecordsSize =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.INMEMORY_REOCRD_SIZE);
-    if (null != defaultInMemoryRecordsSize) {
+  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
+    String batchSizeString =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE);
+    if (null != batchSizeString) {
       try {
-        recordSize = Integer.parseInt(defaultInMemoryRecordsSize);
+        batchSize = Integer.parseInt(batchSizeString);
       } catch (NumberFormatException ne) {
         LOGGER.error("Invalid inmemory records size. Using default value");
-        recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
+        batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
       }
+    } else {
+      batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
     }
-    this.numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    if (numberOfCores == 0) {
-      numberOfCores++;
-    }
-    executor = queryExecutor;
+
     this.blockExecutionInfos = infos;
-    this.blockIndexToBeExecuted = new int[(int) numberOfCores];
+    this.fileReader = FileFactory.getFileHolder(
+        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
     intialiseInfos();
   }
 
   private void intialiseInfos() {
-    this.totalNumberBlockletPerSlice = new long[blockExecutionInfos.size()];
-    this.numberOfBlockletExecutedPerBlock = new long[blockExecutionInfos.size()];
-    int index = -1;
     for (BlockExecutionInfo blockInfo : blockExecutionInfos) {
-      ++index;
       DataRefNodeFinder finder = new BTreeDataRefNodeFinder(blockInfo.getEachColumnValueSize());
       DataRefNode startDataBlock = finder
           .findFirstDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getStartKey());
       DataRefNode endDataBlock = finder
           .findLastDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getEndKey());
-
-      this.totalNumberBlockletPerSlice[index] =
-          endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
-      totalNumberOfNode += this.totalNumberBlockletPerSlice[index];
+      long numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
       blockInfo.setFirstDataBlock(startDataBlock);
-      blockInfo.setNumberOfBlockToScan(1);
+      blockInfo.setNumberOfBlockToScan(numberOfBlockToScan);
     }
-
   }
 
   @Override public boolean hasNext() {
-    return currentCounter < totalNumberOfNode;
+    if ((dataBlockIterator != null && dataBlockIterator.hasNext()) || nextBatch) {
+      return true;
+    } else {
+      dataBlockIterator = getDataBlockIterator();
+      while (dataBlockIterator != null) {
+        if (dataBlockIterator.hasNext()) {
+          return true;
+        }
+        dataBlockIterator = getDataBlockIterator();
+      }
+      return false;
+    }
   }
 
-  protected int updateSliceIndexToBeExecuted() {
-    Arrays.fill(blockIndexToBeExecuted, -1);
-    int currentSliceIndex = 0;
-    int i = 0;
-    for (; i < (int) numberOfCores; ) {
-      if (this.totalNumberBlockletPerSlice[currentSliceIndex]
-          > this.numberOfBlockletExecutedPerBlock[currentSliceIndex]) {
-        this.numberOfBlockletExecutedPerBlock[currentSliceIndex]++;
-        blockIndexToBeExecuted[i] = currentSliceIndex;
-        i++;
-      }
-      currentSliceIndex++;
-      if (currentSliceIndex >= totalNumberBlockletPerSlice.length) {
-        break;
-      }
+  private DataBlockIteratorImpl getDataBlockIterator() {
+    if(blockExecutionInfos.size() > 0) {
+      BlockExecutionInfo executionInfo = blockExecutionInfos.get(0);
+      blockExecutionInfos.remove(executionInfo);
+      return new DataBlockIteratorImpl(executionInfo, fileReader, batchSize);
     }
-    return i;
+    return null;
   }
 
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/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
index 3641e75..458a4c8 100644
--- 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
@@ -19,20 +19,17 @@
 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.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 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.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPreparatorImpl;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
@@ -42,48 +39,46 @@ import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPrepa
 public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator {
 
   /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DetailQueryResultIterator.class.getName());
-
-  /**
    * to prepare the result
    */
   private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
 
-  public DetailQueryResultIterator(List<BlockExecutionInfo> infos,
-      QueryExecutorProperties executerProperties, QueryModel queryModel,
-      InternalQueryExecutor queryExecutor) {
-    super(infos, executerProperties, queryModel, queryExecutor);
-    this.queryResultPreparator =
-        new DetailQueryResultPreparatorImpl(executerProperties, queryModel);
+  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() {
-    currentCounter += updateSliceIndexToBeExecuted();
-    CarbonIterator<Result> result = null;
+    BatchResult result;
     try {
-      result = executor.executeQuery(blockExecutionInfos, blockIndexToBeExecuted);
-    } catch (QueryExecutionException e) {
-      throw new RuntimeException(e.getCause().getMessage());
-    }
-    for (int i = 0; i < blockIndexToBeExecuted.length; i++) {
-      if (blockIndexToBeExecuted[i] != -1) {
-        blockExecutionInfos.get(blockIndexToBeExecuted[i]).setFirstDataBlock(
-            blockExecutionInfos.get(blockIndexToBeExecuted[i]).getFirstDataBlock()
-                .getNextDataRefNode());
+      if (future == null) {
+        future = execute();
       }
-    }
-    if (null != result) {
-      Result next = result.next();
-      if (next.size() > 0) {
-        return queryResultPreparator.prepareQueryResult(next);
+      result = future.get();
+      nextBatch = false;
+      if (hasNext()) {
+        nextBatch = true;
+        future = execute();
       } else {
-        return new BatchResult();
+        fileReader.finish();
       }
-    } else {
-      return new BatchResult();
+    } 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/29360501/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
deleted file mode 100644
index 2b14793..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.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.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-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.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.result.preparator.impl.RawQueryResultPreparatorImpl;
-
-/**
- * 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 DetailRawQueryResultIterator extends AbstractDetailQueryResultIterator {
-
-  private ExecutorService execService = Executors.newFixedThreadPool(1);
-
-  private Future<ResultInfo> future;
-
-  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
-
-  public DetailRawQueryResultIterator(List<BlockExecutionInfo> infos,
-      QueryExecutorProperties executerProperties, QueryModel queryModel,
-      InternalQueryExecutor queryExecutor) {
-    super(infos, executerProperties, queryModel, queryExecutor);
-    this.queryResultPreparator = new RawQueryResultPreparatorImpl(executerProperties, queryModel);
-  }
-
-  @Override public BatchResult next() {
-    BatchResult result;
-    if (future == null) {
-      future = execute();
-    }
-    ResultInfo resultFromFuture = getResultFromFuture(future);
-    result = resultFromFuture.result;
-    currentCounter += resultFromFuture.counter;
-    if (hasNext()) {
-      future = execute();
-    }
-    return result;
-  }
-
-  private ResultInfo getResultFromFuture(Future<ResultInfo> future) {
-    try {
-      return future.get();
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-    return new ResultInfo();
-  }
-
-  private Future<ResultInfo> execute() {
-    return execService.submit(new Callable<ResultInfo>() {
-      @Override public ResultInfo call() {
-        CarbonIterator<Result> result = null;
-        int counter =  updateSliceIndexToBeExecuted();
-        try {
-          result = executor.executeQuery(blockExecutionInfos, blockIndexToBeExecuted);
-        } catch (QueryExecutionException ex) {
-          throw new RuntimeException(ex.getCause());
-        }
-        for (int i = 0; i < blockIndexToBeExecuted.length; i++) {
-          if (blockIndexToBeExecuted[i] != -1) {
-            blockExecutionInfos.get(blockIndexToBeExecuted[i]).setFirstDataBlock(
-                blockExecutionInfos.get(blockIndexToBeExecuted[i]).getFirstDataBlock()
-                    .getNextDataRefNode());
-          }
-        }
-        BatchResult batchResult;
-        if (null != result) {
-          Result next = result.next();
-          batchResult = queryResultPreparator.prepareQueryResult(next);
-        } else {
-          batchResult = queryResultPreparator.prepareQueryResult(null);
-        }
-        ResultInfo resultInfo = new ResultInfo();
-        resultInfo.counter = counter;
-        resultInfo.result = batchResult;
-        return resultInfo;
-      }
-    });
-  }
-
-  private static class ResultInfo {
-    private int counter;
-    private BatchResult result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/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
index 0ae6651..a6a8fbc 100644
--- 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
@@ -4,6 +4,9 @@ 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;
@@ -13,6 +16,8 @@ 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.
@@ -54,29 +59,67 @@ public class RawQueryResultPreparatorImpl
   @Override public BatchResult prepareQueryResult(
       Result<List<ListBasedResultWrapper>, Object> scannedResult) {
     if ((null == scannedResult || scannedResult.size() < 1)) {
-      BatchRawResult batchRawResult = new BatchRawResult();
-      batchRawResult.setQuerySchemaInfo(querySchemaInfo);
-      return batchRawResult;
+      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;
-    while (scannedResult.hasNext()) {
-      value = scannedResult.getValue();
-      row = new Object[msrSize + 1];
-      row[0] = scannedResult.getKey();
-      if(value != null) {
-        System.arraycopy(value, 0, row, 1, msrSize);
+    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++;
       }
-      resultData[counter] = row;
-      counter ++;
     }
+
     LOGGER.info("###########################---- Total Number of records" + scannedResult.size());
     BatchRawResult result = new BatchRawResult();
     result.setRows(resultData);
-    result.setQuerySchemaInfo(querySchemaInfo);
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/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
index 5485f99..2a7c5ae 100644
--- a/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
+++ b/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
@@ -29,6 +29,8 @@ 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
  */
@@ -95,7 +97,7 @@ public class DataTypeUtil {
               new org.apache.spark.sql.types.Decimal();
           return decConverter.set(scalaDecVal);
         default:
-          return data;
+          return UTF8String.fromString(data);
       }
     } catch (NumberFormatException ex) {
       LOGGER.error("Problem while converting data type" + data);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java b/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java
deleted file mode 100644
index a733ec6..0000000
--- a/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java
+++ /dev/null
@@ -1,91 +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.merger.exception;
-
-import java.util.Locale;
-
-public class ResultMergerException extends Exception {
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public ResultMergerException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public ResultMergerException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public ResultMergerException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java b/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java
deleted file mode 100644
index e8289a9..0000000
--- a/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java
+++ /dev/null
@@ -1,91 +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.processor.exception;
-
-import java.util.Locale;
-
-public class DataProcessorException extends Exception {
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public DataProcessorException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public DataProcessorException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public DataProcessorException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java b/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java
deleted file mode 100644
index 691a070..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java
+++ /dev/null
@@ -1,182 +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.queryinterface.filter;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-public class CarbonFilterInfo implements Serializable {
-  private static final long serialVersionUID = -6835223191506253050L;
-
-  protected List<String> excludedMembers;
-
-  protected List<String> includedMembers;
-
-  /**
-   * includedMembers or .
-   */
-  protected List<String> includedOrMembers;
-
-  /**
-   * CarbonFilterInfo.
-   */
-  public CarbonFilterInfo() {
-    super();
-    this.excludedMembers = new ArrayList<String>(10);
-    this.includedMembers = new ArrayList<String>(10);
-    this.includedOrMembers = new ArrayList<String>(10);
-  }
-
-  /**
-   * CarbonFilterInfo.
-   *
-   * @param exludedMembers
-   * @param includedMembers
-   */
-  public CarbonFilterInfo(List<String> exludedMembers, List<String> includedMembers) {
-    super();
-    this.excludedMembers = (null == exludedMembers ? new ArrayList<String>(10) : exludedMembers);
-    this.includedMembers = (null == includedMembers ? new ArrayList<String>(10) : includedMembers);
-    this.includedOrMembers = new ArrayList<String>(10);
-  }
-
-  /**
-   * getExcludedMembers.
-   *
-   * @return List<String>.
-   */
-  public List<String> getExcludedMembers() {
-    return excludedMembers;
-  }
-
-  /**
-   * getIncludedMembers.
-   *
-   * @return List<String>.
-   */
-  public List<String> getIncludedMembers() {
-    return includedMembers;
-  }
-
-  /**
-   * addIncludedMembers.
-   *
-   * @param aMember
-   */
-  public void addIncludedMembers(String aMember) {
-    includedMembers.add(aMember);
-  }
-
-  /**
-   * addExcludedMembers.
-   *
-   * @param aMember
-   */
-  public void addExcludedMembers(String aMember) {
-    excludedMembers.add(aMember);
-  }
-
-  /**
-   * addIncludedMembers.
-   *
-   * @param aMember
-   */
-  public void addAllIncludedMembers(List<String> members) {
-    includedMembers.addAll(members);
-  }
-
-  /**
-   * addExcludedMembers.
-   *
-   * @param aMember
-   */
-  public void addAllExcludedMembers(List<String> members) {
-    excludedMembers.addAll(members);
-  }
-
-  /**
-   * Final filter is a intersection
-   *
-   * @return List<String>.
-   */
-  public List<String> getEffectiveIncludedMembers() {
-
-    List<String> effectiveMems = new ArrayList<String>(includedMembers);
-    effectiveMems.removeAll(excludedMembers);
-    return effectiveMems;
-  }
-
-  /**
-   * getEffectiveExcludedMembers
-   *
-   * @return List<String>.
-   */
-  public List<String> getEffectiveExcludedMembers() {
-    return includedMembers.size() > 0 ? new ArrayList<String>(10) : excludedMembers;
-  }
-
-  /* (non-Javadoc)
-   * @see java.lang.Object#hashCode()
-   */
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((excludedMembers == null) ? 0 : excludedMembers.hashCode());
-    result = prime * result + ((includedMembers == null) ? 0 : includedMembers.hashCode());
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj instanceof CarbonFilterInfo) {
-
-      if (this == obj) {
-        return true;
-      }
-
-      CarbonFilterInfo info = (CarbonFilterInfo) obj;
-
-      if (excludedMembers == null) {
-        if (info.excludedMembers != null) {
-          return false;
-        }
-      } else if (!excludedMembers.equals(info.excludedMembers)) {
-        return false;
-      }
-      if (includedMembers == null) {
-        if (info.includedMembers != null) {
-          return false;
-        }
-      } else if (!includedMembers.equals(info.includedMembers)) {
-        return false;
-      }
-      return true;
-    }
-
-    return false;
-  }
-
-  /**
-   * @return the includedOrMembers
-   */
-  public List<String> getIncludedOrMembers() {
-    return includedOrMembers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java b/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java
deleted file mode 100644
index f65e8d1..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java
+++ /dev/null
@@ -1,233 +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.queryinterface.query;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevel;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevelFilter;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasure;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasureFilter;
-
-public interface CarbonQuery extends Serializable {
-  /**
-   * This property can be set to the extra properties Map
-   * It takes boolean(true or false). it enables/disables pagination.
-   */
-  String PAGINATION_REQUIRED = "PAGINATION_REQUIRED";
-
-  /**
-   * This property can be set to the extra properties Map
-   * It takes string in the following format 0-100 or 1000-1100.
-   * This property can be taken only if PAGINATION_REQUIRED set to true;
-   */
-  String PAGE_RANGE = "PAGE_RANGE";
-
-  /**
-   * This property can be set to the extra properties Map
-   * It takes unique string and also this property can be taken only if PAGINATION_REQUIRED
-   * set to true;.
-   */
-  String QUERY_ID = "QUERY_ID";
-
-  /**
-   * Property can be set to save the result as a Data Set
-   */
-  String DATA_SET_PATH = "DATA_SET_PATH";
-
-  /**
-   * Property can be set to configure the transformations in the query
-   */
-  String TRANSFORMATIONS = "TRANSFORMATIONS";
-
-  /**
-   * Add dimension levels to the query
-   *
-   * @param dimensionLevel
-   * @param axis
-   */
-  void addDimensionLevel(CarbonDimensionLevel dimensionLevel, CarbonDimensionLevelFilter filter,
-      SortType sortType, AxisType axis);
-
-  /**
-   * Add measure to the query
-   *
-   * @param measure
-   * @param axis
-   */
-  void addMeasure(CarbonMeasure measure, List<CarbonMeasureFilter> filters, SortType sortType);
-
-  /**
-   * Add level filter to the query. If this dimension level is already added to any axis,then no
-   * need to add again.
-   *
-   * @param dimensionLevel
-   */
-  void addSlice(CarbonDimensionLevel dimensionLevel, CarbonDimensionLevelFilter filter);
-
-  /**
-   * Add level filter to the query. If this measure is already added to any axis,then no need
-   * to add again.
-   *
-   * @param CarbonMeasure measure
-   */
-  void addSlice(CarbonMeasure measure, List<CarbonMeasureFilter> filters);
-
-  /**
-   * Add top count to the query
-   *
-   * @param dimensionLevel
-   * @param measure
-   * @param count
-   */
-  void addTopCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure, int count);
-
-  /**
-   * Add bottom count to the query.
-   *
-   * @param dimensionLevel
-   * @param measure
-   * @param count
-   */
-  void addBottomCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure, int count);
-
-  /**
-   * Whether to show dimension properties or not.
-   *
-   * @param showProperties
-   */
-  void showLevelProperties(boolean showProperties);
-
-  /**
-   * When it set as true then user needs to provide the filters exactly with there parent members.
-   * For example : To apply day level filter as 1 then he has to provide like [2000].[jan].[1].
-   * Now it exactly fetches the data for that day
-   * If it is false then he can provide just like [1].
-   * But this will fetch the data for the day 1 of all months
-   * and years.
-   *
-   * @param exactLevelsMatch. By default it is false.
-   */
-  void setExactHirarchyLevelsMatch(boolean exactLevelsMatch);
-
-  /**
-   * This is properties will available to the execution.
-   * This is only used from Analyzer client purpose.
-   *
-   * @param extraProperties
-   */
-  void setExtraProperties(Map<String, Object> extraProperties);
-
-  /**
-   * When this property sets, it converts the data to the target time zone.
-   * By default there is no time zone set. When this property sets, even the time filters
-   * passed through this interface will be converted from this timezone.
-   * For example : timezone sets UTC-12 and filters passed are Jan 1 23:00 then it
-   * converts filter to Jan 2 11:00.
-   *
-   * @param timeZone
-   */
-  void setTimeZone(String timeZone);
-
-  /**
-   * Axis
-   */
-  public enum AxisType {
-    /**
-     * Row axis
-     */
-    ROW(0),
-    /**
-     * Column axis
-     */
-    COLUMN(1),
-
-    /**
-     * SLICE
-     */
-    SLICE(2);
-
-    /**
-     * index
-     */
-    private int index;
-
-    /**
-     * Get axis type
-     *
-     * @param index
-     */
-    private AxisType(int index) {
-      this.index = index;
-    }
-
-    /**
-     * Get axis index
-     *
-     * @return index
-     */
-    public int getIndex() {
-      return index;
-    }
-  }
-
-  /**
-   * Sort type
-   */
-  public enum SortType {
-    /**
-     * Ascending order
-     */
-    ASC(0),
-    /**
-     * Descending order
-     */
-    DESC(1),
-    /**
-     * Ascending order
-     */
-    BASC(2),
-    /**
-     * Descending order
-     */
-    BDESC(3),
-    /**
-     * None
-     */
-    NONE(-1);
-
-    private int sortVal;
-
-    SortType(int sortVal) {
-      this.sortVal = sortVal;
-    }
-
-    /**
-     * getSortValue
-     *
-     * @return
-     */
-    public int getSortValue() {
-      return sortVal;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java b/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java
deleted file mode 100644
index b12057f..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java
+++ /dev/null
@@ -1,223 +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.queryinterface.query.impl;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.query.queryinterface.query.CarbonQuery;
-import org.carbondata.query.queryinterface.query.metadata.Axis;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevel;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevelFilter;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasure;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasureFilter;
-import org.carbondata.query.queryinterface.query.metadata.TopCount;
-
-/**
- * It is the implementation class for CarbonQuery interface.
- */
-public class CarbonQueryImpl implements CarbonQuery {
-  private static final long serialVersionUID = -1565369538375956018L;
-
-  /**
-   * Slice number
-   */
-  private static final int SLICE = 2;
-
-  /**
-   * axises
-   */
-  private Axis[] axises;
-
-  /**
-   * Top count list
-   */
-  private List<TopCount> topCounts = new ArrayList<TopCount>(10);
-
-  /**
-   * propertiesRequired
-   */
-  private boolean propertiesRequired;
-
-  /**
-   * When it set as true then user needs to provide the filters exactly with there parent members.
-   */
-  private boolean exactLevelsMatch;
-
-  /**
-   * breakHierarchyTopN
-   */
-  private boolean breakHierarchyTopN;
-
-  /**
-   * Time zone to covert the data.
-   */
-  private String timeZone;
-
-  private Map<String, Object> extraProperties = new HashMap<String, Object>(16);
-
-  /**
-   * Default constructor
-   */
-  public CarbonQueryImpl() {
-    axises = new Axis[3];
-    for (int i = 0; i < axises.length; i++) {
-      axises[i] = new Axis();
-    }
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addDimensionLevel(CarbonDimensionLevel dimensionLevel,
-      CarbonDimensionLevelFilter filter, SortType sortType, AxisType axis) {
-    sortType = sortType == null ? SortType.NONE : sortType;
-    axises[axis.getIndex()].add(dimensionLevel, sortType, null, filter);
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addMeasure(CarbonMeasure measure, List<CarbonMeasureFilter> filters,
-      SortType sortType) {
-    sortType = sortType == null ? SortType.NONE : sortType;
-    axises[AxisType.COLUMN.getIndex()].add(measure, sortType, filters, null);
-
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addSlice(CarbonDimensionLevel dimensionLevel,
-      CarbonDimensionLevelFilter filter) {
-    axises[SLICE].add(dimensionLevel, null, null, filter);
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addSlice(CarbonMeasure measure, List<CarbonMeasureFilter> filters) {
-    axises[SLICE].add(measure, null, filters, null);
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addTopCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure,
-      int count) {
-    topCounts.add(new TopCount(dimensionLevel, measure, count, TopCount.TopNType.TOP));
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addBottomCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure,
-      int count) {
-    topCounts.add(new TopCount(dimensionLevel, measure, count, TopCount.TopNType.BOTTOM));
-  }
-
-  /**
-   * @return the axises
-   */
-  public Axis[] getAxises() {
-    return axises;
-  }
-
-  /**
-   * @return the topCounts
-   */
-  public List<TopCount> getTopCounts() {
-    return topCounts;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public void showLevelProperties(boolean showProerties) {
-    propertiesRequired = showProerties;
-  }
-
-  /**
-   * Whether can show properties or not.
-   *
-   * @return
-   */
-  public boolean isShowLevelProperties() {
-    return propertiesRequired;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public void setExactHirarchyLevelsMatch(boolean exactLevelsMatch) {
-    this.exactLevelsMatch = exactLevelsMatch;
-  }
-
-  /**
-   * @return the exactLevelsMatch
-   */
-  public boolean isExactLevelsMatch() {
-    return exactLevelsMatch;
-  }
-
-  /**
-   * @return the extraProperties
-   */
-  public Map<String, Object> getExtraProperties() {
-    return extraProperties;
-  }
-
-  @Override public void setExtraProperties(Map<String, Object> extraProperties) {
-    this.extraProperties = extraProperties;
-  }
-
-  /**
-   * @return the breakHierarchyTopN
-   */
-  public boolean isBreakHierarchyTopN() {
-    return breakHierarchyTopN;
-  }
-
-  /**
-   * @param breakHierarchyTopN the breakHierarchyTopN to set
-   */
-  public void setBreakHierarchyTopN(boolean breakHierarchyTopN) {
-    this.breakHierarchyTopN = breakHierarchyTopN;
-  }
-
-  /**
-   * getTimeZone
-   *
-   * @return
-   */
-  public String getTimeZone() {
-    return this.timeZone;
-  }
-
-  /**
-   * setTimeZone
-   */
-  @Override public void setTimeZone(String timeZone) {
-    this.timeZone = timeZone;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java
deleted file mode 100644
index 9ee9edf..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.queryinterface.query.metadata;
-
-/**
- * It is abstract class for CarbonLevel interface.
- */
-public abstract class AbstractCarbonLevel implements CarbonLevel {
-  private static final long serialVersionUID = -1487270452433379657L;
-
-  private boolean visible = true;
-
-  /**
-   * See interface comments
-   */
-  @Override public String getDimensionName() {
-    return null;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public String getHierarchyName() {
-    return null;
-  }
-
-  /**
-   * @return the visible
-   */
-  public boolean isVisible() {
-    return visible;
-  }
-
-  /**
-   * @param visible the visible to set
-   */
-  public void setVisible(boolean visible) {
-    this.visible = visible;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java
deleted file mode 100644
index 711fab8..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java
+++ /dev/null
@@ -1,61 +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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.query.queryinterface.query.CarbonQuery;
-
-/**
- * It is Axis class, it can be row,column or slice axis.It contains all information of
- * query depends on levels and measures added in query.
- */
-public class Axis implements Serializable {
-  private static final long serialVersionUID = -574689684553603640L;
-
-  private List<CarbonLevelHolder> dims = new ArrayList<CarbonLevelHolder>(10);
-
-  /**
-   * Add query details to this axis.
-   *
-   * @param level
-   * @param sortType
-   * @param msrFilters
-   * @param dimLevelFilter
-   */
-  public void add(CarbonLevel level, CarbonQuery.SortType sortType,
-      List<CarbonMeasureFilter> msrFilters, CarbonDimensionLevelFilter dimLevelFilter) {
-    CarbonLevelHolder holder = new CarbonLevelHolder(level, sortType);
-    holder.setMsrFilters(msrFilters);
-    holder.setDimLevelFilter(dimLevelFilter);
-    dims.add(holder);
-  }
-
-  /**
-   * Get dims
-   *
-   * @return the dims
-   */
-  public List<CarbonLevelHolder> getDims() {
-    return dims;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java
deleted file mode 100644
index 7f2c0cf..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java
+++ /dev/null
@@ -1,99 +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.queryinterface.query.metadata;
-
-/**
- * Calculated measures can be created by using this class
- */
-public class CarbonCalculatedMeasure extends CarbonMeasure {
-  private static final long serialVersionUID = 4176313704077360543L;
-  private String expression;
-  private boolean groupCount;
-  private String groupDimensionFormula;
-  private CarbonDimensionLevel groupDimensionLevel;
-
-  public CarbonCalculatedMeasure(String measureName, String expr) {
-    super(measureName);
-  }
-
-  /**
-   * @return the expression
-   */
-  public String getExpression() {
-    return expression;
-  }
-
-  /**
-   * @param expression the expression to set
-   */
-  public void setExpression(String expression) {
-    this.expression = expression;
-  }
-
-  /**
-   * @return Returns the groupCount.
-   */
-  public boolean isGroupCount() {
-    return groupCount;
-  }
-
-  /**
-   * @param groupCount The groupCount to set.
-   */
-  public void setGroupCount(boolean groupCount) {
-    this.groupCount = groupCount;
-  }
-
-  /**
-   * @return Returns the groupDimensionFormula.
-   */
-  public String getGroupDimensionFormula() {
-    return groupDimensionFormula;
-  }
-
-  /**
-   * @param groupDimensionFormula The groupDimensionFormula to set.
-   */
-  public void setGroupDimensionFormula(String groupDimensionFormula) {
-    this.groupDimensionFormula = groupDimensionFormula;
-  }
-
-  /**
-   * @return Returns the groupDimensionLevel.
-   */
-  public CarbonDimensionLevel getGroupDimensionLevel() {
-    return groupDimensionLevel;
-  }
-
-  /**
-   * @param groupDimensionLevel The groupDimensionLevel to set.
-   */
-  public void setGroupDimensionLevel(CarbonDimensionLevel groupDimensionLevel) {
-    this.groupDimensionLevel = groupDimensionLevel;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public CarbonLevelType getType() {
-    return CarbonLevelType.CALCULATED_MEASURE;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java
deleted file mode 100644
index a2fcac6..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java
+++ /dev/null
@@ -1,81 +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.queryinterface.query.metadata;
-
-/**
- * Carbon dimension level;
- */
-public class CarbonDimensionLevel extends AbstractCarbonLevel {
-  private static final long serialVersionUID = 4012085091766940643L;
-
-  /**
-   * Dimension name
-   */
-  private String dimensionName;
-
-  /**
-   * Hierarchy name
-   */
-  private String hierarchyName;
-
-  /**
-   * level name
-   */
-  private String levelName;
-
-  /**
-   * Constructor
-   *
-   * @param dimensionName
-   * @param hierarchyName
-   * @param levelName
-   */
-  public CarbonDimensionLevel(String dimensionName, String hierarchyName, String levelName) {
-    this.dimensionName = dimensionName;
-    this.hierarchyName = hierarchyName;
-    this.levelName = levelName;
-  }
-
-  /**
-   * @return the dimensionName
-   */
-  public String getDimensionName() {
-    return dimensionName;
-  }
-
-  /**
-   * @return the hierarchyName
-   */
-  public String getHierarchyName() {
-    return hierarchyName;
-  }
-
-  /**
-   * @return the levelName
-   */
-  public String getName() {
-    return levelName;
-  }
-
-  @Override public CarbonLevelType getType() {
-
-    return CarbonLevelType.DIMENSION;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java
deleted file mode 100644
index 6c85b6e..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java
+++ /dev/null
@@ -1,217 +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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Level filter
- */
-public class CarbonDimensionLevelFilter implements Serializable {
-  private static final long serialVersionUID = 5028332445998450964L;
-
-  /**
-   * Include filter
-   */
-  private List<Object> includeFilter = new ArrayList<Object>(10);
-
-  /**
-   * Exclude filter
-   */
-  private List<Object> excludeFilter = new ArrayList<Object>(10);
-
-  /**
-   * Contains filter
-   */
-  private List<String> containsFilter = new ArrayList<String>(10);
-
-  /**
-   * Does not contain filter
-   */
-  private List<String> doesNotContainsFilter = new ArrayList<String>(10);
-
-  /**
-   * afterTopN
-   */
-  private boolean afterTopN;
-
-  /**
-   * @return the includeFilter
-   */
-  public List<Object> getIncludeFilter() {
-    return includeFilter;
-  }
-
-  /**
-   * Add filters append  with braces []. Like [2000]
-   *
-   * @param includeFilter the includeFilter to set
-   */
-  public void setIncludeFilter(List<Object> includeFilter) {
-    this.includeFilter = includeFilter;
-  }
-
-  /**
-   * @return the excludeFilter
-   */
-  public List<Object> getExcludeFilter() {
-    return excludeFilter;
-  }
-
-  /**
-   * Add filters append  with braces []. Like [2000]
-   *
-   * @param excludeFilter the excludeFilter to set
-   */
-  public void setExcludeFilter(List<Object> excludeFilter) {
-    this.excludeFilter = excludeFilter;
-  }
-
-  /**
-   * @return the containsFilter
-   */
-  public List<String> getContainsFilter() {
-    return containsFilter;
-  }
-
-  /**
-   * This filter does not work along with CarbonQuery.setExactHirarchyLevelsMatch set as true.
-   *
-   * @param containsFilter the containsFilter to set
-   */
-  public void setContainsFilter(List<String> containsFilter) {
-    this.containsFilter = containsFilter;
-  }
-
-  /**
-   * @return the doesNotContainsFilter
-   */
-  public List<String> getDoesNotContainsFilter() {
-    return doesNotContainsFilter;
-  }
-
-  /**
-   * @param doesNotContainsFilter the doesNotContainsFilter to set
-   */
-  public void setDoesNotContainsFilter(List<String> doesNotContainsFilter) {
-    this.doesNotContainsFilter = doesNotContainsFilter;
-  }
-
-  /**
-   * @return the afterTopN
-   */
-  public boolean isAfterTopN() {
-    return afterTopN;
-  }
-
-  /**
-   * @param afterTopN the afterTopN to set
-   */
-  public void setAfterTopN(boolean afterTopN) {
-    this.afterTopN = afterTopN;
-  }
-
-  public String toSQLConstruct(String levelName) {
-    StringBuffer buffer = new StringBuffer();
-
-    boolean appendAndRequired = false;
-
-    // Include filters list
-    if (includeFilter.size() > 1) {
-      buffer.append(levelName);
-      buffer.append(" IN ( ");
-      for (int i = 0; i < includeFilter.size(); i++) {
-        buffer.append("'" + includeFilter.get(i) + "'");
-        if (i != includeFilter.size() - 1) {
-          buffer.append(" , ");
-        }
-      }
-      buffer.append(" ) ");
-
-      appendAndRequired = true;
-    } else if (includeFilter.size() > 0) {
-      buffer.append(levelName);
-      buffer.append(" = '" + includeFilter.get(0) + "'");
-
-      appendAndRequired = true;
-    }
-
-    // Exclude filters list
-    if (excludeFilter.size() > 1) {
-      if (appendAndRequired) {
-        buffer.append(" AND ");
-      }
-      buffer.append(levelName);
-      buffer.append(" NOT IN (");
-      for (int i = 0; i < excludeFilter.size(); i++) {
-        buffer.append("'" + excludeFilter.get(i) + "'");
-        if (i != excludeFilter.size() - 1) {
-          buffer.append(" , ");
-        }
-      }
-      buffer.append(" ) ");
-
-      appendAndRequired = true;
-    } else if (excludeFilter.size() > 0) {
-      if (appendAndRequired) {
-        buffer.append(" AND ");
-      }
-      buffer.append(levelName);
-      buffer.append(" != '" + excludeFilter.get(0) + "'");
-
-      appendAndRequired = true;
-    }
-
-    // Contains filters list
-    if (containsFilter.size() > 0) {
-
-      for (String containsString : containsFilter) {
-        if (appendAndRequired) {
-          buffer.append(" AND ");
-        }
-
-        buffer.append(levelName);
-        buffer.append(" LIKE ");
-        buffer.append("'" + containsString + "'");
-
-        appendAndRequired = true;
-      }
-
-    }
-
-    //Doesn't contain filter
-    if (doesNotContainsFilter.size() > 0) {
-
-      for (String containsString : doesNotContainsFilter) {
-        if (appendAndRequired) {
-          buffer.append(" AND ");
-        }
-        buffer.append(levelName);
-        buffer.append(" NOT LIKE ");
-        buffer.append("'" + containsString + "'");
-        appendAndRequired = true;
-      }
-
-    }
-    return buffer.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.java
deleted file mode 100644
index 62b1180..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * It is level interface for CARBON dimension level and measure.
- */
-public interface CarbonLevel extends Serializable {
-
-  /**
-   * Type of level, either dimension level or measure.
-   *
-   * @return CarbonLevelType
-   */
-  CarbonLevelType getType();
-
-  /**
-   * Dimension name of the level it belonged to.
-   *
-   * @return the dimensionName
-   */
-  String getDimensionName();
-
-  /**
-   * Hierarchy name of the level it belonged to.
-   *
-   * @return the hierarchyName
-   */
-  String getHierarchyName();
-
-  /**
-   * Name of dimension level or measure
-   *
-   * @return the levelName
-   */
-  String getName();
-
-  /**
-   * Type of dimension level, either level or measure
-   *
-   * @author R00900208
-   */
-  public enum CarbonLevelType {
-    /**
-     * DIMENSION
-     */
-    DIMENSION,
-
-    /**
-     * MEASURE
-     */
-    MEASURE,
-
-    /**
-     * MEASURE
-     */
-    CALCULATED_MEASURE,
-
-    /**
-     * DYNAMIC LEVEL
-     */
-    DYNAMIC_DIMENSION;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java
deleted file mode 100644
index b6bc13e..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java
+++ /dev/null
@@ -1,113 +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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.carbondata.query.queryinterface.query.CarbonQuery;
-
-/**
- * It is holder class for a level
- */
-public class CarbonLevelHolder implements Serializable {
-  private static final long serialVersionUID = -6328136034161360231L;
-
-  /**
-   * Level
-   */
-  private CarbonLevel level;
-
-  /**
-   * sortType
-   */
-  private CarbonQuery.SortType sortType;
-
-  /**
-   * msrFilters
-   */
-  private List<CarbonMeasureFilter> msrFilters;
-
-  /**
-   * dimLevelFilter
-   */
-  private CarbonDimensionLevelFilter dimLevelFilter;
-
-  /**
-   * Constructor
-   *
-   * @param level
-   * @param sortType
-   */
-  public CarbonLevelHolder(CarbonLevel level, CarbonQuery.SortType sortType) {
-    super();
-    this.level = level;
-    this.sortType = sortType;
-  }
-
-  /**
-   * @return the level
-   */
-  public CarbonLevel getLevel() {
-    return level;
-  }
-
-  /**
-   * @return the sortType
-   */
-  public CarbonQuery.SortType getSortType() {
-    return sortType;
-  }
-
-  /**
-   * @param sortType the sortType to set
-   */
-  public void setSortType(CarbonQuery.SortType sortType) {
-    this.sortType = sortType;
-  }
-
-  /**
-   * @return the msrFilter
-   */
-  public List<CarbonMeasureFilter> getMsrFilters() {
-    return msrFilters;
-  }
-
-  /**
-   * @param msrFilter the msrFilter to set
-   */
-  public void setMsrFilters(List<CarbonMeasureFilter> msrFilters) {
-    this.msrFilters = msrFilters;
-  }
-
-  /**
-   * @return the dimLevelFilter
-   */
-  public CarbonDimensionLevelFilter getDimLevelFilter() {
-    return dimLevelFilter;
-  }
-
-  /**
-   * @param dimLevelFilter the dimLevelFilter to set
-   */
-  public void setDimLevelFilter(CarbonDimensionLevelFilter dimLevelFilter) {
-    this.dimLevelFilter = dimLevelFilter;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java
deleted file mode 100644
index e96f8a2..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java
+++ /dev/null
@@ -1,81 +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.queryinterface.query.metadata;
-
-/**
- * Carbon Measure class
- */
-public class CarbonMeasure extends AbstractCarbonLevel {
-  private static final long serialVersionUID = 4257185028603048687L;
-
-  /**
-   * Measure name
-   */
-  private String measureName;
-
-  private CarbonDimensionLevel dimensionLevel;
-
-  /**
-   * Constructor
-   *
-   * @param measureName
-   */
-  public CarbonMeasure(String measureName) {
-    this.measureName = measureName;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param measureName
-   * @param aggregateName
-   */
-  public CarbonMeasure(String measureName, String aggregateName) {
-    this.measureName = measureName;
-  }
-
-  /**
-   * @return the measureName
-   */
-  public String getName() {
-    return measureName;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public CarbonLevelType getType() {
-    return CarbonLevelType.MEASURE;
-  }
-
-  /**
-   * @return the dimensionLevel
-   */
-  public CarbonDimensionLevel getDimensionLevel() {
-    return dimensionLevel;
-  }
-
-  /**
-   * @param dimensionLevel the dimensionLevel to set
-   */
-  public void setDimensionLevel(CarbonDimensionLevel dimensionLevel) {
-    this.dimensionLevel = dimensionLevel;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java
deleted file mode 100644
index 7c89dac..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java
+++ /dev/null
@@ -1,142 +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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * Measure filter
- */
-public class CarbonMeasureFilter implements Serializable {
-  private static final long serialVersionUID = -4253090536204072658L;
-
-  /**
-   * filterValue
-   */
-  private double filterValue;
-
-  /**
-   * filterType
-   */
-  private FilterType filterType;
-
-  /**
-   * afterTopN
-   */
-  private boolean afterTopN;
-
-  /**
-   * Constructor that takes filter information for measure filter.
-   *
-   * @param filterValue
-   * @param filterType
-   */
-  public CarbonMeasureFilter(double filterValue, FilterType filterType) {
-    this.filterValue = filterValue;
-    this.filterType = filterType;
-  }
-
-  /**
-   * Constructor that takes filter information for measure filter.
-   *
-   * @param filterType
-   */
-  public CarbonMeasureFilter(FilterType filterType) {
-    this.filterType = filterType;
-  }
-
-  /**
-   * get FilterValue
-   *
-   * @return the filterValue
-   */
-  public double getFilterValue() {
-    return filterValue;
-  }
-
-  /**
-   * FilterType
-   *
-   * @return the filterType
-   */
-  public FilterType getFilterType() {
-    return filterType;
-  }
-
-  /**
-   * @return the afterTopN
-   */
-  public boolean isAfterTopN() {
-    return afterTopN;
-  }
-
-  /**
-   * @param afterTopN the afterTopN to set
-   */
-  public void setAfterTopN(boolean afterTopN) {
-    this.afterTopN = afterTopN;
-  }
-
-  public String toSQLConstruct(String levelName) {
-    return levelName + filterType.symbol + filterValue;
-  }
-
-  /**
-   * '
-   * Enum for measure filter types.
-   */
-  public enum FilterType {
-    /**
-     * EQUAL_TO
-     */
-    EQUAL_TO(" = "),
-    /**
-     * NOT_EQUAL_TO
-     */
-    NOT_EQUAL_TO(" != "),
-    /**
-     * GREATER_THAN
-     */
-    GREATER_THAN(" > "),
-    /**
-     * LESS_THAN
-     */
-    LESS_THAN(" < "),
-    /**
-     * LESS_THAN_EQUAL
-     */
-    LESS_THAN_EQUAL(" <= "),
-    /**
-     * GREATER_THAN_EQUAL
-     */
-    GREATER_THAN_EQUAL(" >= "),
-
-    /**
-     * NOT_EMPTY
-     */
-    NOT_EMPTY(" IS NOT NULL ");
-
-    String symbol;
-
-    FilterType(String symbol) {
-      this.symbol = symbol;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java
deleted file mode 100644
index 294567c..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java
+++ /dev/null
@@ -1,92 +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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * It is the Member object which holds information of each member which contained in query result.
- */
-public class CarbonMember implements Serializable {
-  private static final long serialVersionUID = 2149598237303284053L;
-
-  private Object name;
-
-  private Object[] properties;
-
-  /**
-   * Constructor that takes filter information for each member.
-   *
-   * @param name
-   * @param properties
-   */
-  public CarbonMember(Object name, Object[] properties) {
-    this.name = name;
-    this.properties = properties;
-  }
-
-  /**
-   * @return the name
-   */
-  public Object getName() {
-    return name;
-  }
-
-  /**
-   * @return the properties
-   */
-  public Object[] getProperties() {
-    return properties;
-  }
-
-  /**
-   * @return the properties
-   */
-  public void setProperties(Object[] props) {
-    this.properties = props;
-  }
-
-  @Override public String toString() {
-    return name != null ? name.toString() : "";
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((name == null) ? 0 : name.hashCode());
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj instanceof CarbonMember) {
-      if (this == obj) {
-        return true;
-      }
-
-      CarbonMember other = (CarbonMember) obj;
-      if (!(name == null ? other.name == null : name.equals(other.name))) {
-        return false;
-      }
-      return true;
-
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java
deleted file mode 100644
index 048b7b8..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java
+++ /dev/null
@@ -1,76 +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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-/**
- * CarbonTuple class , it contains the each row or column information of query result.
- */
-public class CarbonTuple implements Serializable {
-  private static final long serialVersionUID = 6432454407461679716L;
-
-  private CarbonMember[] tuple;
-
-  public CarbonTuple(CarbonMember[] tuple) {
-    this.tuple = tuple;
-  }
-
-  /**
-   * Size of tuple.
-   *
-   * @return
-   */
-  public int size() {
-    return tuple.length;
-  }
-
-  /**
-   * Get all members inside tuple.
-   *
-   * @return
-   */
-  public CarbonMember[] getTuple() {
-    return tuple;
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + Arrays.hashCode(tuple);
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj instanceof CarbonTuple) {
-      if (this == obj) {
-        return true;
-      }
-      CarbonTuple other = (CarbonTuple) obj;
-      if (!Arrays.equals(tuple, other.tuple)) {
-        return false;
-      }
-      return true;
-
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.java
deleted file mode 100644
index b8f0eff..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.java
+++ /dev/null
@@ -1,111 +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.queryinterface.query.metadata;
-
-/**
- * Represents DSL transformation which can be added to <code>CarbonQuery<code>
- */
-public class DSLTransformation {
-  /**
-   * Name of the transformation.
-   */
-  private String name;
-
-  /**
-   * DSL script
-   */
-  private String dslExpression;
-
-  /**
-   * The new column name if the DSL script is adding one new column
-   */
-  private String newColumnName;
-
-  /**
-   * Flag to set if the transformation script will resulting to add a new column in
-   * the original result.
-   */
-  private boolean addAsColumn;
-
-  public DSLTransformation(String name, String dslExpression, String newColumnName,
-      boolean addAsColumn) {
-    this.name = name;
-    this.dslExpression = dslExpression;
-    this.newColumnName = newColumnName;
-    this.addAsColumn = addAsColumn;
-  }
-
-  /**
-   * @return Returns the name.
-   */
-  public String getName() {
-    return name;
-  }
-
-  /**
-   * @param name The name to set.
-   */
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /**
-   * @return Returns the dslExpression.
-   */
-  public String getDslExpression() {
-    return dslExpression;
-  }
-
-  /**
-   * @param dslExpression The dslExpression to set.
-   */
-  public void setDslExpression(String dslExpression) {
-    this.dslExpression = dslExpression;
-  }
-
-  /**
-   * @return Returns the newColumnName.
-   */
-  public String getNewColumnName() {
-    return newColumnName;
-  }
-
-  /**
-   * @param newColumnName The newColumnName to set.
-   */
-  public void setNewColumnName(String newColumnName) {
-    this.newColumnName = newColumnName;
-  }
-
-  /**
-   * @return Returns the addAsColumn.
-   */
-  public boolean isAddAsColumn() {
-    return addAsColumn;
-  }
-
-  /**
-   * @param addAsColumn The addAsColumn to set.
-   */
-  public void setAddAsColumn(boolean addAsColumn) {
-    this.addAsColumn = addAsColumn;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java
deleted file mode 100644
index 71f83da..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java
+++ /dev/null
@@ -1,110 +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.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * It is top count meta class
- */
-public class TopCount implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = -8571684898961076954L;
-
-  /**
-   * CarbonDimensionLevel
-   */
-  private CarbonDimensionLevel level;
-
-  /**
-   * Measure
-   */
-  private CarbonMeasure msr;
-
-  /**
-   * TopN count
-   */
-  private int count;
-
-  /**
-   * TopN type
-   */
-  private TopNType type;
-
-  public TopCount(CarbonDimensionLevel level, CarbonMeasure msr, int count, TopNType type) {
-    this.level = level;
-    this.msr = msr;
-    this.count = count;
-    this.type = type;
-  }
-
-  /**
-   * Get level
-   *
-   * @return the level
-   */
-  public CarbonDimensionLevel getLevel() {
-    return level;
-  }
-
-  /**
-   * get measure
-   *
-   * @return the msr
-   */
-  public CarbonMeasure getMsr() {
-    return msr;
-  }
-
-  /**
-   * Get top count
-   *
-   * @return the count
-   */
-  public int getCount() {
-    return count;
-  }
-
-  /**
-   * Get the topn type
-   *
-   * @return the type
-   */
-  public TopNType getType() {
-    return type;
-  }
-
-  /**
-   * Enum for TopN types
-   */
-  public enum TopNType {
-    /**
-     * Top
-     */
-    TOP,
-    /**
-     * Bottom
-     */
-    BOTTOM;
-  }
-}
-