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:15 UTC

[27/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
new file mode 100644
index 0000000..d09bab5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.filter.resolver;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.SortedMap;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.conditional.BinaryConditionalExpression;
+import org.carbondata.scan.expression.logical.BinaryLogicalExpression;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+
+public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverImpl {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 6629319265336666789L;
+
+  private List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
+  private List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
+  private AbsoluteTableIdentifier tableIdentifier;
+
+  public RowLevelRangeFilterResolverImpl(Expression exp, boolean isExpressionResolve,
+      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
+    super(exp, isExpressionResolve, isIncludeFilter);
+    dimColEvaluatorInfoList =
+        new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(
+        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.tableIdentifier = tableIdentifier;
+  }
+
+  /**
+   * This method will return the filter values which is present in the range leve
+   * conditional expressions.
+   *
+   * @return
+   */
+  public byte[][] getFilterRangeValues() {
+    List<byte[]> filterValuesList = new ArrayList<byte[]>();
+    if (null != dimColEvaluatorInfoList.get(0).getFilterValues()) {
+      filterValuesList =
+          dimColEvaluatorInfoList.get(0).getFilterValues().getNoDictionaryFilterValuesList();
+      return filterValuesList.toArray((new byte[filterValuesList.size()][]));
+    }
+    return filterValuesList.toArray((new byte[filterValuesList.size()][]));
+
+  }
+
+  /**
+   * method will get the start key based on the filter surrogates
+   *
+   * @return start IndexKey
+   */
+  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
+      SortedMap<Integer, byte[]> noDictStartKeys) {
+    if (null == dimColEvaluatorInfoList.get(0).getStarIndexKey()) {
+      FilterUtil
+          .getStartKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
+              noDictStartKeys);
+    }
+  }
+
+  /**
+   * method will get the start key based on the filter surrogates
+   *
+   * @return end IndexKey
+   */
+  @Override public void getEndKey(SegmentProperties segmentProperties,
+      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
+      SortedMap<Integer, byte[]> noDicEndKeys) {
+    if (null == dimColEvaluatorInfoList.get(0).getEndIndexKey()) {
+      try {
+        FilterUtil.getEndKey(dimColEvaluatorInfoList.get(0).getDimensionResolvedFilterInstance(),
+            absoluteTableIdentifier, endKeys, segmentProperties);
+        FilterUtil
+            .getEndKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
+                noDicEndKeys);
+      } catch (QueryExecutionException e) {
+        // TODO Auto-generated catch block
+        e.printStackTrace();
+      }
+    }
+  }
+
+  private List<byte[]> getNoDictionaryRangeValues() {
+    List<ExpressionResult> listOfExpressionResults = new ArrayList<ExpressionResult>(20);
+    if (this.getFilterExpression() instanceof BinaryConditionalExpression) {
+      listOfExpressionResults =
+          ((BinaryConditionalExpression) this.getFilterExpression()).getLiterals();
+    }
+    List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
+    for (ExpressionResult result : listOfExpressionResults) {
+      if (result.getString() == null) {
+        filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL.getBytes());
+        continue;
+      }
+      filterValuesList.add(result.getString().getBytes());
+    }
+    Comparator<byte[]> filterNoDictValueComaparator = new Comparator<byte[]>() {
+      @Override public int compare(byte[] filterMember1, byte[] filterMember2) {
+        return ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterMember1, filterMember2);
+      }
+
+    };
+    Collections.sort(filterValuesList, filterNoDictValueComaparator);
+    return filterValuesList;
+  }
+
+  /**
+   * Method which will resolve the filter expression by converting the filter
+   * member to its assigned dictionary values.
+   */
+  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+    DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = null;
+    MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = null;
+    int index = 0;
+    if (exp instanceof BinaryLogicalExpression) {
+      BinaryLogicalExpression conditionalExpression = (BinaryLogicalExpression) exp;
+      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
+      for (ColumnExpression columnExpression : columnList) {
+        if (columnExpression.isDimension()) {
+          dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
+          DimColumnFilterInfo filterInfo = new DimColumnFilterInfo();
+          dimColumnEvaluatorInfo.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
+          //dimColumnEvaluatorInfo.se
+          dimColumnEvaluatorInfo.setRowIndex(index++);
+          dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
+          dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
+          filterInfo.setFilterListForNoDictionaryCols(getNoDictionaryRangeValues());
+          filterInfo.setIncludeFilter(isIncludeFilter);
+          dimColumnEvaluatorInfo.setFilterValues(filterInfo);
+          dimColumnEvaluatorInfo
+              .addDimensionResolvedFilterInstance(columnExpression.getDimension(), filterInfo);
+          dimColEvaluatorInfoList.add(dimColumnEvaluatorInfo);
+        } else {
+          msrColumnEvalutorInfo = new MeasureColumnResolvedFilterInfo();
+          msrColumnEvalutorInfo.setRowIndex(index++);
+          msrColumnEvalutorInfo.setAggregator(
+              ((CarbonMeasure) columnExpression.getCarbonColumn()).getAggregateFunction());
+          msrColumnEvalutorInfo
+              .setColumnIndex(((CarbonMeasure) columnExpression.getCarbonColumn()).getOrdinal());
+          msrColumnEvalutorInfo.setType(columnExpression.getCarbonColumn().getDataType());
+          msrColEvalutorInfoList.add(msrColumnEvalutorInfo);
+        }
+      }
+    }
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which consists
+   * the mapping of the respective dimension and its surrogates involved in
+   * filter expression.
+   *
+   * @return DimColumnResolvedFilterInfo
+   */
+  public List<DimColumnResolvedFilterInfo> getDimColEvaluatorInfoList() {
+    return dimColEvaluatorInfoList;
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which containts
+   * measure level details.
+   *
+   * @return MeasureColumnResolvedFilterInfo
+   */
+  public List<MeasureColumnResolvedFilterInfo> getMsrColEvalutorInfoList() {
+    return msrColEvalutorInfoList;
+  }
+
+  public AbsoluteTableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Expression getFilterExpression() {
+    return this.exp;
+  }
+
+  /**
+   * This method will provide the executer type to the callee inorder to identify
+   * the executer type for the filter resolution, Row level filter executer is a
+   * special executer since it get all the rows of the specified filter dimension
+   * and will be send to the spark for processing
+   */
+  public FilterExecuterType getFilterExecuterType() {
+    switch (exp.getFilterExpressionType()) {
+      case GREATERTHAN:
+        return FilterExecuterType.ROWLEVEL_GREATERTHAN;
+      case GREATERTHAN_EQUALTO:
+        return FilterExecuterType.ROWLEVEL_GREATERTHAN_EQUALTO;
+      case LESSTHAN:
+        return FilterExecuterType.ROWLEVEL_LESSTHAN;
+      case LESSTHAN_EQUALTO:
+        return FilterExecuterType.ROWLEVEL_LESSTHAN_EQUALTO;
+
+      default:
+        return FilterExecuterType.ROWLEVEL;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java b/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
new file mode 100644
index 0000000..c684f5f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.metadata;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+
+public class FilterResolverMetadata {
+  private AbsoluteTableIdentifier tableIdentifier;
+  private Expression expression;
+  private ColumnExpression columnExpression;
+  private boolean isIncludeFilter;
+
+  public AbsoluteTableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public void setTableIdentifier(AbsoluteTableIdentifier tableIdentifier) {
+    this.tableIdentifier = tableIdentifier;
+  }
+
+  public Expression getExpression() {
+    return expression;
+  }
+
+  public void setExpression(Expression expression) {
+    this.expression = expression;
+  }
+
+  public ColumnExpression getColumnExpression() {
+    return columnExpression;
+  }
+
+  public void setColumnExpression(ColumnExpression columnExpression) {
+    this.columnExpression = columnExpression;
+  }
+
+  public boolean isIncludeFilter() {
+    return isIncludeFilter;
+  }
+
+  public void setIncludeFilter(boolean isIncludeFilter) {
+    this.isIncludeFilter = isIncludeFilter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
new file mode 100644
index 0000000..ad7942a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.filter.resolver.resolverinfo;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.carbondata.core.carbon.datastore.IndexKey;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.GenericQueryType;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.visitable.ResolvedFilterInfoVisitable;
+import org.carbondata.scan.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
+
+public class DimColumnResolvedFilterInfo implements Serializable, ResolvedFilterInfoVisitable {
+  /**
+   *
+   */
+  private static final long serialVersionUID = 3428115141211084114L;
+
+  /**
+   * column index in file
+   */
+  private int columnIndex = -1;
+
+  /**
+   * need compressed data from file
+   */
+  private boolean needCompressedData;
+
+  /**
+   * rowIndex
+   */
+  private int rowIndex = -1;
+
+  private boolean isDimensionExistsInCurrentSilce = true;
+
+  private int rsSurrogates;
+
+  private String defaultValue;
+
+  private transient Map<Integer, GenericQueryType> complexTypesWithBlockStartIndex;
+
+  private CarbonDimension dimension;
+
+  /**
+   * start index key of the block based on the keygenerator
+   */
+  private transient IndexKey starIndexKey;
+
+  /**
+   * end index key  which is been formed considering the max surrogate values
+   * from dictionary cache
+   */
+  private transient IndexKey endIndexKey;
+
+  /**
+   * reolved filter object of a particlar filter Expression.
+   */
+  private DimColumnFilterInfo resolvedFilterValueObj;
+
+  private Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionResolvedFilter;
+
+  public DimColumnResolvedFilterInfo() {
+    dimensionResolvedFilter = new HashMap<CarbonDimension, List<DimColumnFilterInfo>>(20);
+  }
+
+  public IndexKey getStarIndexKey() {
+    return starIndexKey;
+  }
+
+  public void setStarIndexKey(IndexKey starIndexKey) {
+    this.starIndexKey = starIndexKey;
+  }
+
+  public IndexKey getEndIndexKey() {
+    return endIndexKey;
+  }
+
+  public void setEndIndexKey(IndexKey endIndexKey) {
+    this.endIndexKey = endIndexKey;
+  }
+
+  public void addDimensionResolvedFilterInstance(CarbonDimension dimension,
+      DimColumnFilterInfo filterResolvedObj) {
+    List<DimColumnFilterInfo> currentVals = dimensionResolvedFilter.get(dimension);
+    if (null == currentVals) {
+      currentVals = new ArrayList<DimColumnFilterInfo>(20);
+      currentVals.add(filterResolvedObj);
+      dimensionResolvedFilter.put(dimension, currentVals);
+    } else {
+      currentVals.add(filterResolvedObj);
+    }
+  }
+
+  public Map<CarbonDimension, List<DimColumnFilterInfo>> getDimensionResolvedFilterInstance() {
+    return dimensionResolvedFilter;
+  }
+
+  public Map<Integer, GenericQueryType> getComplexTypesWithBlockStartIndex() {
+    return complexTypesWithBlockStartIndex;
+  }
+
+  public void setComplexTypesWithBlockStartIndex(
+      Map<Integer, GenericQueryType> complexTypesWithBlockStartIndex) {
+    this.complexTypesWithBlockStartIndex = complexTypesWithBlockStartIndex;
+  }
+
+  public CarbonDimension getDimension() {
+    return dimension;
+  }
+
+  public void setDimension(CarbonDimension dimension) {
+    this.dimension = dimension;
+  }
+
+  public int getColumnIndex() {
+    return columnIndex;
+  }
+
+  public void setColumnIndex(int columnIndex) {
+    this.columnIndex = columnIndex;
+  }
+
+  public boolean isNeedCompressedData() {
+    return needCompressedData;
+  }
+
+  public void setNeedCompressedData(boolean needCompressedData) {
+    this.needCompressedData = needCompressedData;
+  }
+
+  public DimColumnFilterInfo getFilterValues() {
+    return resolvedFilterValueObj;
+  }
+
+  public void setFilterValues(final DimColumnFilterInfo resolvedFilterValueObj) {
+    this.resolvedFilterValueObj = resolvedFilterValueObj;
+  }
+
+  public int getRowIndex() {
+    return rowIndex;
+  }
+
+  public void setRowIndex(int rowIndex) {
+    this.rowIndex = rowIndex;
+  }
+
+  public boolean isDimensionExistsInCurrentSilce() {
+    return isDimensionExistsInCurrentSilce;
+  }
+
+  public void setDimensionExistsInCurrentSilce(boolean isDimensionExistsInCurrentSilce) {
+    this.isDimensionExistsInCurrentSilce = isDimensionExistsInCurrentSilce;
+  }
+
+  public int getRsSurrogates() {
+    return rsSurrogates;
+  }
+
+  public void setRsSurrogates(int rsSurrogates) {
+    this.rsSurrogates = rsSurrogates;
+  }
+
+  public String getDefaultValue() {
+    return defaultValue;
+  }
+
+  public void setDefaultValue(String defaultValue) {
+    this.defaultValue = defaultValue;
+  }
+
+  @Override public void populateFilterInfoBasedOnColumnType(ResolvedFilterInfoVisitorIntf visitor,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    if (null != visitor) {
+      visitor.populateFilterResolvedInfo(this, metadata);
+      this.addDimensionResolvedFilterInstance(metadata.getColumnExpression().getDimension(),
+          this.getFilterValues());
+      this.setDimension(metadata.getColumnExpression().getDimension());
+      this.setColumnIndex(metadata.getColumnExpression().getDimension().getOrdinal());
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
new file mode 100644
index 0000000..d4cac8c
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.filter.resolver.resolverinfo;
+
+import java.io.Serializable;
+
+public class MeasureColumnResolvedFilterInfo implements Serializable {
+  /**
+   *
+   */
+  private static final long serialVersionUID = 4222568289115151561L;
+
+  private int columnIndex = -1;
+
+  private int rowIndex = -1;
+
+  private Object uniqueValue;
+
+  private String aggregator;
+
+  private boolean isMeasureExistsInCurrentSlice = true;
+
+  private Object defaultValue;
+
+  private org.carbondata.core.carbon.metadata.datatype.DataType type;
+
+  public int getColumnIndex() {
+    return columnIndex;
+  }
+
+  public void setColumnIndex(int columnIndex) {
+    this.columnIndex = columnIndex;
+  }
+
+  public int getRowIndex() {
+    return rowIndex;
+  }
+
+  public void setRowIndex(int rowIndex) {
+    this.rowIndex = rowIndex;
+  }
+
+  public Object getUniqueValue() {
+    return uniqueValue;
+  }
+
+  public void setUniqueValue(Object uniqueValue) {
+    this.uniqueValue = uniqueValue;
+  }
+
+  public org.carbondata.core.carbon.metadata.datatype.DataType getType() {
+    return type;
+  }
+
+  public void setType(org.carbondata.core.carbon.metadata.datatype.DataType dataType) {
+    this.type = dataType;
+  }
+
+  /**
+   * @return Returns the aggregator.
+   */
+  public String getAggregator() {
+    return aggregator;
+  }
+
+  /**
+   * @param aggregator The aggregator to set.
+   */
+  public void setAggregator(String aggregator) {
+    this.aggregator = aggregator;
+  }
+
+  public boolean isMeasureExistsInCurrentSlice() {
+    return isMeasureExistsInCurrentSlice;
+  }
+
+  public void setMeasureExistsInCurrentSlice(boolean isMeasureExistsInCurrentSlice) {
+    this.isMeasureExistsInCurrentSlice = isMeasureExistsInCurrentSlice;
+  }
+
+  public Object getDefaultValue() {
+    return defaultValue;
+  }
+
+  public void setDefaultValue(double defaultValue) {
+    this.defaultValue = defaultValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
new file mode 100644
index 0000000..24762ae
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitable;
+
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
+
+public interface ResolvedFilterInfoVisitable {
+  /**
+   * This visitable method will visit through the visitor classes which is passed as parameter
+   * and based on different visitor the visitable filter instance will be resolved.
+   *
+   * @param visitor
+   * @param metadata
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  void populateFilterInfoBasedOnColumnType(ResolvedFilterInfoVisitorIntf visitor,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
new file mode 100644
index 0000000..273bdf1
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class CustomTypeDictionaryVisitor implements ResolvedFilterInfoVisitorIntf {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CustomTypeDictionaryVisitor.class.getName());
+
+  /**
+   * This Visitor method is been used to resolve or populate the filter details
+   * by using custom type dictionary value, the filter membrers will be resolved using
+   * custom type function which will generate dictionary for the direct column type filter members
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws FilterUnsupportedException
+   */
+  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    DimColumnFilterInfo resolvedFilterObject = null;
+
+    List<String> evaluateResultListFinal =
+        metadata.getExpression().evaluate(null).getListAsString();
+    resolvedFilterObject = getDirectDictionaryValKeyMemberForFilter(metadata.getTableIdentifier(),
+        metadata.getColumnExpression(), evaluateResultListFinal, metadata.isIncludeFilter());
+    visitableObj.setFilterValues(resolvedFilterObject);
+  }
+
+  private DimColumnFilterInfo getDirectDictionaryValKeyMemberForFilter(
+      AbsoluteTableIdentifier tableIdentifier, ColumnExpression columnExpression,
+      List<String> evaluateResultListFinal, boolean isIncludeFilter) {
+    List<Integer> surrogates = new ArrayList<Integer>(20);
+    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+        .getDirectDictionaryGenerator(columnExpression.getDimension().getDataType());
+    // Reading the dictionary value direct
+    for (String filterMember : evaluateResultListFinal) {
+      surrogates.add(directDictionaryGenerator.generateDirectSurrogateKey(filterMember));
+    }
+    Collections.sort(surrogates);
+    DimColumnFilterInfo columnFilterInfo = null;
+    if (surrogates.size() > 0) {
+      columnFilterInfo = new DimColumnFilterInfo();
+      columnFilterInfo.setIncludeFilter(isIncludeFilter);
+      columnFilterInfo.setFilterList(surrogates);
+    }
+    return columnFilterInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
new file mode 100644
index 0000000..7ec2751
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class DictionaryColumnVisitor implements ResolvedFilterInfoVisitorIntf {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DictionaryColumnVisitor.class.getName());
+
+  /**
+   * This Visitor method is used to populate the visitableObj with direct dictionary filter details
+   * where the filters values will be resolve using dictionary cache.
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws QueryExecutionException
+   */
+  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    DimColumnFilterInfo resolvedFilterObject = null;
+    List<String> evaluateResultListFinal =
+        metadata.getExpression().evaluate(null).getListAsString();
+    try {
+      resolvedFilterObject = FilterUtil
+          .getFilterValues(metadata.getTableIdentifier(), metadata.getColumnExpression(),
+              evaluateResultListFinal, metadata.isIncludeFilter());
+    } catch (QueryExecutionException e) {
+      throw new FilterUnsupportedException(e);
+    }
+    visitableObj.setFilterValues(resolvedFilterObject);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
new file mode 100644
index 0000000..35d9d70
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.scan.expression.ColumnExpression;
+
+public class FilterInfoTypeVisitorFactory {
+
+  /**
+   * This factory method will be used in order to get the visitor instance based on the
+   * column expression metadata where filters has been applied.
+   *
+   * @param columnExpression
+   * @return
+   */
+  public static ResolvedFilterInfoVisitorIntf getResolvedFilterInfoVisitor(
+      ColumnExpression columnExpression) {
+    if (columnExpression.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+      return new CustomTypeDictionaryVisitor();
+    } else if (!columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return new NoDictionaryTypeVisitor();
+    } else if (columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return new DictionaryColumnVisitor();
+    }
+
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
new file mode 100644
index 0000000..ee3d157
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class NoDictionaryTypeVisitor implements ResolvedFilterInfoVisitorIntf {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(NoDictionaryTypeVisitor.class.getName());
+
+  /**
+   * Visitor Method will update the filter related details in visitableObj, For no dictionary
+   * type columns the filter members will resolved directly, no need to look up in dictionary
+   * since it will not be part of dictionary, directly the actual data can be converted as
+   * byte[] and can be set. this type of encoding is effective when the particular column
+   * is having very high cardinality.
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws FilterUnsupportedException
+   */
+  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    DimColumnFilterInfo resolvedFilterObject = null;
+    List<String> evaluateResultListFinal =
+        metadata.getExpression().evaluate(null).getListAsString();
+    resolvedFilterObject = FilterUtil
+        .getNoDictionaryValKeyMemberForFilter(metadata.getTableIdentifier(),
+            metadata.getColumnExpression(), evaluateResultListFinal, metadata.isIncludeFilter());
+    visitableObj.setFilterValues(resolvedFilterObject);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
new file mode 100644
index 0000000..1cd9197
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public interface ResolvedFilterInfoVisitorIntf {
+
+  /**
+   * Visitor pattern is been used in this scenario inorder to populate the
+   * dimColResolvedFilterInfo visitable object with filter member values based
+   * on the visitor type, currently there 3 types of visitors custom,direct
+   * and no dictionary, all types of visitor populate the visitable instance
+   * as per its buisness logic which is different for all the visitors.
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws QueryExecutionException
+   */
+  void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
new file mode 100644
index 0000000..840aa02
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ *
+ */
+package org.carbondata.scan.model;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.scan.expression.Expression;
+
+/**
+ * This class contains all the logical information about the query like dimensions,measures,
+ * sort order, topN etc..
+ */
+public class CarbonQueryPlan implements Serializable {
+  /**
+   *
+   */
+  private static final long serialVersionUID = -9036044826928017164L;
+
+  /**
+   * Schema name , if user asks select * from datasight.employee.
+   * then datasight is the schame name.
+   * Remains null if the user does not select schema name.
+   */
+  private String schemaName;
+
+  /**
+   * Cube name .
+   * if user asks select * from datasight.employee. then employee is the cube name.
+   * It is mandatory.
+   */
+  private String cubeName;
+
+  /**
+   * List of dimensions.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name
+   * and department_name are dimensions
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<QueryDimension> dimensions =
+      new ArrayList<QueryDimension>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * List of measures.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary)
+   * would be measure.
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<QueryMeasure> measures =
+      new ArrayList<QueryMeasure>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * Limit
+   */
+  private int limit = -1;
+
+  /**
+   * If it is detail query, no need to aggregate in backend
+   */
+  private boolean detailQuery;
+
+  /**
+   * expression
+   */
+  private Expression expression;
+
+  /**
+   * queryId
+   */
+  private String queryId;
+
+  /**
+   * outLocationPath
+   */
+  private String outLocationPath;
+
+  /**
+   * isCountStarQuery
+   */
+  private boolean isCountStartQuery;
+
+  private List<QueryDimension> sortedDimensions;
+
+  /**
+   * If it is raw detail query, no need to aggregate in backend. And it reurns with dictionary data
+   * with out decoding.
+   */
+  private boolean rawDetailQuery;
+
+  /**
+   * Constructor created with cube name.
+   *
+   * @param cubeName
+   */
+  public CarbonQueryPlan(String cubeName) {
+    this.cubeName = cubeName;
+  }
+
+  /**
+   * Constructor created with schema name and cube name.
+   *
+   * @param schemaName
+   * @param cubeName
+   */
+  public CarbonQueryPlan(String schemaName, String cubeName) {
+    this.cubeName = cubeName;
+    this.schemaName = schemaName;
+  }
+
+  /**
+   * @return the dimensions
+   */
+  public List<QueryDimension> getDimensions() {
+    return dimensions;
+  }
+
+  public void addDimension(QueryDimension dimension) {
+    this.dimensions.add(dimension);
+  }
+
+  /**
+   * @return the measures
+   */
+  public List<QueryMeasure> getMeasures() {
+    return measures;
+  }
+
+  public void addMeasure(QueryMeasure measure) {
+    this.measures.add(measure);
+  }
+
+  public Expression getFilterExpression() {
+    return expression;
+  }
+
+  public void setFilterExpression(Expression expression) {
+    this.expression = expression;
+  }
+
+  /**
+   * @return the schemaName
+   */
+  public String getSchemaName() {
+    return schemaName;
+  }
+
+  /**
+   * @return the cubeName
+   */
+  public String getCubeName() {
+    return cubeName;
+  }
+
+  /**
+   * @return the limit
+   */
+  public int getLimit() {
+    return limit;
+  }
+
+  /**
+   * @param limit the limit to set
+   */
+  public void setLimit(int limit) {
+    this.limit = limit;
+  }
+
+  /**
+   * @return the detailQuery
+   */
+  public boolean isDetailQuery() {
+    return detailQuery;
+  }
+
+  /**
+   * @param detailQuery the detailQuery to set
+   */
+  public void setDetailQuery(boolean detailQuery) {
+    this.detailQuery = detailQuery;
+  }
+
+  public String getQueryId() {
+    return queryId;
+  }
+
+  public void setQueryId(String queryId) {
+    this.queryId = queryId;
+  }
+
+  public String getOutLocationPath() {
+    return outLocationPath;
+  }
+
+  public void setOutLocationPath(String outLocationPath) {
+    this.outLocationPath = outLocationPath;
+  }
+
+  public boolean isCountStarQuery() {
+    return isCountStartQuery;
+  }
+
+  public void setCountStartQuery(boolean isCountStartQuery) {
+    this.isCountStartQuery = isCountStartQuery;
+  }
+
+  public List<QueryDimension> getSortedDimemsions() {
+    return sortedDimensions;
+  }
+
+  public void setSortedDimemsions(List<QueryDimension> dims) {
+    this.sortedDimensions = dims;
+  }
+
+  public boolean isRawDetailQuery() {
+    return rawDetailQuery;
+  }
+
+  public void setRawDetailQuery(boolean rawDetailQuery) {
+    this.rawDetailQuery = rawDetailQuery;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryColumn.java b/core/src/main/java/org/carbondata/scan/model/QueryColumn.java
new file mode 100644
index 0000000..85ff41d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryColumn.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * query column  which will have information about column
+ */
+public class QueryColumn implements Serializable {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -4222306600480181084L;
+
+  /**
+   * name of the column
+   */
+  protected String columnName;
+
+  /**
+   * sort order in which column output will be sorted default it will be none
+   */
+  private SortOrderType sortOrder = SortOrderType.NONE;
+
+  /**
+   * query order in which result of the query will be send
+   */
+  private int queryOrder;
+
+  /**
+   * aggregation function applied on column
+   */
+
+  private String aggregationFunction=CarbonCommonConstants.DUMMY;
+
+  public QueryColumn(String columnName) {
+    this.columnName = columnName;
+  }
+
+  /**
+   * @return the sortOrder
+   */
+  public SortOrderType getSortOrder() {
+    return sortOrder;
+  }
+
+  /**
+   * @param sortOrder the sortOrder to set
+   */
+  public void setSortOrder(SortOrderType sortOrder) {
+    this.sortOrder = sortOrder;
+  }
+
+  /**
+   * @return the columnName
+   */
+  public String getColumnName() {
+    return columnName;
+  }
+
+  /**
+   * @return the queryOrder
+   */
+  public int getQueryOrder() {
+    return queryOrder;
+  }
+
+  /**
+   * @param queryOrder the queryOrder to set
+   */
+  public void setQueryOrder(int queryOrder) {
+    this.queryOrder = queryOrder;
+  }
+
+  /**
+   * @return the aggregationFunction
+   */
+  public String getAggregateFunction() {
+    return aggregationFunction;
+  }
+
+  /**
+   * @param aggregationFunction the aggregationFunction to set
+   */
+  public void setAggregateFunction(String aggregationFunction) {
+    this.aggregationFunction = aggregationFunction;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryDimension.java b/core/src/main/java/org/carbondata/scan/model/QueryDimension.java
new file mode 100644
index 0000000..dc07173
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryDimension.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+
+/**
+ * query plan dimension which will holds the information about the query plan dimension
+ * this is done to avoid heavy object serialization
+ */
+public class QueryDimension extends QueryColumn implements Serializable {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -8492704093776645651L;
+  /**
+   * actual dimension column
+   */
+  private transient CarbonDimension dimension;
+
+  public QueryDimension(String columName) {
+    super(columName);
+  }
+
+  /**
+   * @return the dimension
+   */
+  public CarbonDimension getDimension() {
+    return dimension;
+  }
+
+  /**
+   * @param dimension the dimension to set
+   */
+  public void setDimension(CarbonDimension dimension) {
+    this.dimension = dimension;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java b/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java
new file mode 100644
index 0000000..4035e61
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+
+/**
+ * query plan measure, this class will holds the information
+ * about measure present in the query, this is done to avoid the serialization
+ * of the heavy object
+ */
+public class QueryMeasure extends QueryColumn implements Serializable {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 1035512411375495414L;
+
+  /**
+   * actual carbon measure object
+   */
+  private transient CarbonMeasure measure;
+
+  public QueryMeasure(String columName) {
+    super(columName);
+  }
+
+  /**
+   * @return the measure
+   */
+  public CarbonMeasure getMeasure() {
+    return measure;
+  }
+
+  /**
+   * @param measure the measure to set
+   */
+  public void setMeasure(CarbonMeasure measure) {
+    this.measure = measure;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryModel.java b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
new file mode 100644
index 0000000..c979b6d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
@@ -0,0 +1,516 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.model;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.carbondata.core.cache.dictionary.Dictionary;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.UnknownExpression;
+import org.carbondata.scan.expression.conditional.ConditionalExpression;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+
+/**
+ * Query model which will have all the detail
+ * about the query, This will be sent from driver to executor '
+ * This will be refereed to executing the query.
+ */
+public class QueryModel implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = -4674677234007089052L;
+
+  /**
+   * list of dimension selected for in query
+   */
+  private List<QueryDimension> queryDimension;
+
+  /**
+   * list of dimension in which sorting is applied
+   */
+  private List<QueryDimension> sortDimension;
+
+  /**
+   * list of measure selected in query
+   */
+  private List<QueryMeasure> queryMeasures;
+
+  /**
+   * query id
+   */
+  private String queryId;
+
+  /**
+   * to check if it a aggregate table
+   */
+  private boolean isAggTable;
+
+  /**
+   * filter tree
+   */
+  private FilterResolverIntf filterExpressionResolverTree;
+
+  /**
+   * in case of lime query we need to know how many
+   * records will passed from executor
+   */
+  private int limit;
+
+  /**
+   * to check if it is a count star query , so processing will be different
+   */
+  private boolean isCountStarQuery;
+
+  /**
+   * to check whether aggregation is required during query execution
+   */
+  private boolean detailQuery;
+
+  /**
+   * table block information in which query will be executed
+   */
+  private List<TableBlockInfo> tableBlockInfos;
+
+  /**
+   * sort in which dimension will be get sorted
+   */
+  private byte[] sortOrder;
+
+  /**
+   * absolute table identifier
+   */
+  private AbsoluteTableIdentifier absoluteTableIdentifier;
+  /**
+   * in case of detail query with sort we are spilling to disk
+   * to this location will be used to write the temp file in this location
+   */
+  private String queryTempLocation;
+
+  /**
+   * To handle most of the computation in query engines like spark and hive, carbon should give
+   * raw detailed records to it.
+   */
+  private boolean forcedDetailRawQuery;
+
+  /**
+   * paritition column list
+   */
+  private List<String> paritionColumns;
+
+  /**
+   * this will hold the information about the dictionary dimension
+   * which to
+   */
+  public transient Map<String, Dictionary> columnToDictionaryMapping;
+
+  /**
+   * table on which query will be executed
+   * TODO need to remove this ad pass only the path
+   * and carbon metadata will load the table from metadata file
+   */
+  private CarbonTable table;
+
+  /**
+   * This is used only whne [forcedDetailRawQuery = true]. By default forcedDetailRawQuery returns
+   * dictionary values. But user wants in detail raw bytes the user set this field to true.
+   */
+  private boolean rawBytesDetailQuery;
+
+  public QueryModel() {
+    tableBlockInfos = new ArrayList<TableBlockInfo>();
+    queryDimension = new ArrayList<QueryDimension>();
+    queryMeasures = new ArrayList<QueryMeasure>();
+    sortDimension = new ArrayList<QueryDimension>();
+    sortOrder = new byte[0];
+    paritionColumns = new ArrayList<String>();
+
+  }
+
+  public static QueryModel createModel(AbsoluteTableIdentifier absoluteTableIdentifier,
+      CarbonQueryPlan queryPlan, CarbonTable carbonTable) {
+    QueryModel queryModel = new QueryModel();
+    String factTableName = carbonTable.getFactTableName();
+    queryModel.setAbsoluteTableIdentifier(absoluteTableIdentifier);
+
+    fillQueryModel(queryPlan, carbonTable, queryModel, factTableName);
+
+    queryModel.setLimit(queryPlan.getLimit());
+    queryModel.setDetailQuery(queryPlan.isDetailQuery());
+    queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery());
+    queryModel.setQueryId(queryPlan.getQueryId());
+    queryModel.setQueryTempLocation(queryPlan.getOutLocationPath());
+    return queryModel;
+  }
+
+  private static void fillQueryModel(CarbonQueryPlan queryPlan, CarbonTable carbonTable,
+      QueryModel queryModel, String factTableName) {
+    queryModel.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
+    queryModel.setQueryDimension(queryPlan.getDimensions());
+    fillSortInfoInModel(queryModel, queryPlan.getSortedDimemsions());
+    queryModel.setQueryMeasures(
+        queryPlan.getMeasures());
+    if (null != queryPlan.getFilterExpression()) {
+      processFilterExpression(queryPlan.getFilterExpression(),
+          carbonTable.getDimensionByTableName(factTableName),
+          carbonTable.getMeasureByTableName(factTableName));
+    }
+    queryModel.setCountStarQuery(queryPlan.isCountStarQuery());
+    //TODO need to remove this code, and executor will load the table
+    // from file metadata
+    queryModel.setTable(carbonTable);
+  }
+
+  private static void fillSortInfoInModel(QueryModel executorModel,
+      List<QueryDimension> sortedDims) {
+    if (null != sortedDims) {
+      byte[] sortOrderByteArray = new byte[sortedDims.size()];
+      int i = 0;
+      for (QueryColumn mdim : sortedDims) {
+        sortOrderByteArray[i++] = (byte) mdim.getSortOrder().ordinal();
+      }
+      executorModel.setSortOrder(sortOrderByteArray);
+      executorModel.setSortDimension(sortedDims);
+    } else {
+      executorModel.setSortOrder(new byte[0]);
+      executorModel.setSortDimension(new ArrayList<QueryDimension>(0));
+    }
+
+  }
+
+  public static void processFilterExpression(
+      Expression filterExpression, List<CarbonDimension> dimensions, List<CarbonMeasure> measures) {
+    if (null != filterExpression) {
+      if (null != filterExpression.getChildren() && filterExpression.getChildren().size() == 0) {
+        if (filterExpression instanceof ConditionalExpression) {
+          List<ColumnExpression> listOfCol =
+              ((ConditionalExpression) filterExpression).getColumnList();
+          for (ColumnExpression expression : listOfCol) {
+            setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression);
+          }
+
+        }
+      }
+      for (Expression expression : filterExpression.getChildren()) {
+
+        if (expression instanceof ColumnExpression) {
+          setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression);
+        } else if (expression instanceof UnknownExpression) {
+          UnknownExpression exp = ((UnknownExpression) expression);
+          List<ColumnExpression> listOfColExpression = exp.getAllColumnList();
+          for (ColumnExpression col : listOfColExpression) {
+            setDimAndMsrColumnNode(dimensions, measures, col);
+          }
+        } else {
+          processFilterExpression(expression, dimensions, measures);
+        }
+      }
+    }
+
+  }
+
+  private static CarbonMeasure getCarbonMetadataMeasure(String name, List<CarbonMeasure> measures) {
+    for (CarbonMeasure measure : measures) {
+      if (measure.getColName().equalsIgnoreCase(name)) {
+        return measure;
+      }
+    }
+    return null;
+  }
+
+  private static void setDimAndMsrColumnNode(List<CarbonDimension> dimensions,
+      List<CarbonMeasure> measures, ColumnExpression col) {
+    CarbonDimension dim;
+    CarbonMeasure msr;
+    String columnName;
+    columnName = col.getColumnName();
+    dim = CarbonUtil.findDimension(dimensions, columnName);
+    col.setCarbonColumn(dim);
+    col.setDimension(dim);
+    col.setDimension(true);
+    if (null == dim) {
+      msr = getCarbonMetadataMeasure(columnName, measures);
+      col.setCarbonColumn(msr);
+      col.setDimension(false);
+    }
+  }
+
+  /**
+   * It gets the projection columns
+   */
+  public CarbonColumn[] getProjectionColumns() {
+    CarbonColumn[] carbonColumns =
+        new CarbonColumn[getQueryDimension().size() + getQueryMeasures()
+            .size()];
+    for (QueryDimension dimension : getQueryDimension()) {
+      carbonColumns[dimension.getQueryOrder()] = dimension.getDimension();
+    }
+    for (QueryMeasure msr : getQueryMeasures()) {
+      carbonColumns[msr.getQueryOrder()] = msr.getMeasure();
+    }
+    return carbonColumns;
+  }
+
+  /**
+   * @return the queryDimension
+   */
+  public List<QueryDimension> getQueryDimension() {
+    return queryDimension;
+  }
+
+  /**
+   * @param queryDimension the queryDimension to set
+   */
+  public void setQueryDimension(List<QueryDimension> queryDimension) {
+    this.queryDimension = queryDimension;
+  }
+
+  /**
+   * @return the queryMeasures
+   */
+  public List<QueryMeasure> getQueryMeasures() {
+    return queryMeasures;
+  }
+
+  /**
+   * @param queryMeasures the queryMeasures to set
+   */
+  public void setQueryMeasures(List<QueryMeasure> queryMeasures) {
+    this.queryMeasures = queryMeasures;
+  }
+
+  /**
+   * @return the queryId
+   */
+  public String getQueryId() {
+    return queryId;
+  }
+
+  /**
+   * @param queryId the queryId to set
+   */
+  public void setQueryId(String queryId) {
+    this.queryId = queryId;
+  }
+
+  /**
+   * @return the isAggTable
+   */
+  public boolean isAggTable() {
+    return isAggTable;
+  }
+
+  /**
+   * @param isAggTable the isAggTable to set
+   */
+  public void setAggTable(boolean isAggTable) {
+    this.isAggTable = isAggTable;
+  }
+
+  /**
+   * @return the limit
+   */
+  public int getLimit() {
+    return limit;
+  }
+
+  /**
+   * @param limit the limit to set
+   */
+  public void setLimit(int limit) {
+    this.limit = limit;
+  }
+
+  /**
+   * @return the isCountStarQuery
+   */
+  public boolean isCountStarQuery() {
+    return isCountStarQuery;
+  }
+
+  /**
+   * @param isCountStarQuery the isCountStarQuery to set
+   */
+  public void setCountStarQuery(boolean isCountStarQuery) {
+    this.isCountStarQuery = isCountStarQuery;
+  }
+
+  /**
+   * @return the isdetailQuery
+   */
+  public boolean isDetailQuery() {
+    return detailQuery;
+  }
+
+  public void setDetailQuery(boolean detailQuery) {
+    this.detailQuery = detailQuery;
+  }
+
+  /**
+   * @return the tableBlockInfos
+   */
+  public List<TableBlockInfo> getTableBlockInfos() {
+    return tableBlockInfos;
+  }
+
+  /**
+   * @param tableBlockInfos the tableBlockInfos to set
+   */
+  public void setTableBlockInfos(List<TableBlockInfo> tableBlockInfos) {
+    this.tableBlockInfos = tableBlockInfos;
+  }
+
+  /**
+   * @return the queryTempLocation
+   */
+  public String getQueryTempLocation() {
+    return queryTempLocation;
+  }
+
+  /**
+   * @param queryTempLocation the queryTempLocation to set
+   */
+  public void setQueryTempLocation(String queryTempLocation) {
+    this.queryTempLocation = queryTempLocation;
+  }
+
+  /**
+   * @return the sortOrder
+   */
+  public byte[] getSortOrder() {
+    return sortOrder;
+  }
+
+  /**
+   * @param sortOrder the sortOrder to set
+   */
+  public void setSortOrder(byte[] sortOrder) {
+    this.sortOrder = sortOrder;
+  }
+
+  /**
+   * @return the sortDimension
+   */
+  public List<QueryDimension> getSortDimension() {
+    return sortDimension;
+  }
+
+  /**
+   * @param sortDimension the sortDimension to set
+   */
+  public void setSortDimension(List<QueryDimension> sortDimension) {
+    this.sortDimension = sortDimension;
+  }
+
+  /**
+   * @return the filterEvaluatorTree
+   */
+  public FilterResolverIntf getFilterExpressionResolverTree() {
+    return filterExpressionResolverTree;
+  }
+
+  public void setFilterExpressionResolverTree(FilterResolverIntf filterExpressionResolverTree) {
+    this.filterExpressionResolverTree = filterExpressionResolverTree;
+  }
+
+  /**
+   * @return the absoluteTableIdentifier
+   */
+  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
+    return absoluteTableIdentifier;
+  }
+
+  /**
+   * @param absoluteTableIdentifier the absoluteTableIdentifier to set
+   */
+  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
+    this.absoluteTableIdentifier = absoluteTableIdentifier;
+  }
+
+  /**
+   * @return the paritionColumns
+   */
+  public List<String> getParitionColumns() {
+    return paritionColumns;
+  }
+
+  /**
+   * @param paritionColumns the paritionColumns to set
+   */
+  public void setParitionColumns(List<String> paritionColumns) {
+    this.paritionColumns = paritionColumns;
+  }
+
+  /**
+   * @return the table
+   */
+  public CarbonTable getTable() {
+    return table;
+  }
+
+  /**
+   * @param table the table to set
+   */
+  public void setTable(CarbonTable table) {
+    this.table = table;
+  }
+
+  public boolean isForcedDetailRawQuery() {
+    return forcedDetailRawQuery;
+  }
+
+  public void setForcedDetailRawQuery(boolean forcedDetailRawQuery) {
+    this.forcedDetailRawQuery = forcedDetailRawQuery;
+  }
+
+  /**
+   * @return
+   */
+  public Map<String, Dictionary> getColumnToDictionaryMapping() {
+    return columnToDictionaryMapping;
+  }
+
+  /**
+   * @param columnToDictionaryMapping
+   */
+  public void setColumnToDictionaryMapping(Map<String, Dictionary> columnToDictionaryMapping) {
+    this.columnToDictionaryMapping = columnToDictionaryMapping;
+  }
+
+  public boolean isRawBytesDetailQuery() {
+    return rawBytesDetailQuery;
+  }
+
+  public void setRawBytesDetailQuery(boolean rawBytesDetailQuery) {
+    this.rawBytesDetailQuery = rawBytesDetailQuery;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.java b/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.java
new file mode 100644
index 0000000..643e81b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.keygenerator.KeyGenerator;
+
+public class QuerySchemaInfo implements Serializable {
+
+  private int[] maskedByteIndexes;
+
+  private KeyGenerator keyGenerator;
+
+  private QueryDimension[] queryDimensions;
+
+  private QueryMeasure[] queryMeasures;
+
+  private int[] queryOrder;
+
+  private int[] queryReverseOrder;
+
+  public int[] getMaskedByteIndexes() {
+    return maskedByteIndexes;
+  }
+
+  public void setMaskedByteIndexes(int[] maskedByteIndexes) {
+    this.maskedByteIndexes = maskedByteIndexes;
+  }
+
+  public KeyGenerator getKeyGenerator() {
+    return keyGenerator;
+  }
+
+  public void setKeyGenerator(KeyGenerator keyGenerator) {
+    this.keyGenerator = keyGenerator;
+  }
+
+  public QueryDimension[] getQueryDimensions() {
+    return queryDimensions;
+  }
+
+  public void setQueryDimensions(QueryDimension[] queryDimensions) {
+    this.queryDimensions = queryDimensions;
+  }
+
+  public QueryMeasure[] getQueryMeasures() {
+    return queryMeasures;
+  }
+
+  public void setQueryMeasures(QueryMeasure[] queryMeasures) {
+    this.queryMeasures = queryMeasures;
+  }
+
+  public int[] getQueryOrder() {
+    return queryOrder;
+  }
+
+  public void setQueryOrder(int[] queryOrder) {
+    this.queryOrder = queryOrder;
+  }
+
+  public int[] getQueryReverseOrder() {
+    return queryReverseOrder;
+  }
+
+  public void setQueryReverseOrder(int[] queryReverseOrder) {
+    this.queryReverseOrder = queryReverseOrder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/SortOrderType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/SortOrderType.java b/core/src/main/java/org/carbondata/scan/model/SortOrderType.java
new file mode 100644
index 0000000..00b9219
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/SortOrderType.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.model;
+
+/**
+ * enum for sorting the columns
+ */
+public enum SortOrderType {
+
+    /**
+     * Ascending order
+     */
+    ASC(0),
+
+    /**
+     * Descending order.
+     */
+    DSC(1),
+
+    /**
+     * No order mentioned
+     */
+    NONE(-1);
+  /**
+   * Order type in numeric
+   */
+  private int orderType;
+
+  SortOrderType(int orderType) {
+    this.orderType = orderType;
+  }
+
+  /**
+   * Order type in number
+   *
+   * @return orderType int
+   */
+  public int getOrderType() {
+    return orderType;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
new file mode 100644
index 0000000..e3c7cd5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.processor;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.scan.collector.ScannedResultCollector;
+import org.carbondata.scan.collector.impl.ListBasedResultCollector;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.result.AbstractScannedResult;
+import org.carbondata.scan.result.Result;
+import org.carbondata.scan.scanner.BlockletScanner;
+import org.carbondata.scan.scanner.impl.FilterScanner;
+import org.carbondata.scan.scanner.impl.NonFilterScanner;
+
+/**
+ * This abstract class provides a skeletal implementation of the
+ * Block iterator.
+ */
+public abstract class AbstractDataBlockIterator extends CarbonIterator<Result> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName());
+  /**
+   * iterator which will be used to iterate over data blocks
+   */
+  protected CarbonIterator<DataRefNode> dataBlockIterator;
+
+  /**
+   * execution details
+   */
+  protected BlockExecutionInfo blockExecutionInfo;
+
+  /**
+   * result collector which will be used to aggregate the scanned result
+   */
+  protected ScannedResultCollector scannerResultAggregator;
+
+  /**
+   * processor which will be used to process the block processing can be
+   * filter processing or non filter processing
+   */
+  protected BlockletScanner blockletScanner;
+
+  /**
+   * to hold the data block
+   */
+  protected BlocksChunkHolder blocksChunkHolder;
+
+  /**
+   * batch size of result
+   */
+  protected int batchSize;
+
+  protected AbstractScannedResult scannedResult;
+
+  public AbstractDataBlockIterator(BlockExecutionInfo blockExecutionInfo,
+      FileHolder fileReader, int batchSize) {
+    this.blockExecutionInfo = blockExecutionInfo;
+    dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
+        blockExecutionInfo.getNumberOfBlockToScan());
+    blocksChunkHolder = new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(),
+        blockExecutionInfo.getTotalNumberOfMeasureBlock());
+    blocksChunkHolder.setFileReader(fileReader);
+
+    if (blockExecutionInfo.getFilterExecuterTree() != null) {
+      blockletScanner = new FilterScanner(blockExecutionInfo);
+    } else {
+      blockletScanner = new NonFilterScanner(blockExecutionInfo);
+    }
+
+    this.scannerResultAggregator =
+        new ListBasedResultCollector(blockExecutionInfo);
+    this.batchSize = batchSize;
+  }
+
+  public boolean hasNext() {
+    try {
+      if (scannedResult != null && scannedResult.hasNext()) {
+        return true;
+      } else {
+        scannedResult = getNextScannedResult();
+        while (scannedResult != null) {
+          if (scannedResult.hasNext()) {
+            return true;
+          }
+          scannedResult = getNextScannedResult();
+        }
+        return false;
+      }
+    } catch (QueryExecutionException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  private AbstractScannedResult getNextScannedResult() throws QueryExecutionException {
+    if (dataBlockIterator.hasNext()) {
+      blocksChunkHolder.setDataBlock(dataBlockIterator.next());
+      blocksChunkHolder.reset();
+      return blockletScanner.scanBlocklet(blocksChunkHolder);
+    }
+    return null;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
new file mode 100644
index 0000000..0b36ab5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.processor;
+
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.iterator.CarbonIterator;
+
+/**
+ * Below class will be used to iterate over data block
+ */
+public class BlockletIterator extends CarbonIterator<DataRefNode> {
+  /**
+   * data store block
+   */
+  protected DataRefNode datablock;
+  /**
+   * block counter to keep a track how many block has been processed
+   */
+  private int blockCounter;
+
+  /**
+   * flag to be used to check any more data block is present or not
+   */
+  private boolean hasNext = true;
+
+  /**
+   * total number blocks assgned to this iterator
+   */
+  private long totalNumberOfBlocksToScan;
+
+  /**
+   * Constructor
+   *
+   * @param datablock                 first data block
+   * @param totalNumberOfBlocksToScan total number of blocks to be scanned
+   */
+  public BlockletIterator(DataRefNode datablock, long totalNumberOfBlocksToScan) {
+    this.datablock = datablock;
+    this.totalNumberOfBlocksToScan = totalNumberOfBlocksToScan;
+  }
+
+  /**
+   * is all the blocks assigned to this iterator has been processed
+   */
+  @Override public boolean hasNext() {
+    return hasNext;
+  }
+
+  @Override
+  /**
+   * To get the next block
+   * @return next data block
+   *
+   */
+  public DataRefNode next() {
+    // get the current blocks
+    DataRefNode datablockTemp = datablock;
+    // store the next data block
+    datablock = datablock.getNextDataRefNode();
+    // increment the counter
+    blockCounter++;
+    // if all the data block is processed then
+    // set the has next flag to false
+    // or if number of blocks assigned to this iterator is processed
+    // then also set the hasnext flag to false
+    if (null == datablock || blockCounter >= this.totalNumberOfBlocksToScan) {
+      hasNext = false;
+    }
+    return datablockTemp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java b/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java
new file mode 100644
index 0000000..bbf0cd9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.processor;
+
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.datastorage.store.FileHolder;
+
+/**
+ * Block chunk holder which will hold the dimension and
+ * measure chunk
+ */
+public class BlocksChunkHolder {
+
+  /**
+   * dimension column data chunk
+   */
+  private DimensionColumnDataChunk[] dimensionDataChunk;
+
+  /**
+   * measure column data chunk
+   */
+  private MeasureColumnDataChunk[] measureDataChunk;
+
+  /**
+   * file reader which will use to read the block from file
+   */
+  private FileHolder fileReader;
+
+  /**
+   * data block
+   */
+  private DataRefNode dataBlock;
+
+  public BlocksChunkHolder(int numberOfDimensionBlock, int numberOfMeasureBlock) {
+    dimensionDataChunk = new DimensionColumnDataChunk[numberOfDimensionBlock];
+    measureDataChunk = new MeasureColumnDataChunk[numberOfMeasureBlock];
+  }
+
+  /**
+   * @return the dimensionDataChunk
+   */
+  public DimensionColumnDataChunk[] getDimensionDataChunk() {
+    return dimensionDataChunk;
+  }
+
+  /**
+   * @param dimensionDataChunk the dimensionDataChunk to set
+   */
+  public void setDimensionDataChunk(DimensionColumnDataChunk[] dimensionDataChunk) {
+    this.dimensionDataChunk = dimensionDataChunk;
+  }
+
+  /**
+   * @return the measureDataChunk
+   */
+  public MeasureColumnDataChunk[] getMeasureDataChunk() {
+    return measureDataChunk;
+  }
+
+  /**
+   * @param measureDataChunk the measureDataChunk to set
+   */
+  public void setMeasureDataChunk(MeasureColumnDataChunk[] measureDataChunk) {
+    this.measureDataChunk = measureDataChunk;
+  }
+
+  /**
+   * @return the fileReader
+   */
+  public FileHolder getFileReader() {
+    return fileReader;
+  }
+
+  /**
+   * @param fileReader the fileReader to set
+   */
+  public void setFileReader(FileHolder fileReader) {
+    this.fileReader = fileReader;
+  }
+
+  /**
+   * @return the dataBlock
+   */
+  public DataRefNode getDataBlock() {
+    return dataBlock;
+  }
+
+  /**
+   * @param dataBlock the dataBlock to set
+   */
+  public void setDataBlock(DataRefNode dataBlock) {
+    this.dataBlock = dataBlock;
+  }
+
+  /***
+   * To reset the measure chunk and dimension chunk
+   * array
+   */
+  public void reset() {
+    for (int i = 0; i < measureDataChunk.length; i++) {
+      this.measureDataChunk[i] = null;
+    }
+    for (int i = 0; i < dimensionDataChunk.length; i++) {
+      this.dimensionDataChunk[i] = null;
+    }
+  }
+}