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

[34/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/query/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
deleted file mode 100644
index 4897736..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.filter.resolver;
-
-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.encoder.Encoding;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.visitor.FilterInfoTypeVisitorFactory;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-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 QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
-      throws FilterUnsupportedException {
-    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.
-        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)||
-            FilterUtil.isExpressionNeedsToResolved(rightExp,isIncludeFilter) &&
-            columnExpression.getDimension().hasEncoding(Encoding.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.TimestampType)) {
-          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()
-              == org.carbondata.core.carbon.metadata.datatype.DataType.STRUCT
-              || columnList.get(0).getDimension().getDataType()
-              == org.carbondata.core.carbon.metadata.datatype.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(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
-    if (null == dimColResolvedFilterInfo.getStarIndexKey()) {
-      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-          setOfStartKeyByteArray);
-    }
-  }
-
-  /**
-   * 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[]> setOfEndKeyByteArray) {
-    if (null == dimColResolvedFilterInfo.getEndIndexKey()) {
-      try {
-        FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
-            absoluteTableIdentifier, endKeys, segmentProperties);
-        FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-            setOfEndKeyByteArray);
-      } catch (QueryExecutionException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-    }
-  }
-
-  /**
-   * 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/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java b/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
deleted file mode 100644
index a8f1df5..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.filter.resolver;
-
-import java.io.Serializable;
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.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 QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) throws 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 segmentProperties
-   * @param startKey
-   * @param setOfStartKeyByteArray
-   */
-  void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray);
-
-  /**
-   * API will read the end key based on the max surrogate of
-   * particular dimension column
-   *
-   * @param setOfEndKeyByteArray
-   * @param endKeys
-   * @return
-   */
-  void getEndKey(SegmentProperties segmentProperties, AbsoluteTableIdentifier tableIdentifier,
-      long[] endKeys, SortedMap<Integer, byte[]> setOfEndKeyByteArray);
-
-  /**
-   * 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/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
deleted file mode 100644
index 8bf1395..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.filter.resolver;
-
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public class LogicalFilterResolverImpl implements FilterResolverIntf {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 5734382980564402914L;
-
-  protected FilterResolverIntf leftEvalutor;
-
-  protected FilterResolverIntf rightEvalutor;
-
-  protected ExpressionType filterExpressionType;
-
-  public LogicalFilterResolverImpl(FilterResolverIntf leftEvalutor,
-      FilterResolverIntf rightEvalutor, ExpressionType filterExpressionType) {
-    this.leftEvalutor = leftEvalutor;
-    this.rightEvalutor = rightEvalutor;
-    this.filterExpressionType = filterExpressionType;
-  }
-
-  /**
-   * 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(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
-
-  }
-
-  @Override public void getEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
-  }
-
-  @Override public FilterExecuterType getFilterExecuterType() {
-    switch (filterExpressionType) {
-      case OR:
-        return FilterExecuterType.OR;
-      case AND:
-        return FilterExecuterType.AND;
-
-      default:
-        return null;
-    }
-  }
-
-  @Override public Expression getFilterExpression() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java
deleted file mode 100644
index 516de9c..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.filter.resolver;
-
-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.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class RestructureFilterResolverImpl implements FilterResolverIntf {
-  /**
-   *
-   */
-  private static final long serialVersionUID = -5399656036192814524L;
-
-  protected DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo;
-
-  private Expression exp;
-
-  private String defaultValue;
-
-  private int surrogate;
-
-  private boolean isExpressionResolve;
-
-  private boolean isIncludeFilter;
-
-  public RestructureFilterResolverImpl(Expression exp, String defaultValue, int surrogate,
-      boolean isExpressionResolve, boolean isIncludeFilter) {
-    dimColumnResolvedFilterInfo = new DimColumnResolvedFilterInfo();
-    this.exp = exp;
-    this.defaultValue = defaultValue;
-    this.surrogate = surrogate;
-    this.isExpressionResolve = isExpressionResolve;
-    this.isIncludeFilter = isIncludeFilter;
-  }
-
-  /**
-   * Method will resolve the filters and it will replace the newly added dimension with default
-   * value
-   *
-   * @param absoluteTableIdentifier
-   */
-  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
-
-    DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo = new DimColumnResolvedFilterInfo();
-    if (!this.isExpressionResolve && exp instanceof BinaryConditionalExpression) {
-      BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression) exp;
-      Expression left = binaryConditionalExpression.getLeft();
-      Expression right = binaryConditionalExpression.getRight();
-      if (left instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) left;
-        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
-          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(right)) {
-            isExpressionResolve = true;
-          } else {
-            dimColumnResolvedFilterInfo
-                .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
-            dimColumnResolvedFilterInfo.setFilterValues(
-                FilterUtil.getFilterListForRS(right, columnExpression, defaultValue, surrogate));
-          }
-        }
-      } else if (right instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) right;
-        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
-          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 (checkIfExpressionContainsColumn(left)) {
-            isExpressionResolve = true;
-          } else {
-            dimColumnResolvedFilterInfo
-                .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
-            dimColumnResolvedFilterInfo.setFilterValues(
-                FilterUtil.getFilterListForRS(left, columnExpression, defaultValue, surrogate));
-          }
-        }
-      }
-    }
-    if (this.isExpressionResolve && exp instanceof ConditionalExpression) {
-      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
-      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
-      dimColumnResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
-      dimColumnResolvedFilterInfo.setFilterValues(FilterUtil
-          .getFilterListForAllMembersRS(exp, columnList.get(0), defaultValue, surrogate,
-              isIncludeFilter));
-    }
-
-  }
-
-  /**
-   * This method will check if a given expression contains a column expression recursively.
-   *
-   * @return boolean
-   */
-  private boolean checkIfExpressionContainsColumn(Expression expression) {
-    if (expression instanceof ColumnExpression) {
-      return true;
-    }
-    for (Expression child : expression.getChildren()) {
-      if (checkIfExpressionContainsColumn(child)) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  @Override public FilterResolverIntf getLeft() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @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 dimColumnResolvedFilterInfo;
-  }
-
-  /**
-   * For restructure resolver no implementation is required for getting
-   * the start key since it already has default values
-   */
-  @Override public void getStartKey(SegmentProperties segmentProperties, long[] startKeys,
-      SortedMap<Integer, byte[]> noDicStartKeys) {
-
-  }
-
-  /**
-   * For restructure resolver no implementation is required for getting
-   * the end  key since it already has default values
-   *
-   * @return IndexKey.
-   */
-  @Override public void getEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> noDicEndKeys) {
-  }
-
-  /**
-   * Method will get the executer type inorder to create filter executer tree
-   *
-   * @return FilterExecuterType
-   */
-  @Override public FilterExecuterType getFilterExecuterType() {
-    return FilterExecuterType.RESTRUCTURE;
-  }
-
-  @Override public Expression getFilterExpression() {
-    // TODO Auto-generated method stub
-    return exp;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.java
deleted file mode 100644
index ceccb12..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.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.carbondata.query.filter.resolver;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-
-public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 176122729713729929L;
-  protected boolean isExpressionResolve;
-  protected boolean isIncludeFilter;
-
-  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.setAggregator(
-              ((CarbonMeasure) columnExpression.getCarbonColumn()).getAggregateFunction());
-          msrColumnEvalutorInfo
-              .setColumnIndex(((CarbonMeasure) 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/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
deleted file mode 100644
index 6e220b7..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.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.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
-import org.carbondata.query.expression.logical.BinaryLogicalExpression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-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/query/filter/resolver/metadata/FilterResolverMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/metadata/FilterResolverMetadata.java b/core/src/main/java/org/carbondata/query/filter/resolver/metadata/FilterResolverMetadata.java
deleted file mode 100644
index 3f813de..0000000
--- a/core/src/main/java/org/carbondata/query/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.carbondata.query.filter.resolver.metadata;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.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/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
deleted file mode 100644
index 4d1f827..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.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.query.complex.querytypes.GenericQueryType;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.visitable.ResolvedFilterInfoVisitable;
-import org.carbondata.query.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-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/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
deleted file mode 100644
index e6877d5..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.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/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
deleted file mode 100644
index 7f31cb4..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.filter.resolver.resolverinfo.visitable;
-
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.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/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
deleted file mode 100644
index 52d0e26..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.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.query.expression.ColumnExpression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-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/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
deleted file mode 100644
index ee3bd4c..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.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.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-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/query/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
deleted file mode 100644
index 3a38569..0000000
--- a/core/src/main/java/org/carbondata/query/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.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.query.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/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
deleted file mode 100644
index fbafa7d..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-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/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
deleted file mode 100644
index 44d942d..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.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;
-}