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 2013/07/02 16:16:13 UTC

[19/51] [partial] TAJO-22: The package prefix should be org.apache.tajo. (DaeMyung Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java
deleted file mode 100644
index be3b607..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java
+++ /dev/null
@@ -1,353 +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 tajo.engine.eval;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import tajo.catalog.Column;
-import tajo.catalog.Schema;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.engine.eval.EvalNode.Type;
-import tajo.engine.parser.QueryBlock.Target;
-import tajo.engine.utils.SchemaUtil;
-import tajo.exception.InternalException;
-
-import java.util.*;
-
-public class EvalTreeUtil {
-  public static void changeColumnRef(EvalNode node, Column oldName, 
-      Column newName) {
-    node.postOrder(new ChangeColumnRefVisitor(oldName.getQualifiedName(), 
-        newName.getQualifiedName()));
-  }
-  
-  public static void changeColumnRef(EvalNode node, String oldName, 
-      String newName) {
-    node.postOrder(new ChangeColumnRefVisitor(oldName, newName));
-  }
-  
-  public static Set<Column> findDistinctRefColumns(EvalNode node) {
-    DistinctColumnRefFinder finder = new DistinctColumnRefFinder();
-    node.postOrder(finder);
-    return finder.getColumnRefs();
-  }
-  
-  public static List<Column> findAllColumnRefs(EvalNode node) {
-    AllColumnRefFinder finder = new AllColumnRefFinder();
-    node.postOrder(finder);
-    return finder.getColumnRefs();
-  }
-  
-  /**
-   * Convert a list of conjunctive normal forms into a singleton expression.
-   *  
-   * @param evalNode
-   * @return
-   */
-  public static EvalNode transformCNF2Singleton(EvalNode...evalNode) {    
-    if (evalNode.length == 1) {
-      return evalNode[0];
-    }
-    
-    return transformCNF2Singleton_(evalNode, 0);
-  }
-  
-  private static EvalNode transformCNF2Singleton_(EvalNode [] evalNode, int idx) {
-    if (idx == evalNode.length - 2) {
-      return new BinaryEval(Type.AND, evalNode[idx], evalNode[idx + 1]);
-    } else {
-      return new BinaryEval(Type.AND, evalNode[idx], 
-          transformCNF2Singleton_(evalNode, idx + 1));
-    }
-  }
-  
-  /**
-   * Get a list of exprs similar to CNF
-   * 
-   * @param node
-   * @return
-   */
-  public static EvalNode [] getConjNormalForm(EvalNode node) {
-    List<EvalNode> list = new ArrayList<EvalNode>();    
-    getConjNormalForm(node, list);    
-    return list.toArray(new EvalNode[list.size()]);
-  }
-  
-  private static void getConjNormalForm(EvalNode node, List<EvalNode> found) {
-    if (node.getType() == Type.AND) {
-      getConjNormalForm(node.getLeftExpr(), found);
-      getConjNormalForm(node.getRightExpr(), found);
-    } else {
-      found.add(node);
-    }
-  }
-  
-  /**
-   * Compute a schema from a list of exprs.
-   * 
-   * @param inputSchema
-   * @param evalNodes
-   * @return
-   * @throws InternalException
-   */
-  public static Schema getSchemaByExprs(Schema inputSchema, EvalNode [] evalNodes) 
-      throws InternalException {
-    Schema schema = new Schema();
-    for (EvalNode expr : evalNodes) {
-      schema.addColumn(
-          expr.getName(),
-          getDomainByExpr(inputSchema, expr)[0]);
-    }
-    
-    return schema;
-  }
-  
-  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())[0]);
-    }
-    
-    return schema;
-  }
-  
-  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 SchemaUtil.newNoNameSchema(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<Type, 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
-   * 
-   * @param expr
-   * @param target
-   * @return
-   */
-  public static boolean containColumnRef(EvalNode expr, Column target) {
-    Set<EvalNode> exprSet = Sets.newHashSet();
-    _containColumnRef(expr, target, exprSet);
-    
-    return exprSet.size() > 0;
-  }
-  
-  private static void _containColumnRef(EvalNode expr, Column target, 
-      Set<EvalNode> exprSet) {
-    switch (expr.getType()) {
-    case FIELD:
-      FieldEval field = (FieldEval) expr;
-      if (field.getColumnName().equals(target.getColumnName())) {
-        exprSet.add(field);
-      }
-      break;
-    case CONST:
-      return;
-    default: 
-      _containColumnRef(expr.getLeftExpr(), target, exprSet);
-      _containColumnRef(expr.getRightExpr(), target, exprSet);
-    }    
-  }
-
-  public static boolean isComparisonOperator(EvalNode expr) {
-    return expr.getType() == Type.EQUAL ||
-        expr.getType() == Type.LEQ ||
-        expr.getType() == Type.LTH ||
-        expr.getType() == Type.GEQ ||
-        expr.getType() == Type.GTH;
-  }
-
-  public static boolean isJoinQual(EvalNode expr) {
-    return isComparisonOperator(expr) &&
-        expr.getLeftExpr().getType() == Type.FIELD &&
-        expr.getRightExpr().getType() == Type.FIELD;
-  }
-
-  public static boolean isLogicalOperator(EvalNode expr) {
-    return expr.getType() == Type.AND || expr.getType() == Type.OR;
-  }
-  
-  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 == Type.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() == Type.FIELD) {
-        field = (FieldEval) node;
-        colList.add(field.getColumnRef());
-      } 
-    }
-    
-    public List<Column> getColumnRefs() {
-      return this.colList;
-    }
-  }
-  
-  public static class DistinctColumnRefFinder implements EvalNodeVisitor {
-    private Set<Column> colList = new HashSet<Column>(); 
-    private FieldEval field = null;
-    
-    @Override
-    public void visit(EvalNode node) {
-      if (node.getType() == Type.FIELD) {
-        field = (FieldEval) node;
-        colList.add(field.getColumnRef());
-      }
-    }
-    
-    public Set<Column> getColumnRefs() {
-      return this.colList;
-    }
-  }
-  
-  public static class VariableCounter implements EvalNodeVisitor {
-    private final Map<EvalNode.Type, Integer> counter;
-    
-    public VariableCounter() {
-      counter = Maps.newHashMap();
-      counter.put(Type.FUNCTION, 0);
-      counter.put(Type.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<EvalNode.Type, Integer> getCounter() {
-      return counter;
-    }
-  }
-
-  public static List<AggFuncCallEval> findDistinctAggFunction(EvalNode expr) {
-    AllAggFunctionFinder finder = new AllAggFunctionFinder();
-    expr.postOrder(finder);
-    return Lists.newArrayList(finder.getAggregationFunction());
-  }
-
-  public static class AllAggFunctionFinder implements EvalNodeVisitor {
-    private Set<AggFuncCallEval> aggFucntions = Sets.newHashSet();
-    private AggFuncCallEval field = null;
-
-    @Override
-    public void visit(EvalNode node) {
-      if (node.getType() == Type.AGG_FUNCTION) {
-        field = (AggFuncCallEval) node;
-        aggFucntions.add(field);
-      }
-    }
-
-    public Set<AggFuncCallEval> getAggregationFunction() {
-      return this.aggFucntions;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java
deleted file mode 100644
index c0d856a..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java
+++ /dev/null
@@ -1,147 +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 tajo.engine.eval;
-
-import com.google.gson.Gson;
-import com.google.gson.annotations.Expose;
-import tajo.catalog.Column;
-import tajo.catalog.Schema;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.datum.Datum;
-import tajo.engine.json.GsonCreator;
-import tajo.engine.utils.SchemaUtil;
-import 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(Type.FIELD);
-		this.column = new Column(columnName, domain);
-	}
-	
-	public FieldEval(Column column) {
-	  super(Type.FIELD);
-	  this.column = column;
-	}
-
-	@Override
-	public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
-	  if (fieldId == -1) {
-	    if(schema.contains(column.getQualifiedName())) {
-	     fieldId = schema.getColumnId(column.getQualifiedName());
-	    } else {
-	      if(schema.getColumnNum() != 0) {
-	        String schemaColQualName = schema.getColumn(0).getTableName() + 
-	            "." +  column.getColumnName();
-	        fieldId = schema.getColumnId(schemaColQualName);
-	      } else {
-	        fieldId = schema.getColumnId(column.getQualifiedName());
-	      }
-	    }
-	  }
-    FieldEvalContext fieldCtx = (FieldEvalContext) ctx;
-	  fieldCtx.datum = tuple.get(fieldId);
-	}
-
-  @Override
-  public Datum terminate(EvalContext ctx) {
-    return ((FieldEvalContext)ctx).datum;
-  }
-
-  @Override
-  public EvalContext newContext() {
-    return new FieldEvalContext();
-  }
-
-  private static class FieldEvalContext implements EvalContext {
-    private Datum datum;
-
-    public FieldEvalContext() {
-    }
-  }
-
-  @Override
-	public DataType[] getValueType() {
-		return SchemaUtil.newNoNameSchema(column.getDataType());
-	}
-	
-  public Column getColumnRef() {
-    return column;
-  }
-	
-	public String getTableId() {	  
-	  return column.getTableName();
-	}
-	
-	public String getColumnName() {
-	  return column.getColumnName();
-	}
-	
-	public void replaceColumnRef(String columnName) {
-	  this.column.setName(columnName);
-	}
-
-	@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 = (Column) this.column.clone();
-    eval.fieldId = fieldId;
-    
-    return eval;
-  }
-  
-  public String toJSON() {
-    Gson gson = GsonCreator.getInstance();
-    return gson.toJson(this, EvalNode.class);
-  }
-
-  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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncCallEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncCallEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncCallEval.java
deleted file mode 100644
index 2dce9e6..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncCallEval.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 tajo.engine.eval;
-
-import com.google.common.base.Objects;
-import com.google.gson.Gson;
-import com.google.gson.annotations.Expose;
-import tajo.catalog.FunctionDesc;
-import tajo.catalog.Schema;
-import tajo.catalog.function.GeneralFunction;
-import tajo.datum.Datum;
-import tajo.engine.json.GsonCreator;
-import tajo.storage.Tuple;
-import tajo.storage.VTuple;
-import tajo.util.TUtil;
-
-public class FuncCallEval extends FuncEval {
-	@Expose protected GeneralFunction instance;
-  private Tuple tuple;
-  private Tuple params = null;
-  private Schema schema;
-
-	public FuncCallEval(FunctionDesc desc, GeneralFunction instance, EvalNode [] givenArgs) {
-		super(Type.FUNCTION, desc, givenArgs);
-		this.instance = instance;
-  }
-
-  /* (non-Javadoc)
-    * @see nta.query.executor.eval.Expr#evalVal(Tuple)
-    */
-	@Override
-	public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
-    this.schema = schema;
-    this.tuple = tuple;
-	}
-
-  @Override
-  public Datum terminate(EvalContext ctx) {
-    FuncCallCtx localCtx = (FuncCallCtx) ctx;
-    if (this.params == null) {
-      params = new VTuple(argEvals.length);
-    }
-
-    if(argEvals != null) {
-      params.clear();
-      for(int i=0;i < argEvals.length; i++) {
-        argEvals[i].eval(localCtx.argCtxs[i], schema, tuple);
-        params.put(i, argEvals[i].terminate(localCtx.argCtxs[i]));
-      }
-    }
-    return instance.eval(params);
-  }
-
-  @Override
-	public String toJSON() {
-	  Gson gson = GsonCreator.getInstance();
-    return gson.toJson(this, EvalNode.class);
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-	  if (obj instanceof FuncCallEval) {
-      FuncCallEval other = (FuncCallEval) 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 {
-    FuncCallEval eval = (FuncCallEval) super.clone();
-    eval.instance = (GeneralFunction) instance.clone();
-    return eval;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java
deleted file mode 100644
index 4debee8..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.engine.eval;
-
-import com.google.common.base.Objects;
-import com.google.gson.Gson;
-import com.google.gson.annotations.Expose;
-import tajo.catalog.FunctionDesc;
-import tajo.catalog.Schema;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.datum.Datum;
-import tajo.engine.json.GsonCreator;
-import tajo.storage.Tuple;
-import tajo.util.TUtil;
-
-public abstract class FuncEval extends EvalNode implements Cloneable {
-	@Expose protected FunctionDesc funcDesc;
-	@Expose protected EvalNode [] argEvals;
-
-	public FuncEval(Type type, FunctionDesc funcDesc, EvalNode[] argEvals) {
-		super(type);
-		this.funcDesc = funcDesc;
-		this.argEvals = argEvals;
-	}
-
-  @Override
-  public EvalContext newContext() {
-    FuncCallCtx newCtx = new FuncCallCtx(argEvals);
-    return newCtx;
-  }
-	
-	public EvalNode [] getArgs() {
-	  return this.argEvals;
-	}
-
-  public void setArgs(EvalNode [] args) {
-    this.argEvals = args;
-  }
-	
-	public DataType [] getValueType() {
-		return this.funcDesc.getReturnType();
-	}
-
-	@Override
-	public abstract void eval(EvalContext ctx, Schema schema, Tuple tuple);
-
-  public abstract Datum terminate(EvalContext ctx);
-
-	@Override
-	public String getName() {
-		return funcDesc.getSignature();
-	}
-
-  @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.getSignature()+"("+sb+")";
-	}
-
-  @Override
-	public String toJSON() {
-	  Gson gson = GsonCreator.getInstance();
-    return gson.toJson(this, EvalNode.class);
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-	  if (obj instanceof FuncEval) {
-      FuncEval other = (FuncEval) 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 {
-    FuncEval eval = (FuncEval) super.clone();
-    eval.funcDesc = (FunctionDesc) funcDesc.clone();
-    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) {
-    for (EvalNode eval : argEvals) {
-      eval.postOrder(visitor);
-    }
-    visitor.visit(this);
-  }
-	
-	@Override
-	public void postOrder(EvalNodeVisitor visitor) {
-	  for (EvalNode eval : argEvals) {
-	    eval.postOrder(visitor);
-	  }
-	  visitor.visit(this);
-	}
-
-  protected class FuncCallCtx implements EvalContext {
-    EvalContext [] argCtxs;
-    FuncCallCtx(EvalNode [] argEvals) {
-      argCtxs = new EvalContext[argEvals.length];
-      for (int i = 0; i < argEvals.length; i++) {
-        argCtxs[i] = argEvals[i].newContext();
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/InvalidCastException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/InvalidCastException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/InvalidCastException.java
deleted file mode 100644
index 1ad444e..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/InvalidCastException.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 tajo.engine.eval;
-
-public class InvalidCastException extends RuntimeException {
-	private static final long serialVersionUID = -5090530469575858320L;
-
-	public InvalidCastException() {
-	}
-
-	/**
-	 * @param message
-	 */
-	public InvalidCastException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/InvalidEvalException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/InvalidEvalException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/InvalidEvalException.java
deleted file mode 100644
index 5ff28cc..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/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 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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java
deleted file mode 100644
index 20c2161..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.Schema;
-import tajo.common.TajoDataTypes;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.datum.BooleanDatum;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.NullDatum;
-import tajo.storage.Tuple;
-
-public class IsNullEval extends BinaryEval {
-  private final static ConstEval NULL_EVAL = new ConstEval(DatumFactory.createNullDatum());
-  private static final DataType [] RES_TYPE =
-      CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
-
-  // persistent variables
-  @Expose private boolean isNot;
-  @Expose private Column columnRef;
-  @Expose private Integer fieldId = null;
-
-  public IsNullEval(boolean not, FieldEval field) {
-    super(Type.IS, field, NULL_EVAL);
-    this.isNot = not;
-    this.columnRef = field.getColumnRef();
-  }
-
-  @Override
-  public EvalContext newContext() {
-    return new IsNullEvalCtx();
-  }
-
-  @Override
-  public DataType[] getValueType() {
-    return RES_TYPE;
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
-    IsNullEvalCtx isNullCtx = (IsNullEvalCtx) ctx;
-    if (fieldId == null) {
-      fieldId = schema.getColumnId(columnRef.getQualifiedName());
-    }
-    if (isNot) {
-      isNullCtx.result.setValue(!(tuple.get(fieldId) instanceof NullDatum));
-    } else {
-      isNullCtx.result.setValue(tuple.get(fieldId) instanceof NullDatum);
-    }
-  }
-
-  @Override
-  public Datum terminate(EvalContext ctx) {
-    return ((IsNullEvalCtx)ctx).result;
-  }
-
-  public boolean isNot() {
-    return isNot;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof IsNullEval) {
-      IsNullEval other = (IsNullEval) obj;
-      return super.equals(other) &&
-          this.columnRef.equals(other.columnRef) &&
-          this.fieldId == other.fieldId;
-    } else {
-      return false;
-    }
-  }
-
-  public Object clone() throws CloneNotSupportedException {
-    IsNullEval isNullEval = (IsNullEval) super.clone();
-    isNullEval.columnRef = (Column) columnRef.clone();
-    isNullEval.fieldId = fieldId;
-
-    return isNullEval;
-  }
-
-  private class IsNullEvalCtx implements EvalContext {
-    BooleanDatum result;
-
-    IsNullEvalCtx() {
-      this.result = DatumFactory.createBool(false);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java
deleted file mode 100644
index 57c571a..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java
+++ /dev/null
@@ -1,96 +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 tajo.engine.eval;
-
-import com.google.gson.annotations.Expose;
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.Schema;
-import tajo.common.TajoDataTypes;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.datum.BooleanDatum;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.TextDatum;
-import tajo.storage.Tuple;
-
-import java.util.regex.Pattern;
-
-public class LikeEval extends BinaryEval {
-  @Expose private boolean not;
-  @Expose private Column column;
-  @Expose private String pattern;
-  private static final DataType [] RES_TYPE =
-      CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
-
-  // temporal variables
-  private Integer fieldId = null;
-  private Pattern compiled;
-  private BooleanDatum result;
-
-  
-  public LikeEval(boolean not, FieldEval field, ConstEval pattern) {
-    super(Type.LIKE, field, pattern);
-    this.not = not;
-    this.column = field.getColumnRef();
-    this.pattern = pattern.getValue().asChars();
-  }
-  
-  public void compile(String pattern) {
-    String regex = pattern.replace("?", ".");
-    regex = regex.replace("%", ".*");
-    
-    this.compiled = Pattern.compile(regex,
-        Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
-    result = DatumFactory.createBool(false);
-  }
-
-  @Override
-  public DataType [] getValueType() {
-    return RES_TYPE;
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
-    if (fieldId == null) {
-      fieldId = schema.getColumnId(column.getQualifiedName());
-      compile(this.pattern);
-    }    
-    TextDatum str = tuple.getString(fieldId);
-    if (not) {
-      result.setValue(!compiled.matcher(str.asChars()).matches());      
-    } else {
-      result.setValue(compiled.matcher(str.asChars()).matches());
-    }
-  }
-
-  public Datum terminate(EvalContext ctx) {
-    return result;
-  }
-  
-  @Override
-  public String toString() {
-    return this.column + " like '" + pattern +"'";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java
deleted file mode 100644
index fbef406..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.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 tajo.engine.eval;
-
-import com.google.common.base.Preconditions;
-import com.google.gson.annotations.Expose;
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Schema;
-import tajo.common.TajoDataTypes;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.storage.Tuple;
-
-public class NotEval extends EvalNode implements Cloneable {
-  @Expose private EvalNode subExpr;
-  private static final DataType[] RES_TYPE =
-      CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
-
-  public NotEval(EvalNode subExpr) {
-    super(Type.NOT);
-    Preconditions.checkArgument(
-        subExpr instanceof BinaryEval || subExpr instanceof NotEval);
-    this.subExpr = subExpr;
-  }
-
-  @Override
-  public EvalContext newContext() {
-    NotEvalCtx newCtx = new NotEvalCtx();
-    newCtx.subExprCtx = subExpr.newContext();
-    return newCtx;
-  }
-
-  @Override
-  public DataType [] getValueType() {
-    return RES_TYPE;
-  }
-
-  @Override
-  public String getName() {
-    return "?";
-  }
-
-  @Override
-  public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
-    subExpr.eval(((NotEvalCtx)ctx).subExprCtx, schema, tuple);
-  }
-
-  @Override
-  public Datum terminate(EvalContext ctx) {
-    return DatumFactory.createBool(!subExpr.terminate(((NotEvalCtx)ctx).subExprCtx).asBool());
-  }
-
-  @Override
-  public String toString() {
-    return "NOT " + subExpr.toString();
-  }
-
-  @Override
-  public void preOrder(EvalNodeVisitor visitor) {
-    visitor.visit(this);
-    if (subExpr instanceof NotEval) {
-      ((NotEval)subExpr).subExpr.preOrder(visitor);
-    } else {
-      subExpr.leftExpr.preOrder(visitor);
-      subExpr.rightExpr.preOrder(visitor);
-    }
-  }
-
-  @Override
-  public void postOrder(EvalNodeVisitor visitor) {    
-    if (subExpr instanceof NotEval) {
-      ((NotEval)subExpr).subExpr.preOrder(visitor);
-    } else {
-      subExpr.leftExpr.preOrder(visitor);
-      subExpr.rightExpr.preOrder(visitor);
-    }
-    visitor.visit(this);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof NotEval) {
-      NotEval other = (NotEval) obj;
-      return this.subExpr.equals(other.subExpr);
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public Object clone() throws CloneNotSupportedException {
-    NotEval eval = (NotEval) super.clone();
-    eval.subExpr = (EvalNode) this.subExpr.clone();
-    return eval;
-  }
-
-  private class NotEvalCtx implements EvalContext {
-    EvalContext subExprCtx;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java
deleted file mode 100644
index 10a3707..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java
+++ /dev/null
@@ -1,82 +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 tajo.engine.eval;
-
-import tajo.catalog.Schema;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.datum.Datum;
-import tajo.storage.Tuple;
-
-public class PartialBinaryExpr extends EvalNode {
-  
-  public PartialBinaryExpr(Type type) {
-    super(type);
-  }
-
-  public PartialBinaryExpr(Type type, EvalNode left, EvalNode right) {
-    super(type, left, right);
-  }
-
-  @Override
-  public EvalContext newContext() {
-    return null;
-  }
-
-  @Override
-  public DataType[] getValueType() {
-    return null;
-  }
-
-  @Override
-  public String getName() {
-    return "nonamed";
-  }
-
-  @Override
-  public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
-    throw new InvalidEvalException(
-        "ERROR: the partial binary expression cannot be evluated: "
-            + this.toString() );
-  }
-
-  @Override
-  public Datum terminate(EvalContext ctx) {
-    throw new InvalidEvalException(
-        "ERROR: the partial binary expression cannot be terminated: "
-            + 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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/EmptyClusterException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/EmptyClusterException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/EmptyClusterException.java
deleted file mode 100644
index b6e44ff..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/EmptyClusterException.java
+++ /dev/null
@@ -1,34 +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 tajo.engine.exception;
-
-public class EmptyClusterException extends Exception {
-
-  public EmptyClusterException() {
-
-  }
-
-  public EmptyClusterException(String msg) {
-    super(msg);
-  }
-
-  public EmptyClusterException(Exception e) {
-    super(e);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/IllegalQueryStatusException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/IllegalQueryStatusException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/IllegalQueryStatusException.java
deleted file mode 100644
index e462ae0..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/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 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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NTAQueryException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NTAQueryException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NTAQueryException.java
deleted file mode 100644
index bad2c57..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NTAQueryException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package tajo.engine.exception;
-
-import java.io.IOException;
-
-public class NTAQueryException extends IOException {
-	private static final long serialVersionUID = -5012296598261064705L;
-
-	public NTAQueryException() {
-	}
-	
-	public NTAQueryException(Exception e) {
-		super(e);
-	}
-
-	/**
-	 * @param query
-	 */
-	public NTAQueryException(String query) {
-		super(query);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NoSuchQueryIdException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NoSuchQueryIdException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NoSuchQueryIdException.java
deleted file mode 100644
index d29992a..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/NoSuchQueryIdException.java
+++ /dev/null
@@ -1,43 +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 tajo.engine.exception;
-
-public class NoSuchQueryIdException extends Exception {
-
-  private static final long serialVersionUID = -4425982532461186746L;
-
-  public NoSuchQueryIdException() {
-    
-  }
-  
-  public NoSuchQueryIdException(String message) {
-    super(message);
-  }
-  
-  public NoSuchQueryIdException(Exception e) {
-    super(e);
-  }
-  
-  public NoSuchQueryIdException(String message, Exception e) {
-    super(message, e);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnfinishedTaskException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnfinishedTaskException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnfinishedTaskException.java
deleted file mode 100644
index f50b434..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnfinishedTaskException.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 tajo.engine.exception;
-
-public class UnfinishedTaskException extends Exception {
-
-  private static final long serialVersionUID = -3229141373378209229L;
-  
-  public UnfinishedTaskException() {
-    
-  }
-
-  public UnfinishedTaskException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnknownWorkerException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnknownWorkerException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnknownWorkerException.java
deleted file mode 100644
index becb589..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/exception/UnknownWorkerException.java
+++ /dev/null
@@ -1,44 +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 tajo.engine.exception;
-
-public class UnknownWorkerException extends Exception {
-
-  /**
-   * 
-   */
-  private static final long serialVersionUID = -3677733092100608744L;
-  private String unknownName;
-
-  public UnknownWorkerException(String unknownName) {
-    this.unknownName = unknownName;
-  }
-
-  public UnknownWorkerException(String unknownName, Exception e) {
-    super(e);
-    this.unknownName = unknownName;
-  }
-
-  public String getUnknownName() {
-    return this.unknownName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java
deleted file mode 100644
index 50593b3..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.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 tajo.engine.function;
-
-import tajo.catalog.Column;
-import tajo.catalog.function.GeneralFunction;
-import tajo.datum.Datum;
-import tajo.datum.TextDatum;
-import tajo.storage.Tuple;
-import tajo.util.GeoUtil;
-
-import static tajo.common.TajoDataTypes.Type.TEXT;
-
-public class Country extends GeneralFunction<TextDatum> {
-
-  public Country() {
-    super(new Column[] {new Column("addr", TEXT)});
-  }
-
-  @Override
-  public Datum eval(Tuple params) {
-    return new TextDatum(GeoUtil.getCountryCode(params.get(0).asChars()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java
deleted file mode 100644
index c2e86a3..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java
+++ /dev/null
@@ -1,44 +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 tajo.engine.function;
-
-import tajo.catalog.Column;
-import tajo.catalog.function.GeneralFunction;
-import tajo.common.TajoDataTypes;
-import tajo.datum.BooleanDatum;
-import tajo.datum.Datum;
-import tajo.storage.Tuple;
-import tajo.util.GeoUtil;
-
-public class InCountry extends GeneralFunction<BooleanDatum> {
-
-  public InCountry() {
-    super(new Column[] {new Column("addr", TajoDataTypes.Type.TEXT),
-        new Column("code", TajoDataTypes.Type.TEXT)});
-  }
-
-  @Override
-  public Datum eval(Tuple params) {
-    String addr = params.get(0).asChars();
-    String otherCode = params.get(1).asChars();
-    String thisCode = GeoUtil.getCountryCode(addr);
-
-    return new BooleanDatum(thisCode.equals(otherCode));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java
deleted file mode 100644
index af18676..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.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 tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.ArrayDatum;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.storage.Tuple;
-
-public class AvgDouble extends AggFunction {
-  public AvgDouble() {
-    super(new Column[] {
-        new Column("val", Type.FLOAT8)
-    });
-  }
-
-  public AvgContext newContext() {
-    return new AvgContext();
-  }
-
-  public void init() {
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asFloat8();
-    avgCtx.count++;
-  }
-
-  @Override
-  public void merge(FunctionContext ctx, Tuple part) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asFloat8();
-    avgCtx.count += array.get(1).asInt8();
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createFloat8(avgCtx.sum));
-    part.put(1, DatumFactory.createInt8(avgCtx.count));
-
-    return part;
-  }
-
-  @Override
-  public DataType[] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8, Type.INT8);
-  }
-
-  @Override
-  public Datum terminate(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createFloat8(avgCtx.sum / avgCtx.count);
-  }
-
-  private class AvgContext implements FunctionContext {
-    double sum;
-    long count;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java
deleted file mode 100644
index d13b0ae..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java
+++ /dev/null
@@ -1,85 +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 tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.ArrayDatum;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.Float4Datum;
-import tajo.storage.Tuple;
-
-public class AvgFloat extends AggFunction<Float4Datum> {
-
-  public AvgFloat() {
-    super(new Column[] {
-        new Column("val", Type.FLOAT8)
-    });
-  }
-
-  public AvgContext newContext() {
-    return new AvgContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asFloat4();
-    avgCtx.count++;
-  }
-
-  @Override
-  public void merge(FunctionContext ctx, Tuple part) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asFloat8();
-    avgCtx.count += array.get(1).asInt8();
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createFloat8(avgCtx.sum));
-    part.put(1, DatumFactory.createInt8(avgCtx.count));
-
-    return part;
-  }
-
-  @Override
-  public DataType[] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8, Type.INT8);
-  }
-
-  @Override
-  public Float4Datum terminate(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createFloat4((float) (avgCtx.sum / avgCtx.count));
-  }
-
-  private class AvgContext implements FunctionContext {
-    double sum;
-    long count;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java
deleted file mode 100644
index 5adb2fb..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java
+++ /dev/null
@@ -1,85 +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 tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.ArrayDatum;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.Float4Datum;
-import tajo.storage.Tuple;
-
-public class AvgInt extends AggFunction<Float4Datum> {
-
-  public AvgInt() {
-    super(new Column[] {
-        new Column("val", Type.FLOAT8)
-    });
-  }
-
-  public AvgContext newContext() {
-    return new AvgContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asInt4();
-    avgCtx.count++;
-  }
-
-  @Override
-  public void merge(FunctionContext ctx, Tuple part) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asInt8();
-    avgCtx.count += array.get(1).asInt8();
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createInt8(avgCtx.sum));
-    part.put(1, DatumFactory.createInt8(avgCtx.count));
-
-    return part;
-  }
-
-  @Override
-  public DataType [] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.INT8, Type.INT8);
-  }
-
-  @Override
-  public Float4Datum terminate(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createFloat4((float) avgCtx.sum / avgCtx.count);
-  }
-
-  private class AvgContext implements FunctionContext {
-    long sum;
-    long count;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java
deleted file mode 100644
index 3e76133..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java
+++ /dev/null
@@ -1,85 +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 tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.ArrayDatum;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.Float8Datum;
-import tajo.storage.Tuple;
-
-public class AvgLong extends AggFunction<Float8Datum> {
-
-  public AvgLong() {
-    super(new Column[] {
-        new Column("val", Type.FLOAT8)
-    });
-  }
-
-  public AvgContext newContext() {
-    return new AvgContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asInt8();
-    avgCtx.count++;
-  }
-
-  @Override
-  public void merge(FunctionContext ctx, Tuple part) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asInt8();
-    avgCtx.count += array.get(1).asInt8();
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createInt8(avgCtx.sum));
-    part.put(1, DatumFactory.createInt8(avgCtx.count));
-
-    return part;
-  }
-
-  @Override
-  public DataType[] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.INT8, Type.INT8);
-  }
-
-  @Override
-  public Float8Datum terminate(FunctionContext ctx) {
-    AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createFloat8((double) avgCtx.sum / avgCtx.count);
-  }
-
-  private class AvgContext implements FunctionContext {
-    long sum;
-    long count;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java
deleted file mode 100644
index 14a76da..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java
+++ /dev/null
@@ -1,75 +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 tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.Int8Datum;
-import tajo.storage.Tuple;
-
-public class CountRows extends AggFunction<Datum> {
-
-  public CountRows() {
-    super(NoArgs);
-  }
-
-  protected CountRows(Column[] columns) {
-    super(columns);
-  }
-
-  @Override
-  public FunctionContext newContext() {
-    return new CountRowContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    ((CountRowContext) ctx).count++;
-  }
-
-  @Override
-  public void merge(FunctionContext ctx, Tuple part) {
-    ((CountRowContext) ctx).count += part.get(0).asInt8();
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createInt8(((CountRowContext) ctx).count);
-  }
-
-  @Override
-  public DataType[] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.INT8);
-  }
-
-  @Override
-  public Int8Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createInt8(((CountRowContext) ctx).count);
-  }
-
-  protected class CountRowContext implements FunctionContext {
-    long count;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java
deleted file mode 100644
index 01a7cef..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.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 tajo.engine.function.builtin;
-
-import tajo.catalog.Column;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.NullDatum;
-import tajo.storage.Tuple;
-
-public final class CountValue extends CountRows {
-
-  public CountValue() {
-    super(new Column[] {
-        new Column("col", Type.ANY)
-    });
-  }
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    if (params.get(0) instanceof NullDatum) {
-      ((CountRowContext) ctx).count++;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java
deleted file mode 100644
index 80b4124..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.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 tajo.engine.function.builtin;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import tajo.catalog.Column;
-import tajo.catalog.function.GeneralFunction;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.Int8Datum;
-import tajo.storage.Tuple;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-
-import static tajo.common.TajoDataTypes.Type.TEXT;
-
-public class Date extends GeneralFunction<Int8Datum> {
-  private final Log LOG = LogFactory.getLog(Date.class);
-  private final static String dateFormat = "dd/MM/yyyy HH:mm:ss";
-
-  public Date() {
-    super(new Column[] {new Column("val", TEXT)});
-  }
-
-  @Override
-  public Datum eval(Tuple params) {
-    try {
-      return DatumFactory.createInt8(new SimpleDateFormat(dateFormat)
-          .parse(params.get(0).asChars()).getTime());
-    } catch (ParseException e) {
-      LOG.error(e);
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java
deleted file mode 100644
index d895b08..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java
+++ /dev/null
@@ -1,69 +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 tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.Float8Datum;
-import tajo.storage.Tuple;
-
-public class MaxDouble extends AggFunction<Float8Datum> {
-
-  public MaxDouble() {
-    super(new Column[] {
-        new Column("val", Type.FLOAT8)
-    });
-  }
-
-  @Override
-  public FunctionContext newContext() {
-    return new MaxContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asFloat8());
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createFloat8(((MaxContext) ctx).max);
-  }
-
-  @Override
-  public DataType[] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8);
-  }
-
-  @Override
-  public Float8Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createFloat8(((MaxContext) ctx).max);
-  }
-
-  private class MaxContext implements FunctionContext {
-    double max;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java
deleted file mode 100644
index 0a65904..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.storage.Tuple;
-
-public class MaxFloat extends AggFunction<Datum> {
-  public MaxFloat() {
-    super(new Column[] {
-        new Column("val", Type.FLOAT8)
-    });
-  }
-
-  @Override
-  public FunctionContext newContext() {
-    return new MaxContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asFloat4());
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createFloat4(((MaxContext) ctx).max);
-  }
-
-  @Override
-  public DataType[] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4);
-  }
-
-  @Override
-  public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createFloat4(((MaxContext) ctx).max);
-  }
-
-  private class MaxContext implements FunctionContext {
-    float max;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java
deleted file mode 100644
index ad96b4f..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.storage.Tuple;
-
-public class MaxInt extends AggFunction<Datum> {
-
-  public MaxInt() {
-    super(new Column[] {
-        new Column("val", Type.INT8)
-    });
-  }
-
-  @Override
-  public FunctionContext newContext() {
-    return new MaxContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asInt4());
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createInt4(((MaxContext) ctx).max);
-  }
-
-  @Override
-  public DataType [] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.INT4);
-  }
-
-  @Override
-  public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createInt4(((MaxContext) ctx).max);
-  }
-
-  private class MaxContext implements FunctionContext {
-    int max;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java
deleted file mode 100644
index eddf1c5..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.engine.function.builtin;
-
-import tajo.catalog.CatalogUtil;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.common.TajoDataTypes.Type;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.Int8Datum;
-import tajo.storage.Tuple;
-
-public class MaxLong extends AggFunction<Int8Datum> {
-  public MaxLong() {
-    super(new Column[] {
-        new Column("val", Type.INT8)
-    });
-  }
-
-  @Override
-  public FunctionContext newContext() {
-    return new MaxContext();
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asInt8());
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createInt8(((MaxContext) ctx).max);
-  }
-
-  @Override
-  public DataType [] getPartialResultType() {
-    return CatalogUtil.newDataTypesWithoutLen(Type.INT8);
-  }
-
-  @Override
-  public Int8Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createInt8(((MaxContext) ctx).max);
-  }
-
-  private class MaxContext implements FunctionContext {
-    long max;
-  }
-}