You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2014/10/25 20:18:15 UTC

[26/28] TAJO-1125: Separate logical plan and optimizer into a maven module.

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
deleted file mode 100644
index 457f651..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
+++ /dev/null
@@ -1,520 +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.tajo.engine.eval;
-
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.tajo.algebra.ColumnReferenceExpr;
-import org.apache.tajo.algebra.NamedExpr;
-import org.apache.tajo.algebra.OpType;
-import org.apache.tajo.annotation.Nullable;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.engine.planner.ExprFinder;
-import org.apache.tajo.engine.planner.LogicalPlan;
-import org.apache.tajo.engine.planner.Target;
-import org.apache.tajo.exception.InternalException;
-import org.apache.tajo.util.TUtil;
-
-import java.util.*;
-
-public class EvalTreeUtil {
-
-  public static void changeColumnRef(EvalNode node, String oldName, String newName) {
-    node.postOrder(new ChangeColumnRefVisitor(oldName, newName));
-  }
-
-  public static int replace(EvalNode expr, EvalNode targetExpr, EvalNode tobeReplaced) {
-    EvalReplaceVisitor replacer = new EvalReplaceVisitor(targetExpr, tobeReplaced);
-    ReplaceContext context = new ReplaceContext();
-    replacer.visitChild(context, expr, new Stack<EvalNode>());
-    return context.countOfReplaces;
-  }
-
-  private static class ReplaceContext {
-    int countOfReplaces = 0;
-  }
-
-  public static class EvalReplaceVisitor extends BasicEvalNodeVisitor<ReplaceContext, EvalNode> {
-    private EvalNode target;
-    private EvalNode tobeReplaced;
-
-    public EvalReplaceVisitor(EvalNode target, EvalNode tobeReplaced) {
-      this.target = target;
-      this.tobeReplaced = tobeReplaced;
-    }
-
-    @Override
-    public EvalNode visitChild(ReplaceContext context, EvalNode evalNode, Stack<EvalNode> stack) {
-      super.visitChild(context, evalNode, stack);
-
-      if (evalNode.equals(target)) {
-        context.countOfReplaces++;
-
-        EvalNode parent = stack.peek();
-
-        if (parent instanceof BetweenPredicateEval) {
-          BetweenPredicateEval between = (BetweenPredicateEval) parent;
-          if (between.getPredicand().equals(evalNode)) {
-            between.setPredicand(tobeReplaced);
-          }
-          if (between.getBegin().equals(evalNode)) {
-            between.setBegin(tobeReplaced);
-          }
-          if (between.getEnd().equals(evalNode)) {
-            between.setEnd(tobeReplaced);
-          }
-
-        } else if (parent instanceof CaseWhenEval) {
-          CaseWhenEval caseWhen = (CaseWhenEval) parent;
-
-          // Here, we need to only consider only 'Else'
-          // because IfElseEval is handled in the below condition.
-          if (caseWhen.hasElse() && caseWhen.getElse().equals(evalNode)) {
-            caseWhen.setElseResult(tobeReplaced);
-          }
-        } else if (parent instanceof CaseWhenEval.IfThenEval) {
-          CaseWhenEval.IfThenEval ifThen = (CaseWhenEval.IfThenEval) parent;
-          if (ifThen.getCondition().equals(evalNode)) {
-            ifThen.setCondition(tobeReplaced);
-          }
-          if (ifThen.getResult().equals(evalNode)) {
-            ifThen.setResult(tobeReplaced);
-          }
-       } else if (parent instanceof FunctionEval) {
-          FunctionEval functionEval = (FunctionEval) parent;
-          EvalNode [] arguments = functionEval.getArgs();
-          for (int i = 0; i < arguments.length; i++) {
-            if (arguments[i].equals(evalNode)) {
-              arguments[i] = tobeReplaced;
-            }
-          }
-          functionEval.setArgs(arguments);
-
-        } else if (parent instanceof UnaryEval) {
-          if (((UnaryEval)parent).getChild().equals(evalNode)) {
-            ((UnaryEval)parent).setChild(tobeReplaced);
-          }
-        } else if (parent instanceof BinaryEval) {
-          BinaryEval binary = (BinaryEval) parent;
-          if (binary.getLeftExpr() != null && binary.getLeftExpr().equals(evalNode)) {
-            binary.setLeftExpr(tobeReplaced);
-          }
-          if (binary.getRightExpr() != null && binary.getRightExpr().equals(evalNode)) {
-            binary.setRightExpr(tobeReplaced);
-          }
-        }
-      }
-
-      return evalNode;
-    }
-  }
-
-  /**
-   * It finds unique columns from a EvalNode.
-   */
-  public static LinkedHashSet<Column> findUniqueColumns(EvalNode node) {
-    UniqueColumnFinder finder = new UniqueColumnFinder();
-    node.postOrder(finder);
-    return finder.getColumnRefs();
-  }
-  
-  public static List<Column> findAllColumnRefs(EvalNode node) {
-    AllColumnRefFinder finder = new AllColumnRefFinder();
-    node.postOrder(finder);
-    return finder.getColumnRefs();
-  }
-  
-  public static Schema getSchemaByTargets(Schema inputSchema, Target [] targets) 
-      throws InternalException {
-    Schema schema = new Schema();
-    for (Target target : targets) {
-      schema.addColumn(
-          target.hasAlias() ? target.getAlias() : target.getEvalTree().getName(),
-          getDomainByExpr(inputSchema, target.getEvalTree()));
-    }
-    
-    return schema;
-  }
-
-  public static String columnsToStr(Collection<Column> columns) {
-    StringBuilder sb = new StringBuilder();
-    String prefix = "";
-    for (Column column: columns) {
-      sb.append(prefix).append(column.getQualifiedName());
-      prefix = ",";
-    }
-
-    return sb.toString();
-  }
-  
-  public static DataType getDomainByExpr(Schema inputSchema, EvalNode expr)
-      throws InternalException {
-    switch (expr.getType()) {
-    case AND:      
-    case OR:
-    case EQUAL:
-    case NOT_EQUAL:
-    case LTH:
-    case LEQ:
-    case GTH:
-    case GEQ:
-    case PLUS:
-    case MINUS:
-    case MULTIPLY:
-    case DIVIDE:
-    case CONST:
-    case FUNCTION:
-        return expr.getValueType();
-
-    case FIELD:
-      FieldEval fieldEval = (FieldEval) expr;
-      return inputSchema.getColumn(fieldEval.getName()).getDataType();
-
-      
-    default:
-      throw new InternalException("Unknown expr type: " 
-          + expr.getType().toString());
-    }
-  }
-
-  /**
-   * Return all exprs to refer columns corresponding to the target.
-   *
-   * @param expr
-   * @param target to be found
-   * @return a list of exprs
-   */
-  public static Collection<EvalNode> getContainExpr(EvalNode expr, Column target) {
-    Set<EvalNode> exprSet = Sets.newHashSet();
-    getContainExpr(expr, target, exprSet);
-    return exprSet;
-  }
-  
-  /**
-   * Return the counter to count the number of expression types individually.
-   *  
-   * @param expr
-   * @return
-   */
-  public static Map<EvalType, Integer> getExprCounters(EvalNode expr) {
-    VariableCounter counter = new VariableCounter();
-    expr.postOrder(counter);
-    return counter.getCounter();
-  }
-  
-  private static void getContainExpr(EvalNode expr, Column target, Set<EvalNode> exprSet) {
-    switch (expr.getType()) {
-    case EQUAL:
-    case LTH:
-    case LEQ:
-    case GTH:
-    case GEQ:
-    case NOT_EQUAL:
-      if (containColumnRef(expr, target)) {          
-        exprSet.add(expr);
-      }
-    }    
-  }
-  
-  /**
-   * Examine if the expr contains the column reference corresponding 
-   * to the target column
-   */
-  public static boolean containColumnRef(EvalNode expr, Column target) {
-    Set<Column> exprSet = findUniqueColumns(expr);
-    return exprSet.contains(target);
-  }
-
-  /**
-   * If a given expression is join condition, it returns TRUE. Otherwise, it returns FALSE.
-   *
-   * If three conditions are satisfied, we can recognize the expression as a equi join condition.
-   * <ol>
-   *   <li>An expression is an equal comparison expression.</li>
-   *   <li>Both terms in an expression are column references.</li>
-   *   <li>Both column references point come from different tables</li>
-   * </ol>
-   *
-   * For theta join condition, we will use "an expression is a predicate including column references which come
-   * from different two tables" instead of the first rule.
-   *
-   * @param expr EvalNode to be evaluated
-   * @param includeThetaJoin If true, it will return equi as well as non-equi join conditions.
-   *                         Otherwise, it only returns equi-join conditions.
-   * @return True if it is join condition.
-   */
-  public static boolean isJoinQual(EvalNode expr, boolean includeThetaJoin) {
-    return isJoinQual(null, expr, includeThetaJoin);
-  }
-
-  /**
-   * If a given expression is join condition, it returns TRUE. Otherwise, it returns FALSE.
-   *
-   * If three conditions are satisfied, we can recognize the expression as a equi join condition.
-   * <ol>
-   *   <li>An expression is an equal comparison expression.</li>
-   *   <li>Both terms in an expression are column references.</li>
-   *   <li>Both column references point come from different tables</li>
-   * </ol>
-   *
-   * For theta join condition, we will use "an expression is a predicate including column references which come
-   * from different two tables" instead of the first rule.
-   *
-   * @param block if block is not null, it tracks the lineage of aliased name derived from complex expressions.
-   * @param expr EvalNode to be evaluated
-   * @param includeThetaJoin If true, it will return equi as well as non-equi join conditions.
-   *                         Otherwise, it only returns equi-join conditions.
-   * @return True if it is join condition.
-   */
-  public static boolean isJoinQual(@Nullable LogicalPlan.QueryBlock block, EvalNode expr, boolean includeThetaJoin) {
-
-    if (expr instanceof BinaryEval) {
-      boolean joinComparator;
-      if (includeThetaJoin) {
-        joinComparator = EvalType.isComparisonOperator(expr.getType());
-      } else {
-        joinComparator = expr.getType() == EvalType.EQUAL;
-      }
-
-      BinaryEval binaryEval = (BinaryEval) expr;
-      boolean isBothTermFields = isSingleColumn(binaryEval.getLeftExpr()) && isSingleColumn(binaryEval.getRightExpr());
-
-      Set<Column> leftColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getLeftExpr());
-      Set<Column> rightColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getRightExpr());
-
-      boolean ensureColumnsOfDifferentTables = false;
-
-      if (leftColumns.size() == 1 && rightColumns.size() == 1) { // ensure there is only one column of each table
-        Column leftColumn = leftColumns.iterator().next();
-        Column rightColumn = rightColumns.iterator().next();
-
-        String leftQualifier = CatalogUtil.extractQualifier(leftColumn.getQualifiedName());
-        String rightQualifier = CatalogUtil.extractQualifier(rightColumn.getQualifiedName());
-
-        // if block is given, it will track an original expression of each term in order to decide whether
-        // this expression is a join condition, or not.
-        if (block != null) {
-          boolean leftQualified = CatalogUtil.isFQColumnName(leftColumn.getQualifiedName());
-          boolean rightQualified = CatalogUtil.isFQColumnName(rightColumn.getQualifiedName());
-
-          if (!leftQualified) { // if left one is aliased name
-
-            // getting original expression of left term
-            NamedExpr rawExpr = block.getNamedExprsManager().getNamedExpr(leftColumn.getQualifiedName());
-            Set<ColumnReferenceExpr> foundColumns = ExprFinder.finds(rawExpr.getExpr(), OpType.Column);
-
-            // ensure there is only one column of an original expression
-            if (foundColumns.size() == 1) {
-              leftQualifier = CatalogUtil.extractQualifier(foundColumns.iterator().next().getCanonicalName());
-            }
-          }
-          if (!rightQualified) { // if right one is aliased name
-
-            // getting original expression of right term
-            NamedExpr rawExpr = block.getNamedExprsManager().getNamedExpr(rightColumn.getQualifiedName());
-            Set<ColumnReferenceExpr> foundColumns = ExprFinder.finds(rawExpr.getExpr(), OpType.Column);
-
-            // ensure there is only one column of an original expression
-            if (foundColumns.size() == 1) {
-              rightQualifier = CatalogUtil.extractQualifier(foundColumns.iterator().next().getCanonicalName());
-            }
-          }
-        }
-
-        // if columns of both term is different to each other, it will be true.
-        ensureColumnsOfDifferentTables = !leftQualifier.equals(rightQualifier);
-      }
-
-      return joinComparator && isBothTermFields && ensureColumnsOfDifferentTables;
-    } else {
-      return false;
-    }
-  }
-
-  static boolean isSingleColumn(EvalNode evalNode) {
-    return EvalTreeUtil.findUniqueColumns(evalNode).size() == 1;
-  }
-  
-  public static class ChangeColumnRefVisitor implements EvalNodeVisitor {    
-    private final String findColumn;
-    private final String toBeChanged;
-    
-    public ChangeColumnRefVisitor(String oldName, String newName) {
-      this.findColumn = oldName;
-      this.toBeChanged = newName;
-    }
-    
-    @Override
-    public void visit(EvalNode node) {
-      if (node.type == EvalType.FIELD) {
-        FieldEval field = (FieldEval) node;
-        if (field.getColumnName().equals(findColumn)
-            || field.getName().equals(findColumn)) {
-          field.replaceColumnRef(toBeChanged);
-        }
-      }
-    }    
-  }
-  
-  public static class AllColumnRefFinder implements EvalNodeVisitor {
-    private List<Column> colList = new ArrayList<Column>();
-    private FieldEval field = null;
-    
-    @Override
-    public void visit(EvalNode node) {
-      if (node.getType() == EvalType.FIELD) {
-        field = (FieldEval) node;
-        colList.add(field.getColumnRef());
-      } 
-    }
-    
-    public List<Column> getColumnRefs() {
-      return this.colList;
-    }
-  }
-  
-  public static class UniqueColumnFinder implements EvalNodeVisitor {
-    private LinkedHashSet<Column> columnSet = Sets.newLinkedHashSet();
-    private FieldEval field = null;
-    
-    @Override
-    public void visit(EvalNode node) {
-      if (node.getType() == EvalType.FIELD) {
-        field = (FieldEval) node;
-        columnSet.add(field.getColumnRef());
-      }
-    }
-    
-    public LinkedHashSet<Column> getColumnRefs() {
-      return this.columnSet;
-    }
-  }
-  
-  public static class VariableCounter implements EvalNodeVisitor {
-    private final Map<EvalType, Integer> counter;
-    
-    public VariableCounter() {
-      counter = Maps.newHashMap();
-      counter.put(EvalType.FUNCTION, 0);
-      counter.put(EvalType.FIELD, 0);
-    }
-    
-    @Override
-    public void visit(EvalNode node) {
-      if (counter.containsKey(node.getType())) {
-        int val = counter.get(node.getType());
-        val++;
-        counter.put(node.getType(), val);
-      }
-    }
-    
-    public Map<EvalType, Integer> getCounter() {
-      return counter;
-    }
-  }
-
-  public static Set<AggregationFunctionCallEval> findDistinctAggFunction(EvalNode expr) {
-    AllAggFunctionFinder finder = new AllAggFunctionFinder();
-    expr.postOrder(finder);
-    return finder.getAggregationFunction();
-  }
-
-  public static class AllAggFunctionFinder implements EvalNodeVisitor {
-    private Set<AggregationFunctionCallEval> aggFucntions = Sets.newHashSet();
-    private AggregationFunctionCallEval field = null;
-
-    @Override
-    public void visit(EvalNode node) {
-      if (node.getType() == EvalType.AGG_FUNCTION) {
-        field = (AggregationFunctionCallEval) node;
-        aggFucntions.add(field);
-      }
-    }
-
-    public Set<AggregationFunctionCallEval> getAggregationFunction() {
-      return this.aggFucntions;
-    }
-  }
-
-  public static <T extends EvalNode> Collection<T> findEvalsByType(EvalNode evalNode, EvalType type) {
-    EvalFinder finder = new EvalFinder(type);
-    finder.visitChild(null, evalNode, new Stack<EvalNode>());
-    return (Collection<T>) finder.evalNodes;
-  }
-
-  public static <T extends EvalNode> Collection<T> findOuterJoinSensitiveEvals(EvalNode evalNode) {
-    OuterJoinSensitiveEvalFinder finder = new OuterJoinSensitiveEvalFinder();
-    finder.visitChild(null, evalNode, new Stack<EvalNode>());
-    return (Collection<T>) finder.evalNodes;
-  }
-
-  public static class EvalFinder extends BasicEvalNodeVisitor<Object, Object> {
-    private EvalType targetType;
-    List<EvalNode> evalNodes = TUtil.newList();
-
-    public EvalFinder(EvalType targetType) {
-      this.targetType = targetType;
-    }
-
-    @Override
-    public Object visitChild(Object context, EvalNode evalNode, Stack<EvalNode> stack) {
-      super.visitChild(context, evalNode, stack);
-
-      if (evalNode.type == targetType) {
-        evalNodes.add(evalNode);
-      }
-
-      return evalNode;
-    }
-  }
-
-  public static class OuterJoinSensitiveEvalFinder extends BasicEvalNodeVisitor<Object, Object> {
-    private List<EvalNode> evalNodes = TUtil.newList();
-
-    @Override
-    public Object visitChild(Object context, EvalNode evalNode, Stack<EvalNode> stack) {
-      super.visitChild(context, evalNode, stack);
-
-      if (evalNode.type == EvalType.CASE) {
-        evalNodes.add(evalNode);
-      } else if (evalNode.type == EvalType.FUNCTION) {
-        FunctionEval functionEval = (FunctionEval)evalNode;
-        if ("coalesce".equals(functionEval.getName())) {
-          evalNodes.add(evalNode);
-        }
-      } else if (evalNode.type == EvalType.IS_NULL) {
-        evalNodes.add(evalNode);
-      }
-
-      return evalNode;
-    }
-  }
-
-  public static boolean checkIfCanBeConstant(EvalNode evalNode) {
-    return findUniqueColumns(evalNode).size() == 0 && findDistinctAggFunction(evalNode).size() == 0;
-  }
-
-  public static Datum evaluateImmediately(EvalNode evalNode) {
-    return evalNode.eval(null, null);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalType.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalType.java
deleted file mode 100644
index d533510..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalType.java
+++ /dev/null
@@ -1,172 +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.tajo.engine.eval;
-
-public enum EvalType {
-  // Unary expression
-  NOT(NotEval.class, "!"),
-
-  // Binary expression
-  AND(BinaryEval.class),
-  OR(BinaryEval.class),
-  EQUAL(BinaryEval.class, "="),
-  IS_NULL(IsNullEval.class),
-  NOT_EQUAL(BinaryEval.class, "<>"),
-  LTH(BinaryEval.class, "<"),
-  LEQ(BinaryEval.class, "<="),
-  GTH(BinaryEval.class, ">"),
-  GEQ(BinaryEval.class, ">="),
-  PLUS(BinaryEval.class, "+"),
-  MINUS(BinaryEval.class, "-"),
-  MODULAR(BinaryEval.class, "%"),
-  MULTIPLY(BinaryEval.class, "*"),
-  DIVIDE(BinaryEval.class, "/"),
-
-  // Binary Bitwise expressions
-  BIT_AND(BinaryEval.class, "&"),
-  BIT_OR(BinaryEval.class, "|"),
-  BIT_XOR(BinaryEval.class, "|"),
-
-  // Function
-  WINDOW_FUNCTION(WindowFunctionEval.class),
-  AGG_FUNCTION(AggregationFunctionCallEval.class),
-  FUNCTION(GeneralFunctionEval.class),
-
-  // String operator or pattern matching predicates
-  LIKE(LikePredicateEval.class),
-  SIMILAR_TO(SimilarToPredicateEval.class),
-  REGEX(RegexPredicateEval.class),
-  CONCATENATE(BinaryEval.class, "||"),
-
-  // Other predicates
-  BETWEEN(BetweenPredicateEval.class),
-  CASE(CaseWhenEval.class),
-  IF_THEN(CaseWhenEval.IfThenEval.class),
-  IN(InEval.class),
-
-  // Value or Reference
-  SIGNED(SignedEval.class),
-  CAST(CastEval.class),
-  ROW_CONSTANT(RowConstantEval.class),
-  FIELD(FieldEval.class),
-  CONST(ConstEval.class);
-
-  private Class<? extends EvalNode> baseClass;
-  private String operatorName;
-
-  EvalType(Class<? extends EvalNode> type) {
-    this.baseClass = type;
-  }
-
-  EvalType(Class<? extends EvalNode> type, String text) {
-    this(type);
-    this.operatorName = text;
-  }
-
-  public static boolean isUnaryOperator(EvalType type) {
-    boolean match = false;
-
-    match |= type == CAST;
-    match |= type == IS_NULL;
-    match |= type == NOT;
-    match |= type == SIGNED;
-
-    return match;
-  }
-
-  public static boolean isBinaryOperator(EvalType type) {
-    boolean match = false;
-
-    match |= isArithmeticOperator(type);
-    match |= isLogicalOperator(type) && type != NOT;
-    match |= isComparisonOperator(type) && type != BETWEEN;
-
-    match |= type == CONCATENATE;
-    match |= type == IN;
-    match |= type == LIKE;
-    match |= type == REGEX;
-    match |= type == SIMILAR_TO;
-
-    return match;
-  }
-
-  public static boolean isLogicalOperator(EvalType type) {
-    boolean match = false;
-
-    match |= type == AND;
-    match |= type == OR;
-    match |= type == NOT;
-
-    return match;
-  }
-
-  public static boolean isComparisonOperator(EvalType type) {
-    boolean match = false;
-
-    match |= type == EQUAL;
-    match |= type == NOT_EQUAL;
-    match |= type == LTH;
-    match |= type == LEQ;
-    match |= type == GTH;
-    match |= type == GEQ;
-    match |= type == BETWEEN;
-
-    return match;
-  }
-
-  public static boolean isArithmeticOperator(EvalType type) {
-    boolean match = false;
-
-    match |= type == PLUS;
-    match |= type == MINUS;
-    match |= type == MULTIPLY;
-    match |= type == DIVIDE;
-    match |= type == MODULAR;
-
-    return match;
-  }
-
-  public static boolean isFunction(EvalType type) {
-    boolean match = false;
-
-    match |= type == FUNCTION;
-    match |= type == AGG_FUNCTION;
-    match |= type == WINDOW_FUNCTION;
-
-    return match;
-  }
-
-  public static boolean isStringPatternMatchOperator(EvalType type) {
-    boolean match = false;
-
-    match |= type == LIKE;
-    match |= type == SIMILAR_TO;
-    match |= type == REGEX;
-
-    return match;
-  }
-
-  public String getOperatorName() {
-    return operatorName != null ? operatorName : name();
-  }
-
-  public Class<? extends EvalNode> getBaseClass() {
-    return this.baseClass;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/FieldEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
deleted file mode 100644
index 5cedbca..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
+++ /dev/null
@@ -1,129 +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.tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.storage.Tuple;
-
-public class FieldEval extends EvalNode implements Cloneable {
-	@Expose private Column column;
-	@Expose	private int fieldId = -1;
-	
-	public FieldEval(String columnName, DataType domain) {
-		super(EvalType.FIELD);
-		this.column = new Column(columnName, domain);
-	}
-	
-	public FieldEval(Column column) {
-	  super(EvalType.FIELD);
-	  this.column = column;
-	}
-
-	@Override
-	public Datum eval(Schema schema, Tuple tuple) {
-	  if (fieldId == -1) {
-      // TODO - column namespace should be improved to simplify name handling and resolving.
-      if (column.hasQualifier()) {
-        fieldId = schema.getColumnId(column.getQualifiedName());
-      } else {
-        fieldId = schema.getColumnIdByName(column.getSimpleName());
-      }
-      if (fieldId == -1) {
-        throw new IllegalStateException("No Such Column Reference: " + column + ", schema: " + schema);
-      }
-	  }
-	  return tuple.get(fieldId);
-  }
-
-  @Override
-	public DataType getValueType() {
-		return column.getDataType();
-	}
-
-  @Override
-  public int childNum() {
-    return 0;
-  }
-
-  @Override
-  public EvalNode getChild(int idx) {
-    return null;
-  }
-
-  public Column getColumnRef() {
-    return column;
-  }
-	
-	public String getQualifier() {
-	  return column.getQualifier();
-	}
-	
-	public String getColumnName() {
-	  return column.getSimpleName();
-	}
-	
-	public void replaceColumnRef(String columnName) {
-	  this.column = new Column(columnName, this.column.getDataType());
-	}
-
-	@Override
-	public String getName() {
-		return this.column.getQualifiedName();
-	}
-	
-	public String toString() {
-	  return this.column.toString();
-	}
-	
-  public boolean equals(Object obj) {
-    if (obj instanceof FieldEval) {
-      FieldEval other = (FieldEval) obj;
-      
-      return column.equals(other.column);      
-    }
-    return false;
-  }
-  
-  @Override
-  public int hashCode() {
-    return column.hashCode();
-  }
-  
-  @Override
-  public Object clone() throws CloneNotSupportedException {
-    FieldEval eval = (FieldEval) super.clone();
-    eval.column = this.column;
-    eval.fieldId = fieldId;
-    
-    return eval;
-  }
-
-  public void preOrder(EvalNodeVisitor visitor) {
-    visitor.visit(this);
-  }
-  
-  @Override
-  public void postOrder(EvalNodeVisitor visitor) {
-    visitor.visit(this);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
deleted file mode 100644
index b449040..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.eval;
-
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.FunctionDesc;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.util.TUtil;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType.DISTINCT_AGGREGATION;
-import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType.DISTINCT_UDA;
-
-public abstract class FunctionEval extends EvalNode implements Cloneable {
-  public static enum ParamType {
-    CONSTANT, VARIABLE, NULL
-  }
-
-  @Expose protected FunctionDesc funcDesc;
-	@Expose protected EvalNode [] argEvals;
-
-	public FunctionEval(EvalType type, FunctionDesc funcDesc, EvalNode[] argEvals) {
-		super(type);
-		this.funcDesc = funcDesc;
-    Preconditions.checkArgument(argEvals != null, "argEvals cannot be null");
-    this.argEvals = argEvals;
-	}
-
-  public FunctionDesc getFuncDesc() {
-    return funcDesc;
-  }
-
-  public ParamType [] getParamType() {
-    ParamType [] paramTypes = new ParamType[argEvals.length];
-    for (int i = 0; i < argEvals.length; i++) {
-      if (argEvals[i].getType() == EvalType.CONST) {
-        if (argEvals[i].getValueType().getType() == TajoDataTypes.Type.NULL_TYPE) {
-          paramTypes[i] = ParamType.NULL;
-        } else {
-          paramTypes[i] = ParamType.CONSTANT;
-        }
-      } else {
-        paramTypes[i] = ParamType.VARIABLE;
-      }
-    }
-    return paramTypes;
-  }
-
-  public boolean isDistinct() {
-    return funcDesc.getFuncType() == DISTINCT_AGGREGATION || funcDesc.getFuncType() == DISTINCT_UDA;
-  }
-
-	public EvalNode [] getArgs() {
-	  return this.argEvals;
-	}
-
-  public void setArg(int idx, EvalNode arg) {
-    this.argEvals[idx] = arg;
-  }
-
-  public void setArgs(EvalNode [] args) {
-    this.argEvals = args;
-  }
-
-  @Override
-  public int childNum() {
-    if (argEvals != null) {
-      return argEvals.length;
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public EvalNode getChild(int idx) {
-    return argEvals[idx];
-  }
-
-
-	public DataType getValueType() {
-		return this.funcDesc.getReturnType();
-	}
-
-	@Override
-	public abstract Datum eval(Schema schema, Tuple tuple);
-
-	@Override
-	public String getName() {
-		return funcDesc.getFunctionName();
-	}
-
-  @Override
-	public String toString() {
-		StringBuilder sb = new StringBuilder();
-		for(int i=0; i < argEvals.length; i++) {
-			sb.append(argEvals[i]);
-			if(i+1 < argEvals.length)
-				sb.append(",");
-		}
-		return funcDesc.getFunctionName() + "(" + (isDistinct() ? " distinct " : "") + sb+")";
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-	  if (obj instanceof FunctionEval) {
-      FunctionEval other = (FunctionEval) obj;
-
-      boolean b1 = this.type == other.type;
-      boolean b2 = TUtil.checkEquals(funcDesc, other.funcDesc);
-      boolean b3 = TUtil.checkEquals(argEvals, other.argEvals);
-      return b1 && b2 && b3;
-	  }
-	  
-	  return false;
-	}
-	
-	@Override
-	public int hashCode() {
-	  return Objects.hashCode(funcDesc, argEvals);
-	}
-	
-	@Override
-  public Object clone() throws CloneNotSupportedException {
-    FunctionEval eval = (FunctionEval) super.clone();
-    eval.funcDesc = (FunctionDesc) funcDesc.clone();
-    if (argEvals != null) {
-      eval.argEvals = new EvalNode[argEvals.length];
-      for (int i = 0; i < argEvals.length; i++) {
-        eval.argEvals[i] = (EvalNode) argEvals[i].clone();
-      }
-    }
-    return eval;
-  }
-	
-	@Override
-  public void preOrder(EvalNodeVisitor visitor) {
-    if (argEvals != null) {
-      for (EvalNode eval : argEvals) {
-        eval.postOrder(visitor);
-      }
-    }
-    visitor.visit(this);
-  }
-	
-	@Override
-	public void postOrder(EvalNodeVisitor visitor) {
-    if (argEvals != null) {
-      for (EvalNode eval : argEvals) {
-        eval.postOrder(visitor);
-      }
-    }
-	  visitor.visit(this);
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/GeneralFunctionEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/GeneralFunctionEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/GeneralFunctionEval.java
deleted file mode 100644
index 9446d70..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/GeneralFunctionEval.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.eval;
-
-import com.google.common.base.Objects;
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.FunctionDesc;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.engine.function.GeneralFunction;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.util.TUtil;
-
-public class GeneralFunctionEval extends FunctionEval {
-  @Expose protected GeneralFunction instance;
-  private Tuple params = null;
-
-	public GeneralFunctionEval(FunctionDesc desc, GeneralFunction instance, EvalNode[] givenArgs) {
-		super(EvalType.FUNCTION, desc, givenArgs);
-		this.instance = instance;
-    this.instance.init(getParamType());
-  }
-
-  /* (non-Javadoc)
-    * @see nta.query.executor.eval.Expr#evalVal(Tuple)
-    */
-	@Override
-	public Datum eval(Schema schema, Tuple tuple) {
-    if (this.params == null) {
-      params = new VTuple(argEvals.length);
-    }
-    if(argEvals != null) {
-      params.clear();
-      for(int i=0;i < argEvals.length; i++) {
-        params.put(i, argEvals[i].eval(schema, tuple));
-      }
-    }
-
-    return instance.eval(params);
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-	  if (obj instanceof GeneralFunctionEval) {
-      GeneralFunctionEval other = (GeneralFunctionEval) obj;
-      return super.equals(other) &&
-          TUtil.checkEquals(instance, other.instance);
-	  }
-	  
-	  return false;
-	}
-	
-	@Override
-	public int hashCode() {
-	  return Objects.hashCode(funcDesc, instance);
-	}
-	
-	@Override
-  public Object clone() throws CloneNotSupportedException {
-    GeneralFunctionEval eval = (GeneralFunctionEval) super.clone();
-    eval.instance = (GeneralFunction) instance.clone();
-    return eval;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/InEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/InEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/InEval.java
deleted file mode 100644
index 51097e6..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/InEval.java
+++ /dev/null
@@ -1,86 +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.tajo.engine.eval;
-
-
-import com.google.common.collect.Sets;
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.Tuple;
-
-import java.util.Set;
-
-public class InEval extends BinaryEval {
-  private static final TajoDataTypes.DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN);
-
-  @Expose private boolean not;
-  Set<Datum> values;
-
-  public InEval(EvalNode lhs, RowConstantEval valueList, boolean not) {
-    super(EvalType.IN, lhs, valueList);
-    this.not = not;
-  }
-
-  public boolean isNot() {
-    return this.not;
-  }
-
-  @Override
-  public TajoDataTypes.DataType getValueType() {
-    return RES_TYPE;
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    if (values == null) {
-      values = Sets.newHashSet(((RowConstantEval)rightExpr).getValues());
-    }
-
-    Datum leftValue = leftExpr.eval(schema, tuple);
-
-    if (leftValue.isNull()) {
-      return NullDatum.get();
-    }
-
-    return DatumFactory.createBool(not ^ values.contains(leftValue));
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof InEval) {
-      InEval other = (InEval) obj;
-      return super.equals(obj) && not == other.not;
-    }
-    return false;
-  }
-
-  public String toString() {
-    return leftExpr + " IN (" + rightExpr + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidEvalException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidEvalException.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidEvalException.java
deleted file mode 100644
index 96eda91..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidEvalException.java
+++ /dev/null
@@ -1,36 +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.tajo.engine.eval;
-
-public class InvalidEvalException extends RuntimeException {
-	private static final long serialVersionUID = -2897003028483298256L;
-
-	public InvalidEvalException() {
-	}
-
-	/**
-	 * @param message
-	 */
-	public InvalidEvalException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java
deleted file mode 100644
index 25567fd..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.Tuple;
-
-public class IsNullEval extends UnaryEval {
-  // it's just a hack to emulate a binary expression
-  private final static ConstEval DUMMY_EVAL = new ConstEval(DatumFactory.createBool(true));
-  private static final DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN);
-
-  // persistent variables
-  @Expose private boolean isNot;
-
-  public IsNullEval(boolean not, EvalNode predicand) {
-    super(EvalType.IS_NULL, predicand);
-    this.isNot = not;
-  }
-
-  @Override
-  public DataType getValueType() {
-    return RES_TYPE;
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public String toString() {
-    return child + " IS " + (isNot ? "NOT NULL" : "NULL");
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    boolean isNull = child.eval(schema, tuple).isNull();
-    return DatumFactory.createBool(isNot ^ isNull);
-  }
-
-  public boolean isNot() {
-    return isNot;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof IsNullEval) {
-      IsNullEval other = (IsNullEval) obj;
-      return super.equals(other) && isNot == other.isNot();
-    } else {
-      return false;
-    }
-  }
-
-  public Object clone() throws CloneNotSupportedException {
-    IsNullEval isNullEval = (IsNullEval) super.clone();
-    isNullEval.isNot = isNot;
-
-    return isNullEval;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/LikePredicateEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/LikePredicateEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/LikePredicateEval.java
deleted file mode 100644
index 535677f..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/LikePredicateEval.java
+++ /dev/null
@@ -1,54 +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.tajo.engine.eval;
-
-import org.apache.tajo.util.StringUtils;
-
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-
-public class LikePredicateEval extends PatternMatchPredicateEval {
-
-  public LikePredicateEval(boolean not, EvalNode field, ConstEval pattern) {
-    super(EvalType.LIKE, not, field, pattern, false);
-  }
-
-  public LikePredicateEval(boolean not, EvalNode field, ConstEval pattern, boolean caseSensitive) {
-    super(EvalType.LIKE, not, field, pattern, caseSensitive);
-  }
-
-  protected void compile(String pattern) throws PatternSyntaxException {
-    String escaped = StringUtils.escapeRegexp(pattern);
-    String regex = escaped.replace("_", ".").replace("%", ".*");
-    int flags = Pattern.DOTALL;
-    if (caseInsensitive) {
-      flags |= Pattern.CASE_INSENSITIVE;
-    }
-    this.compiled = Pattern.compile(regex, flags);
-  }
-
-  public boolean isLeadingWildCard() {
-    return pattern.indexOf(".*") == 0;
-  }
-
-  @Override
-  public String toString() {
-    return leftExpr.toString() + (caseInsensitive ? "ILIKE" : "LIKE") + "'" + pattern +"'";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java
deleted file mode 100644
index 72c85d2..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.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.apache.tajo.engine.eval;
-
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.Tuple;
-
-public class NotEval extends UnaryEval implements Cloneable {
-  private static final DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN);
-
-  public NotEval(EvalNode child) {
-    super(EvalType.NOT, child);
-  }
-
-  @Override
-  public DataType getValueType() {
-    return RES_TYPE;
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    Datum datum = child.eval(schema, tuple);
-    return !datum.isNull() ? DatumFactory.createBool(!datum.asBool()) : datum;
-  }
-
-  @Override
-  public String toString() {
-    return "NOT " + child.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java
deleted file mode 100644
index 40966e5..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.eval;
-
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.exception.InvalidOperationException;
-import org.apache.tajo.storage.Tuple;
-
-public class PartialBinaryExpr extends BinaryEval {
-  
-  public PartialBinaryExpr(EvalType type) {
-    super(type);
-  }
-
-  public PartialBinaryExpr(EvalType type, EvalNode left, EvalNode right) {
-    super(type);
-    this.leftExpr = left;
-    this.rightExpr = right;
-    // skip to determine the result type
-  }
-
-  @Override
-  public DataType getValueType() {
-    return null;
-  }
-
-  @Override
-  public String getName() {
-    return "nonamed";
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    throw new InvalidOperationException("ERROR: the partial binary expression cannot be evluated: "
-            + this.toString());
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof PartialBinaryExpr) {
-      PartialBinaryExpr other = (PartialBinaryExpr) obj;
-      return type.equals(other.type) &&
-          leftExpr.equals(other.leftExpr) &&
-          rightExpr.equals(other.rightExpr);
-    }
-    return false;
-  }
-
-  public String toString() {
-    return 
-        (leftExpr != null ? leftExpr.toString() : "[EMPTY]") 
-        + " " + type + " " 
-        + (rightExpr != null ? rightExpr.toString() : "[EMPTY]");
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/PatternMatchPredicateEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/PatternMatchPredicateEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/PatternMatchPredicateEval.java
deleted file mode 100644
index 0a8e800..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/PatternMatchPredicateEval.java
+++ /dev/null
@@ -1,90 +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.tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.Tuple;
-
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-
-public abstract class PatternMatchPredicateEval extends BinaryEval {
-  private static final DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN);
-
-  @Expose protected boolean not;
-  @Expose protected String pattern;
-  @Expose protected boolean caseInsensitive;
-
-  // transient variables
-  protected Pattern compiled;
-
-  public PatternMatchPredicateEval(EvalType evalType, boolean not, EvalNode predicand, ConstEval pattern,
-                                   boolean caseInsensitive) {
-    super(evalType, predicand, pattern);
-    this.not = not;
-    this.pattern = pattern.getValue().asChars();
-    this.caseInsensitive = caseInsensitive;
-  }
-
-  public PatternMatchPredicateEval(EvalType evalType, boolean not, EvalNode field, ConstEval pattern) {
-    this(evalType, not, field, pattern, false);
-  }
-
-  abstract void compile(String pattern) throws PatternSyntaxException;
-
-  public boolean isNot() {
-    return not;
-  }
-
-  public boolean isCaseInsensitive() {
-    return caseInsensitive;
-  }
-
-  @Override
-  public DataType getValueType() {
-    return RES_TYPE;
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    if (this.compiled == null) {
-      compile(this.pattern);
-    }
-
-    Datum predicand = leftExpr.eval(schema, tuple);
-    if (predicand.isNull()) {
-      return NullDatum.get();
-    }
-
-    boolean matched = compiled.matcher(predicand.asChars()).matches();
-    return DatumFactory.createBool(matched ^ not);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/RegexPredicateEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/RegexPredicateEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/RegexPredicateEval.java
deleted file mode 100644
index f1e0241..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/RegexPredicateEval.java
+++ /dev/null
@@ -1,53 +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.tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-
-public class RegexPredicateEval extends PatternMatchPredicateEval {
-  @Expose private String operator;
-  public RegexPredicateEval(boolean not, EvalNode field, ConstEval pattern, boolean caseInsensitive) {
-    super(EvalType.REGEX, not, field, pattern, caseInsensitive);
-    StringBuilder sb = new StringBuilder();
-    if (not) {
-      sb.append("!");
-    }
-    sb.append("~");
-    if (caseInsensitive) {
-      sb.append("*");
-    }
-    this.operator = sb.toString();
-  }
-  
-  protected void compile(String regex) throws PatternSyntaxException {
-    int flags = Pattern.DOTALL;
-    if (caseInsensitive) {
-      flags |= Pattern.CASE_INSENSITIVE;
-    }
-    this.compiled = Pattern.compile(regex, flags);
-  }
-
-  @Override
-  public String toString() {
-    return leftExpr.toString() + operator + "'" + pattern +"'";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/RowConstantEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/RowConstantEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/RowConstantEval.java
deleted file mode 100644
index 5b21be0..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/RowConstantEval.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.util.TUtil;
-
-import static org.apache.tajo.common.TajoDataTypes.DataType;
-
-public class RowConstantEval extends EvalNode {
-  @Expose Datum [] values;
-
-  public RowConstantEval(Datum [] values) {
-    super(EvalType.ROW_CONSTANT);
-    this.values = values;
-  }
-
-  @Override
-  public DataType getValueType() {
-    return CatalogUtil.newSimpleDataType(values[0].type());
-  }
-
-  @Override
-  public int childNum() {
-    return 0;
-  }
-
-  @Override
-  public EvalNode getChild(int idx) {
-    return null;
-  }
-
-  @Override
-  public String getName() {
-    return "ROW";
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    return NullDatum.get();
-  }
-
-  public Datum [] getValues() {
-    return values;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof RowConstantEval) {
-      RowConstantEval other = (RowConstantEval) obj;
-      return TUtil.checkEquals(values, other.values);
-    }
-
-    return false;
-  }
-
-  public String toString() {
-    return TUtil.arrayToString(values);
-  }
-
-  public void preOrder(EvalNodeVisitor visitor) {
-    visitor.visit(this);
-  }
-
-  public void postOrder(EvalNodeVisitor visitor) {
-    visitor.visit(this);
-  }
-
-  @Override
-  public Object clone() throws CloneNotSupportedException {
-    RowConstantEval rowConstantEval = (RowConstantEval) super.clone();
-    if (values != null) {
-      rowConstantEval.values = new Datum[values.length];
-      System.arraycopy(values, 0, rowConstantEval.values, 0, values.length);
-    }
-    return rowConstantEval;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
deleted file mode 100644
index 80edf66..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
+++ /dev/null
@@ -1,86 +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.tajo.engine.eval;
-
-import com.google.common.base.Objects;
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NumericDatum;
-import org.apache.tajo.storage.Tuple;
-
-public class SignedEval extends UnaryEval implements Cloneable {
-  @Expose private boolean negative;
-
-  public SignedEval(boolean negative, EvalNode childEval) {
-    super(EvalType.SIGNED, childEval);
-    this.negative = negative;
-  }
-
-  public boolean isNegative() {
-    return negative;
-  }
-
-  @Override
-  public DataType getValueType() {
-    return child.getValueType();
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    NumericDatum result = child.eval(schema, tuple);
-    if (negative) {
-      return result.inverseSign();
-    }
-    return result;
-  }
-
-  @Override
-  public String toString() {
-    return (negative ? "-" : "+") + child.toString();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof SignedEval) {
-      SignedEval other = (SignedEval) obj;
-      return super.equals(other) && negative == other.negative;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(type, negative, child);
-  }
-
-  @Override
-  public Object clone() throws CloneNotSupportedException {
-    SignedEval signedEval = (SignedEval) super.clone();
-    signedEval.negative = negative;
-    return signedEval;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimilarToPredicateEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimilarToPredicateEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimilarToPredicateEval.java
deleted file mode 100644
index a690759..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimilarToPredicateEval.java
+++ /dev/null
@@ -1,48 +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.tajo.engine.eval;
-
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-
-public class SimilarToPredicateEval extends PatternMatchPredicateEval {
-  private static final String SIMILARTO_ESCAPE_SPATIAL_CHARACTERS = "([.])";
-
-  public SimilarToPredicateEval(boolean not, EvalNode field, ConstEval pattern,
-                                @SuppressWarnings("unused") boolean isCaseSensitive) {
-    super(EvalType.SIMILAR_TO, not, field, pattern, false);
-  }
-
-  public SimilarToPredicateEval(boolean not, EvalNode field, ConstEval pattern) {
-    super(EvalType.SIMILAR_TO, not, field, pattern);
-  }
-
-  @Override
-  protected void compile(String pattern) throws PatternSyntaxException {
-    String regex = pattern.replaceAll(SIMILARTO_ESCAPE_SPATIAL_CHARACTERS, "\\\\$1");
-    regex = regex.replace("_", ".").replace("%", ".*"); // transform some special characters to be 'like'.
-
-    this.compiled = Pattern.compile(regex, Pattern.DOTALL);
-  }
-  
-  @Override
-  public String toString() {
-    return leftExpr.toString() + " SIMILAR TO '" + pattern + "'";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimpleEvalNodeVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimpleEvalNodeVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimpleEvalNodeVisitor.java
deleted file mode 100644
index 7e7594a..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimpleEvalNodeVisitor.java
+++ /dev/null
@@ -1,172 +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.tajo.engine.eval;
-
-import com.google.common.base.Preconditions;
-import org.apache.tajo.exception.UnsupportedException;
-
-import java.util.Stack;
-
-/**
- * It provides simple visitor methods for an expression tree. Since <code>SimpleEvalNodeVisitor</code> provides
- * fewer visitor methods, it allows users to write a simple rewriter for expression trees.
- */
-public abstract class SimpleEvalNodeVisitor<CONTEXT> {
-
-  public EvalNode visit(CONTEXT context, EvalNode evalNode, Stack<EvalNode> stack) {
-    Preconditions.checkNotNull(evalNode);
-
-    EvalNode result;
-
-    if (evalNode instanceof UnaryEval) {
-      result = visitUnaryEval(context, stack, (UnaryEval) evalNode);
-    } else if (evalNode instanceof BinaryEval) {
-      result = visitBinaryEval(context, stack, (BinaryEval) evalNode);
-    } else {
-
-      switch (evalNode.getType()) {
-      // Column and Value reference expressions
-      case CONST:
-        result = visitConst(context, (ConstEval) evalNode, stack);
-        break;
-      case ROW_CONSTANT:
-        result = visitRowConstant(context, (RowConstantEval) evalNode, stack);
-        break;
-      case FIELD:
-        result = visitField(context, stack, (FieldEval) evalNode);
-        break;
-
-
-      // SQL standard predicates
-      case BETWEEN:
-        result = visitBetween(context, (BetweenPredicateEval) evalNode, stack);
-        break;
-      case CASE:
-        result = visitCaseWhen(context, (CaseWhenEval) evalNode, stack);
-        break;
-      case IF_THEN:
-        result = visitIfThen(context, (CaseWhenEval.IfThenEval) evalNode, stack);
-        break;
-
-      // Functions
-      case FUNCTION:
-        result = visitFuncCall(context, (FunctionEval) evalNode, stack);
-        break;
-      case AGG_FUNCTION:
-        result = visitFuncCall(context, (FunctionEval) evalNode, stack);
-        break;
-      case WINDOW_FUNCTION:
-        result = visitFuncCall(context, (FunctionEval) evalNode, stack);
-        break;
-
-      default:
-        throw new UnsupportedException("Unknown EvalType: " + evalNode);
-      }
-    }
-
-    return result;
-  }
-
-  protected EvalNode visitUnaryEval(CONTEXT context, Stack<EvalNode> stack, UnaryEval unaryEval) {
-    stack.push(unaryEval);
-    visit(context, unaryEval.getChild(), stack);
-    stack.pop();
-    return unaryEval;
-  }
-
-  protected EvalNode visitBinaryEval(CONTEXT context, Stack<EvalNode> stack, BinaryEval binaryEval) {
-    stack.push(binaryEval);
-    visit(context, binaryEval.getLeftExpr(), stack);
-    visit(context, binaryEval.getRightExpr(), stack);
-    stack.pop();
-    return binaryEval;
-  }
-
-  protected EvalNode visitDefaultFunctionEval(CONTEXT context, Stack<EvalNode> stack, FunctionEval functionEval) {
-    stack.push(functionEval);
-    if (functionEval.getArgs() != null) {
-      for (EvalNode arg : functionEval.getArgs()) {
-        visit(context, arg, stack);
-      }
-    }
-    stack.pop();
-    return functionEval;
-  }
-
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-  // Value and Literal
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-
-  protected EvalNode visitConst(CONTEXT context, ConstEval evalNode, Stack<EvalNode> stack) {
-    return evalNode;
-  }
-
-  protected EvalNode visitRowConstant(CONTEXT context, RowConstantEval evalNode, Stack<EvalNode> stack) {
-    return evalNode;
-  }
-
-  protected EvalNode visitField(CONTEXT context, Stack<EvalNode> stack, FieldEval evalNode) {
-    return evalNode;
-  }
-
-
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-  // SQL standard predicates
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-
-  protected EvalNode visitBetween(CONTEXT context, BetweenPredicateEval evalNode, Stack<EvalNode> stack) {
-    stack.push(evalNode);
-    visit(context, evalNode.getPredicand(), stack);
-    visit(context, evalNode.getBegin(), stack);
-    visit(context, evalNode.getEnd(), stack);
-    return evalNode;
-  }
-
-  protected EvalNode visitCaseWhen(CONTEXT context, CaseWhenEval evalNode, Stack<EvalNode> stack) {
-    stack.push(evalNode);
-    for (CaseWhenEval.IfThenEval ifThenEval : evalNode.getIfThenEvals()) {
-      visitIfThen(context, ifThenEval, stack);
-    }
-    if (evalNode.hasElse()) {
-      visit(context, evalNode.getElse(), stack);
-    }
-    stack.pop();
-    return evalNode;
-  }
-
-  protected EvalNode visitIfThen(CONTEXT context, CaseWhenEval.IfThenEval evalNode, Stack<EvalNode> stack) {
-    stack.push(evalNode);
-    visit(context, evalNode.getCondition(), stack);
-    visit(context, evalNode.getResult(), stack);
-    stack.pop();
-    return evalNode;
-  }
-
-  protected EvalNode visitInPredicate(CONTEXT context, InEval evalNode, Stack<EvalNode> stack) {
-    return visitBinaryEval(context, stack, evalNode);
-  }
-
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-  // Functions
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-
-  protected EvalNode visitFuncCall(CONTEXT context, FunctionEval evalNode, Stack<EvalNode> stack) {
-    return visitDefaultFunctionEval(context, stack, evalNode);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/UnaryEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/UnaryEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/UnaryEval.java
deleted file mode 100644
index e7ae112..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/UnaryEval.java
+++ /dev/null
@@ -1,107 +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.tajo.engine.eval;
-
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.storage.Tuple;
-
-public abstract class UnaryEval extends EvalNode implements Cloneable {
-  @Expose protected EvalNode child;
-
-  public UnaryEval(EvalType type) {
-    super(type);
-  }
-
-  public UnaryEval(EvalType type, EvalNode child) {
-    super(type);
-    this.child = child;
-  }
-
-  @Override
-  public int childNum() {
-    return 1;
-  }
-
-  public EvalNode getChild(int idx) {
-    Preconditions.checkArgument(idx == 0, "UnaryEval always has one child.");
-    return child;
-  }
-
-  public void setChild(EvalNode child) {
-    this.child = child;
-  }
-
-  public EvalNode getChild() {
-    return child;
-  }
-
-  @Override
-  public TajoDataTypes.DataType getValueType() {
-    return null;
-  }
-
-  @Override
-  public String getName() {
-    return null;
-  }
-
-  @Override
-  public <T extends Datum> T eval(Schema schema, Tuple tuple) {
-    return null;
-  }
-
-  @Override
-  public void preOrder(EvalNodeVisitor visitor) {
-    visitor.visit(this);
-    child.preOrder(visitor);
-  }
-
-  @Override
-  public void postOrder(EvalNodeVisitor visitor) {
-    child.postOrder(visitor);
-    visitor.visit(this);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof UnaryEval) {
-      UnaryEval another = (UnaryEval) obj;
-      return type == another.type && child.equals(another.child);
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(type, child);
-  }
-
-  @Override
-  public Object clone() throws CloneNotSupportedException {
-    UnaryEval unaryEval = (UnaryEval) super.clone();
-    unaryEval.child = (EvalNode) this.child.clone();
-    return unaryEval;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/eval/WindowFunctionEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/WindowFunctionEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/WindowFunctionEval.java
deleted file mode 100644
index 4057e70..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/WindowFunctionEval.java
+++ /dev/null
@@ -1,117 +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.tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.FunctionDesc;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SortSpec;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.engine.function.AggFunction;
-import org.apache.tajo.engine.function.FunctionContext;
-import org.apache.tajo.engine.planner.logical.WindowSpec;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.util.TUtil;
-
-public class WindowFunctionEval extends AggregationFunctionCallEval implements Cloneable {
-  @Expose private SortSpec [] sortSpecs;
-  @Expose WindowSpec.WindowFrame windowFrame;
-  private Tuple params;
-
-  public WindowFunctionEval(FunctionDesc desc, AggFunction instance, EvalNode[] givenArgs,
-                            WindowSpec.WindowFrame windowFrame) {
-    super(EvalType.WINDOW_FUNCTION, desc, instance, givenArgs);
-    this.windowFrame = windowFrame;
-  }
-
-  public boolean hasSortSpecs() {
-    return sortSpecs != null;
-  }
-
-  public void setSortSpecs(SortSpec [] sortSpecs) {
-    this.sortSpecs = sortSpecs;
-  }
-
-  public SortSpec [] getSortSpecs() {
-    return sortSpecs;
-  }
-
-  public WindowSpec.WindowFrame getWindowFrame() {
-    return windowFrame;
-  }
-
-  @Override
-  public Datum eval(Schema schema, Tuple tuple) {
-    throw new UnsupportedOperationException("Cannot execute eval() of aggregation function");
-  }
-
-  public void merge(FunctionContext context, Schema schema, Tuple tuple) {
-    if (params == null) {
-      this.params = new VTuple(argEvals.length);
-    }
-
-    if (argEvals != null) {
-      for (int i = 0; i < argEvals.length; i++) {
-        params.put(i, argEvals[i].eval(schema, tuple));
-      }
-    }
-
-    instance.eval(context, params);
-  }
-
-  public Datum terminate(FunctionContext context) {
-    return instance.terminate(context);
-  }
-
-  @Override
-  public DataType getValueType() {
-    return funcDesc.getReturnType();
-  }
-
-  @Override
-  public Object clone() throws CloneNotSupportedException {
-    WindowFunctionEval windowFunctionEval = (WindowFunctionEval) super.clone();
-    if (sortSpecs != null) {
-      windowFunctionEval.sortSpecs = new SortSpec[sortSpecs.length];
-      for (int i = 0; i < sortSpecs.length; i++) {
-        windowFunctionEval.sortSpecs[i] = (SortSpec) sortSpecs[i].clone();
-      }
-    }
-    return windowFunctionEval;
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    if (argEvals != null) {
-      for(int i=0; i < argEvals.length; i++) {
-        sb.append(argEvals[i]);
-        if(i+1 < argEvals.length)
-          sb.append(",");
-      }
-    }
-    sb.append(funcDesc.getFunctionName()).append("(").append(isDistinct() ? " distinct" : "").append(sb)
-        .append(")");
-    if (hasSortSpecs()) {
-      sb.append("ORDER BY ").append(TUtil.arrayToString(sortSpecs));
-    }
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java b/tajo-core/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java
deleted file mode 100644
index 3125e30..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java
+++ /dev/null
@@ -1,30 +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.tajo.engine.exception;
-
-public class AmbiguousFieldException extends InvalidQueryException {
-	private static final long serialVersionUID = 3102675985226352347L;
-
-	/**
-	 * @param fieldName
-	 */
-	public AmbiguousFieldException(String fieldName) {
-		super("ERROR: column name "+ fieldName + " is ambiguous");	
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java b/tajo-core/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java
deleted file mode 100644
index 66e9ee7..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.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.apache.tajo.engine.exception;
-
-public class IllegalQueryStatusException extends Exception {
-
-  public IllegalQueryStatusException() {
-
-  }
-
-  public IllegalQueryStatusException(String msg) {
-    super(msg);
-  }
-
-  public IllegalQueryStatusException(Exception e) {
-    super(e);
-  }
-
-  public IllegalQueryStatusException(String msg, Exception e) {
-    super(msg, e);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java b/tajo-core/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java
deleted file mode 100644
index fba691d..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java
+++ /dev/null
@@ -1,35 +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.tajo.engine.exception;
-
-public class InvalidQueryException extends RuntimeException {
-	private static final long serialVersionUID = -7085849718839416246L;
-
-  public InvalidQueryException() {
-    super();
-  }
-
-	public InvalidQueryException(String message) {
-    super(message);
-  }
-	
-	public InvalidQueryException(Throwable t) {
-		super(t);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b143f991/tajo-core/src/main/java/org/apache/tajo/engine/exception/NoSuchColumnException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/exception/NoSuchColumnException.java b/tajo-core/src/main/java/org/apache/tajo/engine/exception/NoSuchColumnException.java
deleted file mode 100644
index 70191a9..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/exception/NoSuchColumnException.java
+++ /dev/null
@@ -1,25 +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.tajo.engine.exception;
-
-public class NoSuchColumnException extends VerifyException {
-  public NoSuchColumnException(String columnName) {
-    super("ERROR: no such a column '" + columnName + "'");
-  }
-}