http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
deleted file mode 100644
index 82c6b84..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ /dev/null
@@ -1,229 +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.apache.carbondata.scan.filter.resolver;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.SortedMap;
-
-import org.apache.carbondata.core.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.metadata.DataType;
-import org.apache.carbondata.core.metadata.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.scan.expression.ColumnExpression;
-import org.apache.carbondata.scan.expression.Expression;
-import 
org.apache.carbondata.scan.expression.conditional.BinaryConditionalExpression;
-import org.apache.carbondata.scan.expression.conditional.ConditionalExpression;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.filter.FilterUtil;
-import org.apache.carbondata.scan.filter.intf.FilterExecuterType;
-import 
org.apache.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.visitor.FilterInfoTypeVisitorFactory;
-
-public class ConditionalFilterResolverImpl implements FilterResolverIntf {
-
-  private static final long serialVersionUID = 1838955268462201691L;
-  protected Expression exp;
-  protected boolean isExpressionResolve;
-  protected boolean isIncludeFilter;
-  private DimColumnResolvedFilterInfo dimColResolvedFilterInfo;
-
-  public ConditionalFilterResolverImpl(Expression exp, boolean 
isExpressionResolve,
-      boolean isIncludeFilter) {
-    this.exp = exp;
-    this.isExpressionResolve = isExpressionResolve;
-    this.isIncludeFilter = isIncludeFilter;
-    this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo();
-  }
-
-  /**
-   * This API will resolve the filter expression and generates the
-   * dictionaries for executing/evaluating the filter expressions in the
-   * executer layer.
-   *
-   * @throws FilterUnsupportedException
-   */
-  @Override public void resolve(AbsoluteTableIdentifier 
absoluteTableIdentifier)
-      throws FilterUnsupportedException, IOException {
-    FilterResolverMetadata metadata = new FilterResolverMetadata();
-    metadata.setTableIdentifier(absoluteTableIdentifier);
-    if ((!isExpressionResolve) && exp instanceof BinaryConditionalExpression) {
-      BinaryConditionalExpression binaryConditionalExpression = 
(BinaryConditionalExpression) exp;
-      Expression leftExp = binaryConditionalExpression.getLeft();
-      Expression rightExp = binaryConditionalExpression.getRight();
-      if (leftExp instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) leftExp;
-        metadata.setColumnExpression(columnExpression);
-        metadata.setExpression(rightExp);
-        metadata.setIncludeFilter(isIncludeFilter);
-        // If imei=imei comes in filter condition then we need to
-        // skip processing of right expression.
-        // This flow has reached here assuming that this is a single
-        // column expression.
-        // we need to check if the other expression contains column
-        // expression or not in depth.
-        CarbonDimension dimension = columnExpression.getDimension();
-        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)
-            || FilterUtil.isExpressionNeedsToResolved(rightExp, 
isIncludeFilter) &&
-            dimension.hasEncoding(Encoding.DICTIONARY) && !dimension
-            .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          isExpressionResolve = true;
-        } else {
-          //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.
-          dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
-              
FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
-              metadata);
-        }
-      } else if (rightExp instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) rightExp;
-        metadata.setColumnExpression(columnExpression);
-        metadata.setExpression(leftExp);
-        metadata.setIncludeFilter(isIncludeFilter);
-        if (columnExpression.getDataType().equals(DataType.TIMESTAMP) ||
-            columnExpression.getDataType().equals(DataType.DATE)) {
-          isExpressionResolve = true;
-        } else {
-          // if imei=imei comes in filter condition then we need to
-          // skip processing of right expression.
-          // This flow has reached here assuming that this is a single
-          // column expression.
-          // we need to check if the other expression contains column
-          // expression or not in depth.
-          if (FilterUtil.checkIfExpressionContainsColumn(leftExp)) {
-            isExpressionResolve = true;
-          } else {
-
-            dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
-                
FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
-                metadata);
-
-          }
-        }
-      } else {
-        isExpressionResolve = true;
-      }
-    }
-    if (isExpressionResolve && exp instanceof ConditionalExpression) {
-      ConditionalExpression conditionalExpression = (ConditionalExpression) 
exp;
-      List<ColumnExpression> columnList = 
conditionalExpression.getColumnList();
-      metadata.setColumnExpression(columnList.get(0));
-      metadata.setExpression(exp);
-      metadata.setIncludeFilter(isIncludeFilter);
-      if (!columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) ||
-          
columnList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
-            
FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnList.get(0)), 
metadata);
-      } else if 
(columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) &&
-          !(columnList.get(0).getDimension().getDataType() == DataType.STRUCT 
||
-              columnList.get(0).getDimension().getDataType() == 
DataType.ARRAY)) {
-        dimColResolvedFilterInfo.setFilterValues(
-            FilterUtil.getFilterListForAllValues(absoluteTableIdentifier, exp, 
columnList.get(0),
-                isIncludeFilter));
-
-        
dimColResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
-        
dimColResolvedFilterInfo.setDimension(columnList.get(0).getDimension());
-      }
-    }
-
-  }
-
-  /**
-   * Left node will not be presentin this scenario
-   *
-   * @return left node of type FilterResolverIntf instance
-   */
-  public FilterResolverIntf getLeft() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  /**
-   * Right node will not be presentin this scenario
-   *
-   * @return left node of type FilterResolverIntf instance
-   */
-  @Override public FilterResolverIntf getRight() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  /**
-   * Method will return the DimColumnResolvedFilterInfo instance which consists
-   * the mapping of the respective dimension and its surrogates involved in
-   * filter expression.
-   *
-   * @return DimColumnResolvedFilterInfo
-   */
-  public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
-    return dimColResolvedFilterInfo;
-  }
-
-  /**
-   * method will calculates the start key based on the filter surrogates
-   */
-  public void getStartKey(long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray, List<long[]> 
startKeyList) {
-    
FilterUtil.getStartKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
-        startKey, startKeyList);
-    FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo,
-        setOfStartKeyByteArray);
-  }
-
-  /**
-   * get the start key based on the filter surrogates
-   */
-  @Override public void getEndKey(SegmentProperties segmentProperties, long[] 
endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray, List<long[]> 
endKeyList) {
-    
FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
 endKeys,
-        segmentProperties, endKeyList);
-    FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo,
-        setOfEndKeyByteArray);
-  }
-
-  /**
-   * Method will return the executer type for particular conditional resolver
-   * basically two types of executers will be formed for the conditional query.
-   *
-   * @return the filter executer type
-   */
-  @Override public FilterExecuterType getFilterExecuterType() {
-    switch (exp.getFilterExpressionType()) {
-      case NOT_EQUALS:
-      case NOT_IN:
-        return FilterExecuterType.EXCLUDE;
-
-      default:
-        return FilterExecuterType.INCLUDE;
-    }
-
-  }
-
-  @Override public Expression getFilterExpression() {
-    // TODO Auto-generated method stub
-    return exp;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/FilterResolverIntf.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/FilterResolverIntf.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/FilterResolverIntf.java
deleted file mode 100644
index a04984b..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/FilterResolverIntf.java
+++ /dev/null
@@ -1,100 +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.apache.carbondata.scan.filter.resolver;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import java.util.SortedMap;
-
-import org.apache.carbondata.core.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.scan.expression.Expression;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.filter.intf.FilterExecuterType;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public interface FilterResolverIntf extends Serializable {
-
-  /**
-   * This API will resolve the filter expression and generates the
-   * dictionaries for executing/evaluating the filter expressions in the
-   * executer layer.
-   *
-   * @throws IOException
-   * @throws FilterUnsupportedException
-   */
-  void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
-      throws IOException, FilterUnsupportedException;
-
-  /**
-   * This API will provide the left column filter expression
-   * inorder to resolve the left expression filter.
-   *
-   * @return FilterResolverIntf
-   */
-  FilterResolverIntf getLeft();
-
-  /**
-   * API will provide the right column filter expression inorder to resolve
-   * the right expression filter.
-   *
-   * @return FilterResolverIntf
-   */
-  FilterResolverIntf getRight();
-
-  /**
-   * API will return the resolved filter instance, this instance will provide
-   * the resolved surrogates based on the applied filter
-   *
-   * @return DimColumnResolvedFilterInfo object
-   */
-  DimColumnResolvedFilterInfo getDimColResolvedFilterInfo();
-
-  /**
-   * API will get the start key based on the filter applied based on the key 
generator
-   *
-   * @param startKey
-   * @param setOfStartKeyByteArray
-   */
-  void getStartKey(long[] startKey, SortedMap<Integer, byte[]> 
setOfStartKeyByteArray,
-      List<long[]> startKeyList);
-
-  /**
-   * API will read the end key based on the max surrogate of
-   * particular dimension column
-   *
-   * @param endKeys
-   * @param setOfEndKeyByteArray
-   * @return
-   */
-  void getEndKey(SegmentProperties segmentProperties, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray, List<long[]> 
endKeyList);
-
-  /**
-   * API will return the filter executer type which will be used to evaluate
-   * the resolved filter while query execution
-   *
-   * @return FilterExecuterType.
-   */
-  FilterExecuterType getFilterExecuterType();
-
-  Expression getFilterExpression();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
deleted file mode 100644
index 786c2d4..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
+++ /dev/null
@@ -1,115 +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.apache.carbondata.scan.filter.resolver;
-
-import java.util.List;
-import java.util.SortedMap;
-
-import org.apache.carbondata.core.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.scan.expression.BinaryExpression;
-import org.apache.carbondata.scan.expression.Expression;
-import org.apache.carbondata.scan.filter.intf.ExpressionType;
-import org.apache.carbondata.scan.filter.intf.FilterExecuterType;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public class LogicalFilterResolverImpl implements FilterResolverIntf {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 5734382980564402914L;
-
-  protected FilterResolverIntf leftEvalutor;
-
-  protected FilterResolverIntf rightEvalutor;
-
-  protected ExpressionType filterExpressionType;
-
-  private BinaryExpression filterExpression;
-
-  public LogicalFilterResolverImpl(FilterResolverIntf leftEvalutor,
-      FilterResolverIntf rightEvalutor, BinaryExpression currentExpression) {
-    this.leftEvalutor = leftEvalutor;
-    this.rightEvalutor = rightEvalutor;
-    this.filterExpressionType = currentExpression.getFilterExpressionType();
-    this.filterExpression = currentExpression;
-  }
-
-  /**
-   * Logical filter resolver will return the left and right filter expresison
-   * node for filter evaluation, so in this instance no implementation is 
required.
-   *
-   * @param absoluteTableIdentifier
-   */
-  @Override public void resolve(AbsoluteTableIdentifier 
absoluteTableIdentifier) {
-
-  }
-
-  /**
-   * Since its a binary condition expresion the getLeft method will get the 
left
-   * node of filter expression
-   *
-   * @return FilterResolverIntf.
-   */
-  public FilterResolverIntf getLeft() {
-    return leftEvalutor;
-  }
-
-  /**
-   * Since its a binary condition expresion the getRight method will get the 
left
-   * node of filter expression
-   *
-   * @return FilterResolverIntf.
-   */
-  public FilterResolverIntf getRight() {
-    return rightEvalutor;
-  }
-
-  @Override public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
-    return null;
-  }
-
-  @Override
-  public void getStartKey(long[] startKey, SortedMap<Integer, byte[]> 
setOfStartKeyByteArray,
-      List<long[]> startKeyList) {
-
-  }
-
-  @Override public void getEndKey(SegmentProperties segmentProperties, long[] 
endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray, List<long[]> 
endKeyList) {
-
-  }
-
-  @Override public FilterExecuterType getFilterExecuterType() {
-    switch (filterExpressionType) {
-      case OR:
-        return FilterExecuterType.OR;
-      case AND:
-        return FilterExecuterType.AND;
-
-      default:
-        return null;
-    }
-  }
-
-  @Override public Expression getFilterExpression() {
-    return filterExpression;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java
deleted file mode 100644
index bad43ba..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java
+++ /dev/null
@@ -1,136 +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.apache.carbondata.scan.filter.resolver;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.scan.expression.ColumnExpression;
-import org.apache.carbondata.scan.expression.Expression;
-import org.apache.carbondata.scan.expression.conditional.ConditionalExpression;
-import org.apache.carbondata.scan.filter.intf.FilterExecuterType;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-
-public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl {
-
-  private static final long serialVersionUID = 176122729713729929L;
-
-  private List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
-  private List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
-  private AbsoluteTableIdentifier tableIdentifier;
-
-  public RowLevelFilterResolverImpl(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;
-  }
-
-  /**
-   * 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 ConditionalExpression) {
-      ConditionalExpression conditionalExpression = (ConditionalExpression) 
exp;
-      List<ColumnExpression> columnList = 
conditionalExpression.getColumnList();
-      for (ColumnExpression columnExpression : columnList) {
-        if (columnExpression.isDimension()) {
-          dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
-          
dimColumnEvaluatorInfo.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-          dimColumnEvaluatorInfo.setRowIndex(index++);
-          dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
-          dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
-          dimColEvaluatorInfoList.add(dimColumnEvaluatorInfo);
-        } else {
-          msrColumnEvalutorInfo = new MeasureColumnResolvedFilterInfo();
-          msrColumnEvalutorInfo.setRowIndex(index++);
-          msrColumnEvalutorInfo
-              .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-          
msrColumnEvalutorInfo.setType(columnExpression.getCarbonColumn().getDataType());
-          msrColEvalutorInfoList.add(msrColumnEvalutorInfo);
-        }
-      }
-    }
-  }
-
-  /**
-   * 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
-   */
-  @Override public FilterExecuterType getFilterExecuterType() {
-    return FilterExecuterType.ROWLEVEL;
-  }
-
-  /**
-   * Method will the read filter expression corresponding to the resolver.
-   * This method is required in row level executer inorder to evaluate the 
filter
-   * expression against spark, as mentioned above row level is a special type
-   * filter resolver.
-   *
-   * @return Expression
-   */
-  public Expression getFilterExpresion() {
-    return exp;
-  }
-
-  /**
-   * 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;
-  }
-
-  /**
-   * Method will return table information which will be required for retrieving
-   * dictionary cache inorder to read all the members of respective dimension.
-   *
-   * @return AbsoluteTableIdentifier
-   */
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
deleted file mode 100644
index 54b4e96..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ /dev/null
@@ -1,266 +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.apache.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.apache.carbondata.core.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.metadata.Encoding;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.scan.expression.ColumnExpression;
-import org.apache.carbondata.scan.expression.Expression;
-import org.apache.carbondata.scan.expression.ExpressionResult;
-import 
org.apache.carbondata.scan.expression.conditional.BinaryConditionalExpression;
-import 
org.apache.carbondata.scan.expression.exception.FilterIllegalMemberException;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.expression.logical.BinaryLogicalExpression;
-import org.apache.carbondata.scan.filter.DimColumnFilterInfo;
-import org.apache.carbondata.scan.filter.FilterUtil;
-import org.apache.carbondata.scan.filter.intf.FilterExecuterType;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import 
org.apache.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 
level
-   * conditional expressions.
-   *
-   * @return
-   */
-  public byte[][] getFilterRangeValues(SegmentProperties segmentProperties) {
-
-    if (null != dimColEvaluatorInfoList.get(0).getFilterValues() && 
!dimColEvaluatorInfoList.get(0)
-        .getDimension().hasEncoding(Encoding.DICTIONARY)) {
-      List<byte[]> noDictFilterValuesList =
-          
dimColEvaluatorInfoList.get(0).getFilterValues().getNoDictionaryFilterValuesList();
-      return noDictFilterValuesList.toArray((new 
byte[noDictFilterValuesList.size()][]));
-    } else if (null != dimColEvaluatorInfoList.get(0).getFilterValues() && 
dimColEvaluatorInfoList
-        .get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-      return 
FilterUtil.getKeyArray(this.dimColEvaluatorInfoList.get(0).getFilterValues(),
-          this.dimColEvaluatorInfoList.get(0).getDimension(),
-          segmentProperties);
-    }
-    return null;
-
-  }
-
-  /**
-   * method will get the start key based on the filter surrogates
-   *
-   * @return start IndexKey
-   */
-  public void getStartKey(long[] startKey,
-      SortedMap<Integer, byte[]> noDictStartKeys, List<long[]> startKeyList) {
-    
FilterUtil.getStartKey(dimColEvaluatorInfoList.get(0).getDimensionResolvedFilterInstance(),
-        startKey, startKeyList);
-    FilterUtil
-        .getStartKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), 
noDictStartKeys);
-  }
-
-  /**
-   * method will get the start key based on the filter surrogates
-   *
-   * @return end IndexKey
-   */
-  @Override public void getEndKey(SegmentProperties segmentProperties, long[] 
endKeys,
-      SortedMap<Integer, byte[]> noDicEndKeys, List<long[]> endKeyList) {
-    
FilterUtil.getEndKey(dimColEvaluatorInfoList.get(0).getDimensionResolvedFilterInstance(),
-        endKeys, segmentProperties, endKeyList);
-    FilterUtil
-        .getEndKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), 
noDicEndKeys);
-  }
-
-  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);
-    boolean invalidRowsPresent = false;
-    for (ExpressionResult result : listOfExpressionResults) {
-      try {
-        if (result.getString() == null) {
-          
filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL.getBytes());
-          continue;
-        }
-        filterValuesList.add(result.getString().getBytes());
-      } catch (FilterIllegalMemberException e) {
-        // Any invalid member while evaluation shall be ignored, system will 
log the
-        // error only once since all rows the evaluation happens so inorder to 
avoid
-        // too much log inforation only once the log will be printed.
-        FilterUtil.logError(e, invalidRowsPresent);
-      }
-    }
-    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)
-      throws FilterUnsupportedException {
-    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.setRowIndex(index++);
-          dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
-          dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
-          if 
(columnExpression.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-            
filterInfo.setFilterList(getDirectSurrogateValues(columnExpression));
-          } else {
-            
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
-              .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-          
msrColumnEvalutorInfo.setType(columnExpression.getCarbonColumn().getDataType());
-          msrColEvalutorInfoList.add(msrColumnEvalutorInfo);
-        }
-      }
-    }
-  }
-
-  private List<Integer> getDirectSurrogateValues(ColumnExpression 
columnExpression)
-      throws FilterUnsupportedException {
-    List<ExpressionResult> listOfExpressionResults = new 
ArrayList<ExpressionResult>(20);
-    DirectDictionaryGenerator directDictionaryGenerator = 
DirectDictionaryKeyGeneratorFactory
-        
.getDirectDictionaryGenerator(columnExpression.getDimension().getDataType());
-
-    if (this.getFilterExpression() instanceof BinaryConditionalExpression) {
-      listOfExpressionResults =
-          ((BinaryConditionalExpression) 
this.getFilterExpression()).getLiterals();
-    }
-    List<Integer> filterValuesList = new ArrayList<Integer>(20);
-    try {
-      // if any filter member provided by user is invalid throw error else
-      // system can display inconsistent result.
-      for (ExpressionResult result : listOfExpressionResults) {
-        filterValuesList.add(directDictionaryGenerator
-            .generateDirectSurrogateKey(result.getString(),
-                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-      }
-    } catch (FilterIllegalMemberException e) {
-      throw new FilterUnsupportedException(e);
-    }
-    return filterValuesList;
-  }
-
-  /**
-   * 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/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
deleted file mode 100644
index 1a1cc1d..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.scan.filter.resolver.metadata;
-
-import org.apache.carbondata.core.AbsoluteTableIdentifier;
-import org.apache.carbondata.scan.expression.ColumnExpression;
-import org.apache.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/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
deleted file mode 100644
index ec59f24..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
+++ /dev/null
@@ -1,143 +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.apache.carbondata.scan.filter.resolver.resolverinfo;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.filter.DimColumnFilterInfo;
-import 
org.apache.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
-
-public class DimColumnResolvedFilterInfo implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 3428115141211084114L;
-
-  /**
-   * column index in file
-   */
-  private int columnIndex = -1;
-
-  /**
-   * rowIndex
-   */
-  private int rowIndex = -1;
-
-  private boolean isDimensionExistsInCurrentSilce = true;
-
-  private String defaultValue;
-
-  private CarbonDimension dimension;
-
-  /**
-   * 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 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 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 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 String getDefaultValue() {
-    return defaultValue;
-  }
-
-  public void setDefaultValue(String defaultValue) {
-    this.defaultValue = defaultValue;
-  }
-
-  public void 
populateFilterInfoBasedOnColumnType(ResolvedFilterInfoVisitorIntf visitor,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException, 
IOException {
-    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/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
deleted file mode 100644
index 32505fb..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
+++ /dev/null
@@ -1,60 +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.apache.carbondata.scan.filter.resolver.resolverinfo;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.metadata.DataType;
-
-public class MeasureColumnResolvedFilterInfo implements Serializable {
-
-  private static final long serialVersionUID = 4222568289115151561L;
-
-  private int columnIndex = -1;
-
-  private int rowIndex = -1;
-
-  private 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 DataType getType() {
-    return type;
-  }
-
-  public void setType(DataType dataType) {
-    this.type = dataType;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
deleted file mode 100644
index 9af4b71..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
+++ /dev/null
@@ -1,101 +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.apache.carbondata.scan.filter.resolver.resolverinfo.visitor;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.scan.expression.ColumnExpression;
-import 
org.apache.carbondata.scan.expression.exception.FilterIllegalMemberException;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.filter.DimColumnFilterInfo;
-import 
org.apache.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public class CustomTypeDictionaryVisitor implements 
ResolvedFilterInfoVisitorIntf {
-
-  /**
-   * 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,if exception occurs while evaluating
-   * filter models.
-   */
-  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo 
visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException {
-    DimColumnFilterInfo resolvedFilterObject = null;
-
-    List<String> evaluateResultListFinal;
-    try {
-      evaluateResultListFinal = 
metadata.getExpression().evaluate(null).getListAsString();
-    } catch (FilterIllegalMemberException e) {
-      throw new FilterUnsupportedException(e);
-    }
-    resolvedFilterObject = 
getDirectDictionaryValKeyMemberForFilter(metadata.getColumnExpression(),
-        evaluateResultListFinal, metadata.isIncludeFilter());
-    if (!metadata.isIncludeFilter() && null != resolvedFilterObject && 
!resolvedFilterObject
-        
.getFilterList().contains(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY))
 {
-      // Adding default surrogate key of null member inorder to not display 
the same while
-      // displaying the report as per hive compatibility.
-      resolvedFilterObject.getFilterList()
-          .add(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY);
-      Collections.sort(resolvedFilterObject.getFilterList());
-    }
-    visitableObj.setFilterValues(resolvedFilterObject);
-  }
-
-  private DimColumnFilterInfo getDirectDictionaryValKeyMemberForFilter(
-      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
-    getSurrogateValuesForDictionary(evaluateResultListFinal, surrogates,
-        directDictionaryGenerator);
-
-    Collections.sort(surrogates);
-    DimColumnFilterInfo columnFilterInfo = null;
-    if (surrogates.size() > 0) {
-      columnFilterInfo = new DimColumnFilterInfo();
-      columnFilterInfo.setIncludeFilter(isIncludeFilter);
-      columnFilterInfo.setFilterList(surrogates);
-    }
-    return columnFilterInfo;
-  }
-
-  private void getSurrogateValuesForDictionary(List<String> 
evaluateResultListFinal,
-      List<Integer> surrogates, DirectDictionaryGenerator 
directDictionaryGenerator) {
-    String timeFormat = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-    for (String filterMember : evaluateResultListFinal) {
-      surrogates
-          
.add(directDictionaryGenerator.generateDirectSurrogateKey(filterMember, 
timeFormat));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
deleted file mode 100644
index 532af60..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
+++ /dev/null
@@ -1,67 +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.apache.carbondata.scan.filter.resolver.resolverinfo.visitor;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import 
org.apache.carbondata.scan.expression.exception.FilterIllegalMemberException;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.filter.DimColumnFilterInfo;
-import org.apache.carbondata.scan.filter.FilterUtil;
-import 
org.apache.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public class DictionaryColumnVisitor implements ResolvedFilterInfoVisitorIntf {
-
-  /**
-   * 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 FilterUnsupportedException,if exception occurs while evaluating
-   * filter models.
-   * @throws IOException
-   * @throws FilterUnsupportedException
-   */
-  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo 
visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException, 
IOException {
-    DimColumnFilterInfo resolvedFilterObject = null;
-    List<String> evaluateResultListFinal;
-    try {
-      evaluateResultListFinal = 
metadata.getExpression().evaluate(null).getListAsString();
-    } catch (FilterIllegalMemberException e) {
-      throw new FilterUnsupportedException(e);
-    }
-    resolvedFilterObject = FilterUtil
-        .getFilterValues(metadata.getTableIdentifier(), 
metadata.getColumnExpression(),
-            evaluateResultListFinal, metadata.isIncludeFilter());
-    if (!metadata.isIncludeFilter() && null != resolvedFilterObject) {
-      // Adding default surrogate key of null member inorder to not display 
the same while
-      // displaying the report as per hive compatibility.
-      resolvedFilterObject.getFilterList()
-          .add(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY);
-      Collections.sort(resolvedFilterObject.getFilterList());
-    }
-    visitableObj.setFilterValues(resolvedFilterObject);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
deleted file mode 100644
index ffb02fc..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.scan.filter.resolver.resolverinfo.visitor;
-
-import org.apache.carbondata.core.metadata.Encoding;
-import org.apache.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/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
deleted file mode 100644
index 035fd0d..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
+++ /dev/null
@@ -1,64 +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.apache.carbondata.scan.filter.resolver.resolverinfo.visitor;
-
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import 
org.apache.carbondata.scan.expression.exception.FilterIllegalMemberException;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.filter.DimColumnFilterInfo;
-import org.apache.carbondata.scan.filter.FilterUtil;
-import 
org.apache.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
-import 
org.apache.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public class NoDictionaryTypeVisitor implements ResolvedFilterInfoVisitorIntf {
-
-  /**
-   * 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,if exception occurs while evaluating
-   * filter models.
-   */
-  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo 
visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException {
-    DimColumnFilterInfo resolvedFilterObject = null;
-    List<String> evaluateResultListFinal;
-    try {
-      evaluateResultListFinal = 
metadata.getExpression().evaluate(null).getListAsString();
-      // Adding default  null member inorder to not display the same while
-      // displaying the report as per hive compatibility.
-      if (!metadata.isIncludeFilter() && !evaluateResultListFinal
-          .contains(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
-        evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-      }
-    } catch (FilterIllegalMemberException e) {
-      throw new FilterUnsupportedException(e);
-    }
-    resolvedFilterObject = FilterUtil
-        .getNoDictionaryValKeyMemberForFilter(evaluateResultListFinal, 
metadata.isIncludeFilter());
-    visitableObj.setFilterValues(resolvedFilterObject);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
 
b/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
deleted file mode 100644
index 2c46bde..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
+++ /dev/null
@@ -1,42 +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.apache.carbondata.scan.filter.resolver.resolverinfo.visitor;
-
-import java.io.IOException;
-
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import 
org.apache.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
-import 
org.apache.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 FilterUnsupportedException
-   */
-  void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException, 
IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/model/CarbonQueryPlan.java 
b/core/src/main/java/org/apache/carbondata/scan/model/CarbonQueryPlan.java
deleted file mode 100644
index c77e59e..0000000
--- a/core/src/main/java/org/apache/carbondata/scan/model/CarbonQueryPlan.java
+++ /dev/null
@@ -1,156 +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.apache.carbondata.scan.model;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.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;
-
-  /**
-   * Database name
-   */
-  private String databaseName;
-
-  /**
-   * Table name
-   */
-  private String tableName;
-
-  /**
-   * 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);
-
-  /**
-   * expression
-   */
-  private Expression expression;
-
-  /**
-   * queryId
-   */
-  private String queryId;
-
-  /**
-   * If it is raw detail query, no need to aggregate in backend. And it 
returns with dictionary data
-   * with out decoding.
-   */
-  private boolean rawDetailQuery;
-
-  /**
-   * Constructor created with database name and table name.
-   *
-   * @param databaseName
-   * @param tableName
-   */
-  public CarbonQueryPlan(String databaseName, String tableName) {
-    this.tableName = tableName;
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * @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 databaseName
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * @return the tableName
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  public String getQueryId() {
-    return queryId;
-  }
-
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  public boolean isRawDetailQuery() {
-    return rawDetailQuery;
-  }
-
-  public void setRawDetailQuery(boolean rawDetailQuery) {
-    this.rawDetailQuery = rawDetailQuery;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/model/QueryColumn.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/model/QueryColumn.java 
b/core/src/main/java/org/apache/carbondata/scan/model/QueryColumn.java
deleted file mode 100644
index d026227..0000000
--- a/core/src/main/java/org/apache/carbondata/scan/model/QueryColumn.java
+++ /dev/null
@@ -1,68 +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.apache.carbondata.scan.model;
-
-import java.io.Serializable;
-
-/**
- * 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;
-
-  /**
-   * query order in which result of the query will be send
-   */
-  private int queryOrder;
-
-  public QueryColumn(String columnName) {
-    this.columnName = columnName;
-  }
-
-  /**
-   * @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;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/model/QueryDimension.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/model/QueryDimension.java 
b/core/src/main/java/org/apache/carbondata/scan/model/QueryDimension.java
deleted file mode 100644
index 9c3002d..0000000
--- a/core/src/main/java/org/apache/carbondata/scan/model/QueryDimension.java
+++ /dev/null
@@ -1,58 +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.apache.carbondata.scan.model;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.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 columnName) {
-    super(columnName);
-  }
-
-  /**
-   * @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/952cf517/core/src/main/java/org/apache/carbondata/scan/model/QueryMeasure.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/model/QueryMeasure.java 
b/core/src/main/java/org/apache/carbondata/scan/model/QueryMeasure.java
deleted file mode 100644
index f01fa9e..0000000
--- a/core/src/main/java/org/apache/carbondata/scan/model/QueryMeasure.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.apache.carbondata.scan.model;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.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 columnName) {
-    super(columnName);
-  }
-
-  /**
-   * @return the measure
-   */
-  public CarbonMeasure getMeasure() {
-    return measure;
-  }
-
-  /**
-   * @param measure the measure to set
-   */
-  public void setMeasure(CarbonMeasure measure) {
-    this.measure = measure;
-  }
-
-}

Reply via email to