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/04/18 11:19:23 UTC

[01/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Repository: tajo
Updated Branches:
  refs/heads/master 795077d89 -> 6594ac1c0


http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
new file mode 100644
index 0000000..8692070
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -0,0 +1,325 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+
+@Category(IntegrationTest.class)
+public class TestJoinQuery extends QueryTestCaseBase {
+
+  public TestJoinQuery() {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test
+  public final void testCrossJoin() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin2() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin3() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin4() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin5() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin6() throws Exception {
+    ResultSet res = executeQuery();
+    System.out.println(resultSetToString(res));
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testTPCHQ2Join() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testJoinWithMultipleJoinQual1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testJoinWithMultipleJoinQual2() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testJoinWithMultipleJoinQual3() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testJoinWithMultipleJoinQual4() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithConstantExpr1() throws Exception {
+    // outer join with constant projections
+    //
+    // select c_custkey, orders.o_orderkey, 'val' as val from customer
+    // left outer join orders on c_custkey = o_orderkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithConstantExpr2() throws Exception {
+    // outer join with constant projections
+    //
+    // select c_custkey, o.o_orderkey, 'val' as val from customer left outer join
+    // (select * from orders) o on c_custkey = o.o_orderkey
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithConstantExpr3() throws Exception {
+    // outer join with constant projections
+    //
+    // select a.c_custkey, 123::INT8 as const_val, b.min_name from customer a
+    // left outer join ( select c_custkey, min(c_name) as min_name from customer group by c_custkey) b
+    // on a.c_custkey = b.c_custkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testRightOuterJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testFullOuterJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testJoinCoReferredEvals1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testJoinCoReferredEvalsWithSameExprs1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testJoinCoReferredEvalsWithSameExprs2() throws Exception {
+    // including grouping operator
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinAndCaseWhen() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testOuterJoinAndCaseWhen1() throws Exception {
+    executeDDL("oj_table1_ddl.sql", "table1");
+    executeDDL("oj_table2_ddl.sql", "table2");
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk1() throws Exception {
+    // select region.*, customer.* from region, customer;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+   public void testCrossJoinWithAsterisk2() throws Exception {
+    // select region.*, customer.* from customer, region;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk3() throws Exception {
+    // select * from customer, region
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk4() throws Exception {
+    // select length(r_regionkey), *, c_custkey*10 from customer, region
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testInnerJoinWithEmptyTable() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable1() throws Exception {
+    /*
+    select
+      c_custkey,
+      empty_orders.o_orderkey,
+      empty_orders.o_orderstatus,
+      empty_orders.o_orderdate
+    from
+      customer left outer join empty_orders on c_custkey = o_orderkey
+    order by
+      c_custkey, o_orderkey;
+     */
+
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable2() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable3() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable4() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testRightOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testFullOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testCrossJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testJoinOnMultipleDatabases() throws Exception {
+    executeString("CREATE DATABASE JOINS");
+    assertDatabaseExists("joins");
+    executeString("CREATE TABLE JOINS.part_ as SELECT * FROM part");
+    assertTableExists("joins.part_");
+    executeString("CREATE TABLE JOINS.supplier_ as SELECT * FROM supplier");
+    assertTableExists("joins.supplier_");
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+}


[23/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
new file mode 100644
index 0000000..23b0def
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -0,0 +1,556 @@
+/**
+ * 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.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tajo.*;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.parser.HiveQLAnalyzer;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.GlobalEngine;
+import org.apache.tajo.master.TajoAsyncDispatcher;
+import org.apache.tajo.master.TajoContainerProxy;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.util.metrics.TajoMetrics;
+import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
+import org.apache.tajo.worker.AbstractResourceAllocator;
+import org.apache.tajo.worker.TajoResourceAllocator;
+import org.apache.tajo.worker.YarnResourceAllocator;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.tajo.TajoProtos.QueryState;
+
+public class QueryMasterTask extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName());
+
+  // query submission directory is private!
+  final public static FsPermission STAGING_DIR_PERMISSION =
+      FsPermission.createImmutable((short) 0700); // rwx--------
+
+  private QueryId queryId;
+
+  private Session session;
+
+  private QueryContext queryContext;
+
+  private QueryMasterTaskContext queryTaskContext;
+
+  private QueryMaster.QueryMasterContext queryMasterContext;
+
+  private Query query;
+
+  private MasterPlan masterPlan;
+
+  private String sql;
+
+  private String logicalPlanJson;
+
+  private TajoAsyncDispatcher dispatcher;
+
+  private final long querySubmitTime;
+
+  private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
+
+  private TajoConf systemConf;
+
+  private AtomicLong lastClientHeartbeat = new AtomicLong(-1);
+
+  private AbstractResourceAllocator resourceAllocator;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private TajoMetrics queryMetrics;
+
+  private Throwable initError;
+
+  public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext,
+                         QueryId queryId, Session session, QueryContext queryContext, String sql,
+                         String logicalPlanJson) {
+
+    super(QueryMasterTask.class.getName());
+    this.queryMasterContext = queryMasterContext;
+    this.queryId = queryId;
+    this.session = session;
+    this.queryContext = queryContext;
+    this.sql = sql;
+    this.logicalPlanJson = logicalPlanJson;
+    this.querySubmitTime = System.currentTimeMillis();
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    systemConf = (TajoConf)conf;
+
+    try {
+      queryTaskContext = new QueryMasterTaskContext();
+      String resourceManagerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_MANAGER_CLASS);
+
+      if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+        resourceAllocator = new TajoResourceAllocator(queryTaskContext);
+      } else {
+        resourceAllocator = new YarnResourceAllocator(queryTaskContext);
+      }
+      addService(resourceAllocator);
+
+      dispatcher = new TajoAsyncDispatcher(queryId.toString());
+      addService(dispatcher);
+
+      dispatcher.register(SubQueryEventType.class, new SubQueryEventDispatcher());
+      dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
+      dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
+      dispatcher.register(QueryMasterQueryCompletedEvent.EventType.class, new QueryFinishEventHandler());
+      dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
+      dispatcher.register(LocalTaskEventType.class, new LocalTaskEventHandler());
+
+      initStagingDir();
+
+      queryMetrics = new TajoMetrics(queryId.toString());
+
+      super.init(systemConf);
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+      initError = t;
+    }
+  }
+
+  public boolean isStopped() {
+    return stopped.get();
+  }
+
+  @Override
+  public void start() {
+    startQuery();
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+
+    if(stopped.getAndSet(true)) {
+      return;
+    }
+
+    LOG.info("Stopping QueryMasterTask:" + queryId);
+
+    CallFuture future = new CallFuture();
+
+    RpcConnectionPool connPool = RpcConnectionPool.getPool(queryMasterContext.getConf());
+    NettyClientBase tmClient = null;
+    try {
+      tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+          TajoMasterProtocol.class, true);
+      TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+      masterClientService.stopQueryMaster(null, queryId.getProto(), future);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    } finally {
+      connPool.releaseConnection(tmClient);
+    }
+
+    try {
+      future.get(3, TimeUnit.SECONDS);
+    } catch (Throwable t) {
+      LOG.warn(t);
+    }
+
+    super.stop();
+
+    //TODO change report to tajo master
+    queryMetrics.report(new MetricsConsoleReporter());
+
+    LOG.info("Stopped QueryMasterTask:" + queryId);
+  }
+
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+    ExecutionBlockId id = event.getExecutionBlockId();
+    query.getSubQuery(id).handleTaskRequestEvent(event);
+  }
+
+  private class SubQueryEventDispatcher implements EventHandler<SubQueryEvent> {
+    public void handle(SubQueryEvent event) {
+      ExecutionBlockId id = event.getSubQueryId();
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("SubQueryEventDispatcher:" + id + "," + event.getType());
+      }
+      //Query query = queryMasterTasks.get(id.getQueryId()).getQuery();
+      query.getSubQuery(id).handle(event);
+    }
+  }
+
+  private class TaskEventDispatcher
+      implements EventHandler<TaskEvent> {
+    public void handle(TaskEvent event) {
+      QueryUnitId taskId = event.getTaskId();
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("TaskEventDispatcher>" + taskId + "," + event.getType());
+      }
+      //Query query = queryMasterTasks.get(taskId.getExecutionBlockId().getQueryId()).getQuery();
+      QueryUnit task = query.getSubQuery(taskId.getExecutionBlockId()).
+          getQueryUnit(taskId);
+      task.handle(event);
+    }
+  }
+
+  private class TaskAttemptEventDispatcher
+      implements EventHandler<TaskAttemptEvent> {
+    public void handle(TaskAttemptEvent event) {
+      QueryUnitAttemptId attemptId = event.getTaskAttemptId();
+      //Query query = queryMasterTasks.get(attemptId.getQueryUnitId().getExecutionBlockId().getQueryId()).getQuery();
+      SubQuery subQuery = query.getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId());
+      QueryUnit task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
+      QueryUnitAttempt attempt = task.getAttempt(attemptId);
+      attempt.handle(event);
+    }
+  }
+
+  private class TaskSchedulerDispatcher
+      implements EventHandler<TaskSchedulerEvent> {
+    public void handle(TaskSchedulerEvent event) {
+      //Query query = queryMasterTasks.get(event.getExecutionBlockId().getQueryId()).getQuery();
+      SubQuery subQuery = query.getSubQuery(event.getExecutionBlockId());
+      subQuery.getTaskScheduler().handle(event);
+    }
+  }
+
+  private class LocalTaskEventHandler implements EventHandler<LocalTaskEvent> {
+    @Override
+    public void handle(LocalTaskEvent event) {
+      TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId());
+      if (proxy != null) {
+        proxy.killTaskAttempt(event.getTaskAttemptId());
+      }
+    }
+  }
+
+  private class QueryFinishEventHandler implements EventHandler<QueryMasterQueryCompletedEvent> {
+    @Override
+    public void handle(QueryMasterQueryCompletedEvent event) {
+      QueryId queryId = event.getQueryId();
+      LOG.info("Query completion notified from " + queryId);
+
+      while (!isTerminatedState(query.getState())) {
+        try {
+          synchronized (this) {
+            wait(10);
+          }
+        } catch (InterruptedException e) {
+          LOG.error(e);
+        }
+      }
+      LOG.info("Query final state: " + query.getState());
+      queryMasterContext.stopQuery(queryId);
+    }
+
+    private boolean isTerminatedState(QueryState state) {
+      return
+          state == QueryState.QUERY_SUCCEEDED ||
+          state == QueryState.QUERY_FAILED ||
+          state == QueryState.QUERY_KILLED ||
+          state == QueryState.QUERY_ERROR;
+    }
+  }
+
+  public synchronized void startQuery() {
+    try {
+      if (query != null) {
+        LOG.warn("Query already started");
+        return;
+      }
+      CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
+      LogicalPlanner planner = new LogicalPlanner(catalog);
+      LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
+      Expr expr;
+      if (queryContext.isHiveQueryMode()) {
+        HiveQLAnalyzer HiveQLAnalyzer = new HiveQLAnalyzer();
+        expr = HiveQLAnalyzer.parse(sql);
+      } else {
+        SQLAnalyzer analyzer = new SQLAnalyzer();
+        expr = analyzer.parse(sql);
+      }
+      LogicalPlan plan = planner.createPlan(session, expr);
+      optimizer.optimize(plan);
+
+      GlobalEngine.DistributedQueryHookManager hookManager = new GlobalEngine.DistributedQueryHookManager();
+      hookManager.addHook(new GlobalEngine.InsertHook());
+      hookManager.doHooks(queryContext, plan);
+
+      for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+        LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);
+        if (scanNodes != null) {
+          for (LogicalNode eachScanNode : scanNodes) {
+            ScanNode scanNode = (ScanNode) eachScanNode;
+            tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
+          }
+        }
+
+        scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.PARTITIONS_SCAN);
+        if (scanNodes != null) {
+          for (LogicalNode eachScanNode : scanNodes) {
+            ScanNode scanNode = (ScanNode) eachScanNode;
+            tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
+          }
+        }
+      }
+      MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+      queryMasterContext.getGlobalPlanner().build(masterPlan);
+
+      query = new Query(queryTaskContext, queryId, querySubmitTime,
+          "", queryTaskContext.getEventHandler(), masterPlan);
+
+      dispatcher.register(QueryEventType.class, query);
+      queryTaskContext.getEventHandler().handle(new QueryEvent(queryId, QueryEventType.START));
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+      initError = t;
+    }
+  }
+
+  /**
+   * It initializes the final output and staging directory and sets
+   * them to variables.
+   */
+  private void initStagingDir() throws IOException {
+
+    String realUser;
+    String currentUser;
+    UserGroupInformation ugi;
+    ugi = UserGroupInformation.getLoginUser();
+    realUser = ugi.getShortUserName();
+    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+    FileSystem defaultFS = TajoConf.getWarehouseDir(systemConf).getFileSystem(systemConf);
+
+    Path stagingDir = null;
+    Path outputDir = null;
+    try {
+      ////////////////////////////////////////////
+      // Create Output Directory
+      ////////////////////////////////////////////
+
+      stagingDir = new Path(TajoConf.getStagingDir(systemConf), queryId.toString());
+
+      if (defaultFS.exists(stagingDir)) {
+        throw new IOException("The staging directory '" + stagingDir + "' already exists");
+      }
+      defaultFS.mkdirs(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+      FileStatus fsStatus = defaultFS.getFileStatus(stagingDir);
+      String owner = fsStatus.getOwner();
+
+      if (!owner.isEmpty() && !(owner.equals(currentUser) || owner.equals(realUser))) {
+        throw new IOException("The ownership on the user's query " +
+            "directory " + stagingDir + " is not as expected. " +
+            "It is owned by " + owner + ". The directory must " +
+            "be owned by the submitter " + currentUser + " or " +
+            "by " + realUser);
+      }
+
+      if (!fsStatus.getPermission().equals(STAGING_DIR_PERMISSION)) {
+        LOG.info("Permissions on staging directory " + stagingDir + " are " +
+            "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
+            "to correct value " + STAGING_DIR_PERMISSION);
+        defaultFS.setPermission(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+      }
+
+      // Create a subdirectories
+      defaultFS.mkdirs(new Path(stagingDir, TajoConstants.RESULT_DIR_NAME));
+      LOG.info("The staging dir '" + stagingDir + "' is created.");
+      queryContext.setStagingDir(stagingDir);
+
+      /////////////////////////////////////////////////
+      // Check and Create Output Directory If Necessary
+      /////////////////////////////////////////////////
+      if (queryContext.hasOutputPath()) {
+        outputDir = queryContext.getOutputPath();
+        if (!queryContext.isOutputOverwrite()) {
+          if (defaultFS.exists(outputDir)) {
+            throw new IOException("The output directory '" + outputDir + " already exists.");
+          }
+        }
+      }
+    } catch (IOException ioe) {
+      if (stagingDir != null && defaultFS.exists(stagingDir)) {
+        defaultFS.delete(stagingDir, true);
+        LOG.info("The staging directory '" + stagingDir + "' is deleted");
+      }
+
+      throw ioe;
+    }
+  }
+
+  public Query getQuery() {
+    return query;
+  }
+
+  public void expiredSessionTimeout() {
+    stop();
+  }
+
+  public QueryMasterTaskContext getQueryTaskContext() {
+    return queryTaskContext;
+  }
+
+  public EventHandler getEventHandler() {
+    return queryTaskContext.getEventHandler();
+  }
+
+  public void touchSessionTime() {
+    this.lastClientHeartbeat.set(System.currentTimeMillis());
+  }
+
+  public long getLastClientHeartbeat() {
+    return this.lastClientHeartbeat.get();
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public boolean isInitError() {
+    return initError != null;
+  }
+
+  public QueryState getState() {
+    if(query == null) {
+      if (isInitError()) {
+        return QueryState.QUERY_ERROR;
+      } else {
+        return QueryState.QUERY_NOT_ASSIGNED;
+      }
+    } else {
+      return query.getState();
+    }
+  }
+
+  public String getErrorMessage() {
+    if (isInitError()) {
+      return StringUtils.stringifyException(initError);
+    } else {
+      return null;
+    }
+  }
+
+  public long getQuerySubmitTime() {
+    return this.querySubmitTime;
+  }
+
+  public class QueryMasterTaskContext {
+    EventHandler eventHandler;
+    public QueryMaster.QueryMasterContext getQueryMasterContext() {
+      return queryMasterContext;
+    }
+
+    public Session getSession() {
+      return session;
+    }
+
+    public QueryContext getQueryContext() {
+      return queryContext;
+    }
+
+    public TajoConf getConf() {
+      return systemConf;
+    }
+
+    public Clock getClock() {
+      return queryMasterContext.getClock();
+    }
+
+    public Query getQuery() {
+      return query;
+    }
+
+    public QueryId getQueryId() {
+      return queryId;
+    }
+
+    public AbstractStorageManager getStorageManager() {
+      return queryMasterContext.getStorageManager();
+    }
+
+    public Path getStagingDir() {
+      return queryContext.getStagingDir();
+    }
+
+    public synchronized EventHandler getEventHandler() {
+      if(eventHandler == null) {
+        eventHandler = dispatcher.getEventHandler();
+      }
+      return eventHandler;
+    }
+
+    public TajoAsyncDispatcher getDispatcher() {
+      return dispatcher;
+    }
+
+    public SubQuery getSubQuery(ExecutionBlockId id) {
+      return query.getSubQuery(id);
+    }
+
+    public Map<String, TableDesc> getTableDescMap() {
+      return tableDescMap;
+    }
+
+    public float getProgress() {
+      if(query == null) {
+        return 0.0f;
+      }
+      return query.getProgress();
+    }
+
+    public AbstractResourceAllocator getResourceAllocator() {
+      return resourceAllocator;
+    }
+
+    public TajoMetrics getQueryMetrics() {
+      return queryMetrics;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
new file mode 100644
index 0000000..42fbf8a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
@@ -0,0 +1,658 @@
+/**
+ * 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.master.querymaster;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.QueryUnitId;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.master.FragmentPair;
+import org.apache.tajo.master.TaskState;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext;
+import org.apache.tajo.storage.DataLocation;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.net.URI;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput;
+
+public class QueryUnit implements EventHandler<TaskEvent> {
+  /** Class Logger */
+  private static final Log LOG = LogFactory.getLog(QueryUnit.class);
+
+  private final Configuration systemConf;
+	private QueryUnitId taskId;
+  private EventHandler eventHandler;
+	private StoreTableNode store = null;
+	private LogicalNode plan = null;
+	private List<ScanNode> scan;
+	
+	private Map<String, Set<FragmentProto>> fragMap;
+	private Map<String, Set<URI>> fetchMap;
+
+  private int totalFragmentNum;
+
+  private List<ShuffleFileOutput> shuffleFileOutputs;
+	private TableStats stats;
+  private final boolean isLeafTask;
+  private List<IntermediateEntry> intermediateData;
+
+  private Map<QueryUnitAttemptId, QueryUnitAttempt> attempts;
+  private final int maxAttempts = 3;
+  private Integer nextAttempt = -1;
+  private QueryUnitAttemptId lastAttemptId;
+
+  private QueryUnitAttemptId successfulAttempt;
+  private String succeededHost;
+  private int succeededPullServerPort;
+
+  private int failedAttempts;
+  private int finishedAttempts; // finish are total of success, failed and killed
+
+  private long launchTime;
+  private long finishTime;
+
+  private List<DataLocation> dataLocations = Lists.newArrayList();
+
+  private static final AttemptKilledTransition ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
+
+  protected static final StateMachineFactory
+      <QueryUnit, TaskState, TaskEventType, TaskEvent> stateMachineFactory =
+      new StateMachineFactory <QueryUnit, TaskState, TaskEventType, TaskEvent>(TaskState.NEW)
+
+          // Transitions from NEW state
+          .addTransition(TaskState.NEW, TaskState.SCHEDULED,
+              TaskEventType.T_SCHEDULE,
+              new InitialScheduleTransition())
+          .addTransition(TaskState.NEW, TaskState.KILLED,
+              TaskEventType.T_KILL,
+              new KillNewTaskTransition())
+
+          // Transitions from SCHEDULED state
+          .addTransition(TaskState.SCHEDULED, TaskState.RUNNING,
+              TaskEventType.T_ATTEMPT_LAUNCHED,
+              new AttemptLaunchedTransition())
+          .addTransition(TaskState.SCHEDULED, TaskState.KILL_WAIT,
+              TaskEventType.T_KILL,
+              new KillTaskTransition())
+
+          // Transitions from RUNNING state
+          .addTransition(TaskState.RUNNING, TaskState.RUNNING,
+              TaskEventType.T_ATTEMPT_LAUNCHED)
+          .addTransition(TaskState.RUNNING, TaskState.SUCCEEDED,
+              TaskEventType.T_ATTEMPT_SUCCEEDED,
+              new AttemptSucceededTransition())
+          .addTransition(TaskState.RUNNING, TaskState.KILL_WAIT,
+              TaskEventType.T_KILL,
+              new KillTaskTransition())
+          .addTransition(TaskState.RUNNING,
+              EnumSet.of(TaskState.RUNNING, TaskState.FAILED),
+              TaskEventType.T_ATTEMPT_FAILED,
+              new AttemptFailedOrRetryTransition())
+
+          // Transitions from KILL_WAIT state
+          .addTransition(TaskState.KILL_WAIT, TaskState.KILLED,
+              TaskEventType.T_ATTEMPT_KILLED,
+              ATTEMPT_KILLED_TRANSITION)
+          .addTransition(TaskState.KILL_WAIT, TaskState.KILL_WAIT,
+              TaskEventType.T_ATTEMPT_LAUNCHED,
+              new KillTaskTransition())
+          .addTransition(TaskState.KILL_WAIT, TaskState.FAILED,
+              TaskEventType.T_ATTEMPT_FAILED,
+              new AttemptFailedTransition())
+          .addTransition(TaskState.KILL_WAIT, TaskState.KILLED,
+              TaskEventType.T_ATTEMPT_SUCCEEDED,
+              ATTEMPT_KILLED_TRANSITION)
+              // Ignore-able transitions.
+          .addTransition(TaskState.KILL_WAIT, TaskState.KILL_WAIT,
+              EnumSet.of(
+                  TaskEventType.T_KILL,
+                  TaskEventType.T_SCHEDULE))
+
+          // Transitions from SUCCEEDED state
+          // Ignore-able transitions
+          .addTransition(TaskState.SUCCEEDED, TaskState.SUCCEEDED,
+              EnumSet.of(TaskEventType.T_KILL, TaskEventType.T_ATTEMPT_KILLED, TaskEventType.T_ATTEMPT_SUCCEEDED))
+
+          // Transitions from FAILED state
+          // Ignore-able transitions
+          .addTransition(TaskState.FAILED, TaskState.FAILED,
+              EnumSet.of(TaskEventType.T_KILL, TaskEventType.T_ATTEMPT_KILLED, TaskEventType.T_ATTEMPT_SUCCEEDED))
+
+          .installTopology();
+
+  private final StateMachine<TaskState, TaskEventType, TaskEvent> stateMachine;
+
+
+  private final Lock readLock;
+  private final Lock writeLock;
+  private QueryUnitAttemptScheduleContext scheduleContext;
+
+	public QueryUnit(Configuration conf, QueryUnitAttemptScheduleContext scheduleContext,
+                   QueryUnitId id, boolean isLeafTask, EventHandler eventHandler) {
+    this.systemConf = conf;
+		this.taskId = id;
+    this.eventHandler = eventHandler;
+    this.isLeafTask = isLeafTask;
+		scan = new ArrayList<ScanNode>();
+    fetchMap = Maps.newHashMap();
+    fragMap = Maps.newHashMap();
+    shuffleFileOutputs = new ArrayList<ShuffleFileOutput>();
+    attempts = Collections.emptyMap();
+    lastAttemptId = null;
+    nextAttempt = -1;
+    failedAttempts = 0;
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+    this.scheduleContext = scheduleContext;
+
+    stateMachine = stateMachineFactory.make(this);
+    totalFragmentNum = 0;
+	}
+
+  public boolean isLeafTask() {
+    return this.isLeafTask;
+  }
+
+  public TaskState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+	public void setLogicalPlan(LogicalNode plan) {
+	  this.plan = plan;
+
+	  LogicalNode node = plan;
+	  ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
+	  s.add(node);
+	  while (!s.isEmpty()) {
+	    node = s.remove(s.size()-1);
+	    if (node instanceof UnaryNode) {
+	      UnaryNode unary = (UnaryNode) node;
+	      s.add(s.size(), unary.getChild());
+	    } else if (node instanceof BinaryNode) {
+	      BinaryNode binary = (BinaryNode) node;
+	      s.add(s.size(), binary.getLeftChild());
+	      s.add(s.size(), binary.getRightChild());
+	    } else if (node instanceof ScanNode) {
+	      scan.add((ScanNode)node);
+	    } else if (node instanceof TableSubQueryNode) {
+        s.add(((TableSubQueryNode) node).getSubQuery());
+      }
+	  }
+	}
+
+  private void addDataLocation(FileFragment fragment) {
+    String[] hosts = fragment.getHosts();
+    int[] diskIds = fragment.getDiskIds();
+    for (int i = 0; i < hosts.length; i++) {
+      dataLocations.add(new DataLocation(hosts[i], diskIds[i]));
+    }
+  }
+
+  public void addFragment(FileFragment fragment, boolean useDataLocation) {
+    Set<FragmentProto> fragmentProtos;
+    if (fragMap.containsKey(fragment.getTableName())) {
+      fragmentProtos = fragMap.get(fragment.getTableName());
+    } else {
+      fragmentProtos = new HashSet<FragmentProto>();
+      fragMap.put(fragment.getTableName(), fragmentProtos);
+    }
+    fragmentProtos.add(fragment.getProto());
+    if (useDataLocation) {
+      addDataLocation(fragment);
+    }
+    totalFragmentNum++;
+  }
+
+  public void addFragments(Collection<FileFragment> fragments) {
+    for (FileFragment eachFragment: fragments) {
+      addFragment(eachFragment, false);
+    }
+  }
+
+  public void setFragment(FragmentPair[] fragmentPairs) {
+    for (FragmentPair eachFragmentPair : fragmentPairs) {
+      this.addFragment(eachFragmentPair.getLeftFragment(), true);
+      if (eachFragmentPair.getRightFragment() != null) {
+        this.addFragment(eachFragmentPair.getRightFragment(), true);
+      }
+    }
+  }
+
+  public List<DataLocation> getDataLocations() {
+    return dataLocations;
+  }
+
+  public String getSucceededHost() {
+    return succeededHost;
+  }
+	
+	public void addFetches(String tableId, Collection<URI> urilist) {
+	  Set<URI> uris;
+    if (fetchMap.containsKey(tableId)) {
+      uris = fetchMap.get(tableId);
+    } else {
+      uris = Sets.newHashSet();
+    }
+    uris.addAll(urilist);
+    fetchMap.put(tableId, uris);
+	}
+	
+	public void setFetches(Map<String, Set<URI>> fetches) {
+	  this.fetchMap.clear();
+	  this.fetchMap.putAll(fetches);
+	}
+
+  public Collection<FragmentProto> getAllFragments() {
+    Set<FragmentProto> fragmentProtos = new HashSet<FragmentProto>();
+    for (Set<FragmentProto> eachFragmentSet : fragMap.values()) {
+      fragmentProtos.addAll(eachFragmentSet);
+    }
+    return fragmentProtos;
+  }
+	
+	public LogicalNode getLogicalPlan() {
+	  return this.plan;
+	}
+	
+	public QueryUnitId getId() {
+		return taskId;
+	}
+	
+	public Collection<URI> getFetchHosts(String tableId) {
+	  return fetchMap.get(tableId);
+	}
+	
+	public Collection<Set<URI>> getFetches() {
+	  return fetchMap.values();
+	}
+
+  public Map<String, Set<URI>> getFetchMap() {
+    return fetchMap;
+  }
+	
+	public Collection<URI> getFetch(ScanNode scan) {
+	  return this.fetchMap.get(scan.getTableName());
+	}
+	
+	public ScanNode[] getScanNodes() {
+	  return this.scan.toArray(new ScanNode[scan.size()]);
+	}
+	
+	@Override
+	public String toString() {
+		String str = new String(plan.getType() + " \n");
+		for (Entry<String, Set<FragmentProto>> e : fragMap.entrySet()) {
+		  str += e.getKey() + " : ";
+      for (FragmentProto fragment : e.getValue()) {
+        str += fragment + ", ";
+      }
+		}
+		for (Entry<String, Set<URI>> e : fetchMap.entrySet()) {
+      str += e.getKey() + " : ";
+      for (URI t : e.getValue()) {
+        str += t + " ";
+      }
+    }
+		
+		return str;
+	}
+	
+	public void setStats(TableStats stats) {
+	  this.stats = stats;
+	}
+	
+	public void setShuffleFileOutputs(List<ShuffleFileOutput> partitions) {
+	  this.shuffleFileOutputs = Collections.unmodifiableList(partitions);
+	}
+	
+	public TableStats getStats() {
+	  return this.stats;
+	}
+	
+	public List<ShuffleFileOutput> getShuffleFileOutputs() {
+	  return this.shuffleFileOutputs;
+	}
+	
+	public int getShuffleOutpuNum() {
+	  return this.shuffleFileOutputs.size();
+	}
+
+  public QueryUnitAttempt newAttempt() {
+    QueryUnitAttempt attempt = new QueryUnitAttempt(scheduleContext,
+        QueryIdFactory.newQueryUnitAttemptId(this.getId(), ++nextAttempt),
+        this, eventHandler);
+    lastAttemptId = attempt.getId();
+    return attempt;
+  }
+
+  public QueryUnitAttempt getAttempt(QueryUnitAttemptId attemptId) {
+    return attempts.get(attemptId);
+  }
+
+  public QueryUnitAttempt getAttempt(int attempt) {
+    return this.attempts.get(QueryIdFactory.newQueryUnitAttemptId(this.getId(), attempt));
+  }
+
+  public QueryUnitAttempt getLastAttempt() {
+    return getAttempt(this.lastAttemptId);
+  }
+
+  public QueryUnitAttempt getSuccessfulAttempt() {
+    readLock.lock();
+    try {
+      if (null == successfulAttempt) {
+        return null;
+      }
+      return attempts.get(successfulAttempt);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public int getRetryCount () {
+    return this.nextAttempt;
+  }
+
+  public int getTotalFragmentNum() {
+    return totalFragmentNum;
+  }
+
+  private static class InitialScheduleTransition implements
+    SingleArcTransition<QueryUnit, TaskEvent> {
+
+    @Override
+    public void transition(QueryUnit task, TaskEvent taskEvent) {
+      task.addAndScheduleAttempt();
+    }
+  }
+
+  public long getLaunchTime() {
+    return launchTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  @VisibleForTesting
+  public void setLaunchTime(long launchTime) {
+    this.launchTime = launchTime;
+  }
+
+  @VisibleForTesting
+  public void setFinishTime(long finishTime) {
+    this.finishTime = finishTime;
+  }
+
+  public long getRunningTime() {
+    if(finishTime > 0) {
+      return finishTime - launchTime;
+    } else {
+      return System.currentTimeMillis() - launchTime;
+    }
+  }
+
+  // This is always called in the Write Lock
+  private void addAndScheduleAttempt() {
+    // Create new task attempt
+    QueryUnitAttempt attempt = newAttempt();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Created attempt " + attempt.getId());
+    }
+    switch (attempts.size()) {
+      case 0:
+        attempts = Collections.singletonMap(attempt.getId(), attempt);
+        break;
+
+      case 1:
+        Map<QueryUnitAttemptId, QueryUnitAttempt> newAttempts
+            = new LinkedHashMap<QueryUnitAttemptId, QueryUnitAttempt>(3);
+        newAttempts.putAll(attempts);
+        attempts = newAttempts;
+        attempts.put(attempt.getId(), attempt);
+        break;
+
+      default:
+        attempts.put(attempt.getId(), attempt);
+        break;
+    }
+
+    if (failedAttempts > 0) {
+      eventHandler.handle(new TaskAttemptScheduleEvent(systemConf, attempt.getId(),
+          TaskAttemptEventType.TA_RESCHEDULE));
+    } else {
+      eventHandler.handle(new TaskAttemptScheduleEvent(systemConf, attempt.getId(),
+          TaskAttemptEventType.TA_SCHEDULE));
+    }
+  }
+
+  private void finishTask() {
+    this.finishTime = System.currentTimeMillis();
+  }
+
+  private static class KillNewTaskTransition implements SingleArcTransition<QueryUnit, TaskEvent> {
+
+    @Override
+    public void transition(QueryUnit task, TaskEvent taskEvent) {
+      task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.KILLED));
+    }
+  }
+
+  private static class KillTaskTransition implements SingleArcTransition<QueryUnit, TaskEvent> {
+
+    @Override
+    public void transition(QueryUnit task, TaskEvent taskEvent) {
+      task.finishTask();
+      task.eventHandler.handle(new TaskAttemptEvent(task.lastAttemptId, TaskAttemptEventType.TA_KILL));
+    }
+  }
+
+  private static class AttemptKilledTransition implements SingleArcTransition<QueryUnit, TaskEvent>{
+
+    @Override
+    public void transition(QueryUnit task, TaskEvent event) {
+      task.finishTask();
+      task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.KILLED));
+    }
+  }
+
+  private static class AttemptSucceededTransition
+      implements SingleArcTransition<QueryUnit, TaskEvent>{
+
+    @Override
+    public void transition(QueryUnit task,
+                           TaskEvent event) {
+      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
+      QueryUnitAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId());
+
+      task.successfulAttempt = attemptEvent.getTaskAttemptId();
+      task.succeededHost = attempt.getHost();
+      task.succeededPullServerPort = attempt.getPullServerPort();
+
+      task.finishTask();
+      task.eventHandler.handle(new SubQueryTaskEvent(event.getTaskId(), TaskState.SUCCEEDED));
+    }
+  }
+
+  private static class AttemptLaunchedTransition implements SingleArcTransition<QueryUnit, TaskEvent> {
+    @Override
+    public void transition(QueryUnit task,
+                           TaskEvent event) {
+      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
+      QueryUnitAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId());
+      task.launchTime = System.currentTimeMillis();
+      task.succeededHost = attempt.getHost();
+    }
+  }
+
+  private static class AttemptFailedTransition implements SingleArcTransition<QueryUnit, TaskEvent> {
+    @Override
+    public void transition(QueryUnit task, TaskEvent event) {
+      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
+      LOG.info("=============================================================");
+      LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + " <<<");
+      LOG.info("=============================================================");
+      task.failedAttempts++;
+      task.finishedAttempts++;
+
+      task.finishTask();
+      task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.FAILED));
+    }
+  }
+
+  private static class AttemptFailedOrRetryTransition implements
+    MultipleArcTransition<QueryUnit, TaskEvent, TaskState> {
+
+    @Override
+    public TaskState transition(QueryUnit task, TaskEvent taskEvent) {
+      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) taskEvent;
+      LOG.info("=============================================================");
+      LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + " <<<");
+      LOG.info("=============================================================");
+      task.failedAttempts++;
+      task.finishedAttempts++;
+
+      if (task.failedAttempts < task.maxAttempts) {
+        if (task.successfulAttempt == null) {
+          task.addAndScheduleAttempt();
+        }
+      } else {
+        task.finishTask();
+        task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.FAILED));
+        return TaskState.FAILED;
+      }
+
+      return task.getState();
+    }
+  }
+
+  @Override
+  public void handle(TaskEvent event) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getTaskId() + " of type "
+          + event.getType());
+    }
+
+    try {
+      writeLock.lock();
+      TaskState oldState = getState();
+      try {
+        stateMachine.doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state", e);
+        eventHandler.handle(new QueryEvent(TajoIdUtils.parseQueryId(getId().toString()),
+            QueryEventType.INTERNAL_ERROR));
+      }
+
+      //notify the eventhandler of state change
+      if (LOG.isDebugEnabled()) {
+        if (oldState != getState()) {
+          LOG.debug(taskId + " Task Transitioned from " + oldState + " to "
+              + getState());
+        }
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void setIntermediateData(Collection<IntermediateEntry> partitions) {
+    this.intermediateData = new ArrayList<IntermediateEntry>(partitions);
+  }
+
+  public List<IntermediateEntry> getIntermediateData() {
+    return this.intermediateData;
+  }
+
+  public static class IntermediateEntry {
+    int taskId;
+    int attemptId;
+    int partId;
+    String pullHost;
+    int port;
+
+    public IntermediateEntry(int taskId, int attemptId, int partId,
+                             String pullServerAddr, int pullServerPort) {
+      this.taskId = taskId;
+      this.attemptId = attemptId;
+      this.partId = partId;
+      this.pullHost = pullServerAddr;
+      this.port = pullServerPort;
+    }
+
+    public int getTaskId() {
+      return this.taskId;
+    }
+
+    public int getAttemptId() {
+      return this.attemptId;
+    }
+
+    public int getPartId() {
+      return this.partId;
+    }
+
+    public String getPullHost() {
+      return this.pullHost;
+    }
+
+    public int getPullPort() {
+      return port;
+    }
+
+    public String getPullAddress() {
+      return pullHost + ":" + port;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(taskId, attemptId, partId, pullHost, port);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
new file mode 100644
index 0000000..7993ce9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
@@ -0,0 +1,442 @@
+/**
+ * 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.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoProtos.TaskAttemptState;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext;
+import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput;
+
+public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
+
+  private static final Log LOG = LogFactory.getLog(QueryUnitAttempt.class);
+
+  private final static int EXPIRE_TIME = 15000;
+
+  private final QueryUnitAttemptId id;
+  private final QueryUnit queryUnit;
+  final EventHandler eventHandler;
+
+  private ContainerId containerId;
+  private String hostName;
+  private int port;
+  private int expire;
+
+  private final Lock readLock;
+  private final Lock writeLock;
+
+  private final List<String> diagnostics = new ArrayList<String>();
+
+  private final QueryUnitAttemptScheduleContext scheduleContext;
+
+  private float progress;
+  private CatalogProtos.TableStatsProto inputStats;
+  private CatalogProtos.TableStatsProto resultStats;
+
+  protected static final StateMachineFactory
+      <QueryUnitAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+      stateMachineFactory = new StateMachineFactory
+      <QueryUnitAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+      (TaskAttemptState.TA_NEW)
+
+      // Transitions from TA_NEW state
+      .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
+          TaskAttemptEventType.TA_SCHEDULE, new TaskAttemptScheduleTransition())
+      .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
+          TaskAttemptEventType.TA_RESCHEDULE, new TaskAttemptScheduleTransition())
+      .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_KILLED,
+          TaskAttemptEventType.TA_KILL,
+          new TaskKilledCompleteTransition())
+
+      // Transitions from TA_UNASSIGNED state
+      .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_ASSIGNED,
+          TaskAttemptEventType.TA_ASSIGNED,
+          new LaunchTransition())
+      .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_KILL_WAIT,
+          TaskAttemptEventType.TA_KILL,
+          new KillUnassignedTaskTransition())
+
+      // Transitions from TA_ASSIGNED state
+      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_ASSIGNED,
+          TaskAttemptEventType.TA_ASSIGNED, new AlreadyAssignedTransition())
+      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_KILL_WAIT,
+          TaskAttemptEventType.TA_KILL,
+          new KillTaskTransition())
+      .addTransition(TaskAttemptState.TA_ASSIGNED,
+          EnumSet.of(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILLED),
+          TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
+      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_DONE, new SucceededTransition())
+      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_FAILED,
+          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+
+      // Transitions from TA_RUNNING state
+      .addTransition(TaskAttemptState.TA_RUNNING,
+          EnumSet.of(TaskAttemptState.TA_RUNNING),
+          TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
+      .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILL_WAIT,
+          TaskAttemptEventType.TA_KILL,
+          new KillTaskTransition())
+      .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_DONE, new SucceededTransition())
+      .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_FAILED,
+          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+
+      .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
+          TaskAttemptEventType.TA_LOCAL_KILLED,
+          new TaskKilledCompleteTransition())
+      .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILL_WAIT,
+          TaskAttemptEventType.TA_ASSIGNED,
+          new KillTaskTransition())
+      .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
+          TaskAttemptEventType.TA_SCHEDULE_CANCELED,
+          new TaskKilledCompleteTransition())
+      .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
+          TaskAttemptEventType.TA_DONE,
+          new TaskKilledCompleteTransition())
+      .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_FAILED,
+          TaskAttemptEventType.TA_FATAL_ERROR)
+      .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILL_WAIT,
+          EnumSet.of(
+              TaskAttemptEventType.TA_KILL,
+              TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
+              TaskAttemptEventType.TA_UPDATE))
+
+      // Transitions from TA_SUCCEEDED state
+      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_UPDATE)
+      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_DONE, new AlreadyDoneTransition())
+      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_FAILED,
+          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+       // Ignore-able transitions
+      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_KILL)
+
+      // Transitions from TA_KILLED state
+      .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
+          TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE)
+      // Ignore-able transitions
+      .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
+          EnumSet.of(
+              TaskAttemptEventType.TA_UPDATE))
+
+      .installTopology();
+
+  private final StateMachine<TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+    stateMachine;
+
+
+  public QueryUnitAttempt(final QueryUnitAttemptScheduleContext scheduleContext,
+                          final QueryUnitAttemptId id, final QueryUnit queryUnit,
+                          final EventHandler eventHandler) {
+    this.scheduleContext = scheduleContext;
+    this.id = id;
+    this.expire = QueryUnitAttempt.EXPIRE_TIME;
+    this.queryUnit = queryUnit;
+    this.eventHandler = eventHandler;
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+
+    stateMachine = stateMachineFactory.make(this);
+  }
+
+  public TaskAttemptState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public QueryUnitAttemptId getId() {
+    return this.id;
+  }
+
+  public boolean isLeafTask() {
+    return this.queryUnit.isLeafTask();
+  }
+
+  public QueryUnit getQueryUnit() {
+    return this.queryUnit;
+  }
+
+  public String getHost() {
+    return this.hostName;
+  }
+
+  public int getPort() {
+    return this.port;
+  }
+
+  public void setContainerId(ContainerId containerId) {
+    this.containerId = containerId;
+  }
+
+  public void setHost(String host) {
+    this.hostName = host;
+  }
+
+  public void setPullServerPort(int port) {
+    this.port = port;
+  }
+
+  public int getPullServerPort() {
+    return port;
+  }
+
+  public synchronized void setExpireTime(int expire) {
+    this.expire = expire;
+  }
+
+  public synchronized void updateExpireTime(int period) {
+    this.setExpireTime(this.expire - period);
+  }
+
+  public synchronized void resetExpireTime() {
+    this.setExpireTime(QueryUnitAttempt.EXPIRE_TIME);
+  }
+
+  public int getLeftTime() {
+    return this.expire;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  public TableStats getInputStats() {
+    if (inputStats == null) {
+      return null;
+    }
+
+    return new TableStats(inputStats);
+  }
+
+  public TableStats getResultStats() {
+    if (resultStats == null) {
+      return null;
+    }
+    return new TableStats(resultStats);
+  }
+
+  private void fillTaskStatistics(TaskCompletionReport report) {
+    this.progress = 1.0f;
+
+    List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
+
+    if (report.getShuffleFileOutputsCount() > 0) {
+      this.getQueryUnit().setShuffleFileOutputs(report.getShuffleFileOutputsList());
+
+      for (ShuffleFileOutput p : report.getShuffleFileOutputsList()) {
+        IntermediateEntry entry = new IntermediateEntry(getId().getQueryUnitId().getId(),
+            getId().getId(), p.getPartId(), getHost(), getPullServerPort());
+        partitions.add(entry);
+      }
+    }
+    this.getQueryUnit().setIntermediateData(partitions);
+
+    if (report.hasInputStats()) {
+      this.inputStats = report.getInputStats();
+    }
+    if (report.hasResultStats()) {
+      this.resultStats = report.getResultStats();
+      this.getQueryUnit().setStats(new TableStats(resultStats));
+    }
+  }
+
+  private static class TaskAttemptScheduleTransition implements
+      SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt, TaskAttemptEvent taskAttemptEvent) {
+      taskAttempt.eventHandler.handle(new QueryUnitAttemptScheduleEvent(
+          EventType.T_SCHEDULE, taskAttempt.getQueryUnit().getId().getExecutionBlockId(),
+          taskAttempt.scheduleContext, taskAttempt));
+    }
+  }
+
+  private static class KillUnassignedTaskTransition implements
+      SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt, TaskAttemptEvent taskAttemptEvent) {
+      taskAttempt.eventHandler.handle(new QueryUnitAttemptScheduleEvent(
+          EventType.T_SCHEDULE_CANCEL, taskAttempt.getQueryUnit().getId().getExecutionBlockId(),
+          taskAttempt.scheduleContext, taskAttempt));
+    }
+  }
+
+  private static class LaunchTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt,
+                           TaskAttemptEvent event) {
+      TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event;
+      taskAttempt.containerId = castEvent.getContainerId();
+      taskAttempt.setHost(castEvent.getHostName());
+      taskAttempt.setPullServerPort(castEvent.getPullServerPort());
+      taskAttempt.eventHandler.handle(
+          new TaskTAttemptEvent(taskAttempt.getId(),
+              TaskEventType.T_ATTEMPT_LAUNCHED));
+    }
+  }
+
+  private static class TaskKilledCompleteTransition implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt,
+                           TaskAttemptEvent event) {
+      taskAttempt.getQueryUnit().handle(new TaskEvent(taskAttempt.getId().getQueryUnitId(),
+          TaskEventType.T_ATTEMPT_KILLED));
+      LOG.info(taskAttempt.getId() + " Received TA_KILLED Status from LocalTask");
+    }
+  }
+
+  private static class StatusUpdateTransition
+      implements MultipleArcTransition<QueryUnitAttempt, TaskAttemptEvent, TaskAttemptState> {
+
+    @Override
+    public TaskAttemptState transition(QueryUnitAttempt taskAttempt,
+                                       TaskAttemptEvent event) {
+      TaskAttemptStatusUpdateEvent updateEvent = (TaskAttemptStatusUpdateEvent) event;
+
+      taskAttempt.progress = updateEvent.getStatus().getProgress();
+      taskAttempt.inputStats = updateEvent.getStatus().getInputStats();
+      taskAttempt.resultStats = updateEvent.getStatus().getResultStats();
+
+      return TaskAttemptState.TA_RUNNING;
+    }
+  }
+
+  private void addDiagnosticInfo(String diag) {
+    if (diag != null && !diag.equals("")) {
+      diagnostics.add(diag);
+    }
+  }
+
+  private static class AlreadyAssignedTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
+
+    @Override
+    public void transition(QueryUnitAttempt queryUnitAttempt,
+                           TaskAttemptEvent taskAttemptEvent) {
+    }
+  }
+
+  private static class AlreadyDoneTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
+
+    @Override
+    public void transition(QueryUnitAttempt queryUnitAttempt,
+                           TaskAttemptEvent taskAttemptEvent) {
+    }
+  }
+
+  private static class SucceededTransition implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt,
+                           TaskAttemptEvent event) {
+      TaskCompletionReport report = ((TaskCompletionEvent)event).getReport();
+
+      try {
+        taskAttempt.fillTaskStatistics(report);
+        taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(), TaskEventType.T_ATTEMPT_SUCCEEDED));
+      } catch (Throwable t) {
+        taskAttempt.eventHandler.handle(new TaskFatalErrorEvent(taskAttempt.getId(), t.getMessage()));
+      }
+    }
+  }
+
+  private static class KillTaskTransition implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt, TaskAttemptEvent event) {
+      taskAttempt.eventHandler.handle(new LocalTaskEvent(taskAttempt.getId(), taskAttempt.containerId,
+          LocalTaskEventType.KILL));
+    }
+  }
+
+  private static class FailedTransition implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt, TaskAttemptEvent event) {
+      TaskFatalErrorEvent errorEvent = (TaskFatalErrorEvent) event;
+      taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(), TaskEventType.T_ATTEMPT_FAILED));
+      taskAttempt.addDiagnosticInfo(errorEvent.errorMessage());
+      LOG.error("FROM " + taskAttempt.getHost() + " >> " + errorEvent.errorMessage());
+    }
+  }
+
+  @Override
+  public void handle(TaskAttemptEvent event) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getTaskAttemptId() + " of type " + event.getType());
+    }
+    try {
+      writeLock.lock();
+      TaskAttemptState oldState = getState();
+      try {
+        stateMachine.doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state of " + event.getTaskAttemptId() + ")", e);
+        eventHandler.handle(
+            new SubQueryDiagnosticsUpdateEvent(event.getTaskAttemptId().getQueryUnitId().getExecutionBlockId(),
+                "Can't handle this event at current state of " + event.getTaskAttemptId() + ")"));
+        eventHandler.handle(
+            new SubQueryEvent(event.getTaskAttemptId().getQueryUnitId().getExecutionBlockId(),
+                SubQueryEventType.SQ_INTERNAL_ERROR));
+      }
+
+      //notify the eventhandler of state change
+      if (LOG.isDebugEnabled()) {
+       if (oldState != getState()) {
+          LOG.debug(id + " TaskAttempt Transitioned from " + oldState + " to "
+              + getState());
+        }
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
new file mode 100644
index 0000000..31d433d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -0,0 +1,653 @@
+/**
+ * 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.master.querymaster;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
+import org.apache.tajo.engine.planner.UniformRangePartition;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.master.TaskSchedulerContext;
+import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.math.BigDecimal;
+import java.net.URI;
+import java.util.*;
+import java.util.Map.Entry;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.HASH_SHUFFLE;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.RANGE_SHUFFLE;
+
+/**
+ * Repartitioner creates non-leaf tasks and shuffles intermediate data.
+ * It supports two repartition methods, such as hash and range repartition.
+ */
+public class Repartitioner {
+  private static final Log LOG = LogFactory.getLog(Repartitioner.class);
+
+  private static int HTTP_REQUEST_MAXIMUM_LENGTH = 1900;
+  private final static String UNKNOWN_HOST = "unknown";
+
+  public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerContext, SubQuery subQuery)
+      throws IOException {
+    MasterPlan masterPlan = subQuery.getMasterPlan();
+    ExecutionBlock execBlock = subQuery.getBlock();
+    QueryMasterTask.QueryMasterTaskContext masterContext = subQuery.getContext();
+    AbstractStorageManager storageManager = subQuery.getStorageManager();
+
+    ScanNode[] scans = execBlock.getScanNodes();
+
+    Path tablePath;
+    FileFragment[] fragments = new FileFragment[scans.length];
+    long[] stats = new long[scans.length];
+
+    // initialize variables from the child operators
+    for (int i = 0; i < scans.length; i++) {
+      TableDesc tableDesc = masterContext.getTableDescMap().get(scans[i].getCanonicalName());
+      if (tableDesc == null) { // if it is a real table stored on storage
+        // TODO - to be fixed (wrong directory)
+        ExecutionBlock [] childBlocks = new ExecutionBlock[2];
+        childBlocks[0] = masterPlan.getChild(execBlock.getId(), 0);
+        childBlocks[1] = masterPlan.getChild(execBlock.getId(), 1);
+
+        tablePath = storageManager.getTablePath(scans[i].getTableName());
+        stats[i] = masterContext.getSubQuery(childBlocks[i].getId()).getResultStats().getNumBytes();
+        fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+      } else {
+        tablePath = tableDesc.getPath();
+        try {
+          stats[i] = GlobalPlanner.computeDescendentVolume(scans[i]);
+        } catch (PlanningException e) {
+          throw new IOException(e);
+        }
+
+        // if table has no data, storageManager will return empty FileFragment.
+        // So, we need to handle FileFragment by its size.
+        // If we don't check its size, it can cause IndexOutOfBoundsException.
+        List<FileFragment> fileFragments = storageManager.getSplits(scans[i].getCanonicalName(), tableDesc.getMeta(), tableDesc.getSchema(), tablePath);
+        if (fileFragments.size() > 0) {
+          fragments[i] = fileFragments.get(0);
+        } else {
+          fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+        }
+      }
+    }
+
+    // If one of inner join tables has no input data,
+    // it should return zero rows.
+    JoinNode joinNode = PlannerUtil.findMostBottomNode(execBlock.getPlan(), NodeType.JOIN);
+    if (joinNode != null) {
+      if ( (joinNode.getJoinType().equals(JoinType.INNER))) {
+        for (int i = 0; i < stats.length; i++) {
+          if (stats[i] == 0) {
+            return;
+          }
+        }
+      }
+    }
+
+    // Assigning either fragments or fetch urls to query units
+    boolean isAllBroadcastTable = true;
+    int baseScanIdx = -1;
+    for (int i = 0; i < scans.length; i++) {
+      if (!execBlock.isBroadcastTable(scans[i].getCanonicalName())) {
+        isAllBroadcastTable = false;
+        baseScanIdx = i;
+      }
+    }
+
+    if (isAllBroadcastTable) {
+      LOG.info("[Distributed Join Strategy] : Immediate " +  fragments.length + " Way Join on Single Machine");
+      SubQuery.scheduleFragment(subQuery, fragments[0], Arrays.asList(Arrays.copyOfRange(fragments, 1, fragments.length)));
+      schedulerContext.setEstimatedTaskNum(1);
+    } else if (!execBlock.getBroadcastTables().isEmpty()) {
+      LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join, base_table=%s, base_volume=%d",
+          scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx]));
+      scheduleLeafTasksWithBroadcastTable(schedulerContext, subQuery, baseScanIdx, fragments);
+    } else {
+      LOG.info("[Distributed Join Strategy] : Symmetric Repartition Join");
+      // The hash map is modeling as follows:
+      // <Part Id, <Table Name, Intermediate Data>>
+      Map<Integer, Map<String, List<IntermediateEntry>>> hashEntries = new HashMap<Integer, Map<String, List<IntermediateEntry>>>();
+
+      // Grouping IntermediateData by a partition key and a table name
+      for (ScanNode scan : scans) {
+        SubQuery childSubQuery = masterContext.getSubQuery(TajoIdUtils.createExecutionBlockId(scan.getCanonicalName()));
+        for (QueryUnit task : childSubQuery.getQueryUnits()) {
+          if (task.getIntermediateData() != null && !task.getIntermediateData().isEmpty()) {
+            for (IntermediateEntry intermEntry : task.getIntermediateData()) {
+              if (hashEntries.containsKey(intermEntry.getPartId())) {
+                Map<String, List<IntermediateEntry>> tbNameToInterm =
+                    hashEntries.get(intermEntry.getPartId());
+
+                if (tbNameToInterm.containsKey(scan.getCanonicalName())) {
+                  tbNameToInterm.get(scan.getCanonicalName()).add(intermEntry);
+                } else {
+                  tbNameToInterm.put(scan.getCanonicalName(), TUtil.newList(intermEntry));
+                }
+              } else {
+                Map<String, List<IntermediateEntry>> tbNameToInterm =
+                    new HashMap<String, List<IntermediateEntry>>();
+                tbNameToInterm.put(scan.getCanonicalName(), TUtil.newList(intermEntry));
+                hashEntries.put(intermEntry.getPartId(), tbNameToInterm);
+              }
+            }
+          } else {
+            //if no intermidatedata(empty table), make empty entry
+            int emptyPartitionId = 0;
+            if (hashEntries.containsKey(emptyPartitionId)) {
+              Map<String, List<IntermediateEntry>> tbNameToInterm = hashEntries.get(emptyPartitionId);
+              if (tbNameToInterm.containsKey(scan.getCanonicalName()))
+                tbNameToInterm.get(scan.getCanonicalName())
+                    .addAll(new ArrayList<IntermediateEntry>());
+              else
+                tbNameToInterm.put(scan.getCanonicalName(), new ArrayList<IntermediateEntry>());
+            } else {
+              Map<String, List<IntermediateEntry>> tbNameToInterm = new HashMap<String, List<IntermediateEntry>>();
+              tbNameToInterm.put(scan.getCanonicalName(), new ArrayList<IntermediateEntry>());
+              hashEntries.put(emptyPartitionId, tbNameToInterm);
+            }
+          }
+        }
+      }
+
+      // hashEntries can be zero if there are no input data.
+      // In the case, it will cause the zero divided exception.
+      // it avoids this problem.
+      int [] avgSize = new int[2];
+      avgSize[0] = hashEntries.size() == 0 ? 0 : (int) (stats[0] / hashEntries.size());
+      avgSize[1] = hashEntries.size() == 0 ? 0 : (int) (stats[1] / hashEntries.size());
+      int bothFetchSize = avgSize[0] + avgSize[1];
+
+      // Getting the desire number of join tasks according to the volumn
+      // of a larger table
+      int largerIdx = stats[0] >= stats[1] ? 0 : 1;
+      int desireJoinTaskVolumn = subQuery.getContext().getConf().
+          getIntVar(ConfVars.DIST_QUERY_JOIN_TASK_VOLUME);
+
+      // calculate the number of tasks according to the data size
+      int mb = (int) Math.ceil((double)stats[largerIdx] / 1048576);
+      LOG.info("Larger intermediate data is approximately " + mb + " MB");
+      // determine the number of task per 64MB
+      int maxTaskNum = (int) Math.ceil((double)mb / desireJoinTaskVolumn);
+      LOG.info("The calculated number of tasks is " + maxTaskNum);
+      LOG.info("The number of total shuffle keys is " + hashEntries.size());
+      // the number of join tasks cannot be larger than the number of
+      // distinct partition ids.
+      int joinTaskNum = Math.min(maxTaskNum, hashEntries.size());
+      LOG.info("The determined number of join tasks is " + joinTaskNum);
+
+      SubQuery.scheduleFragment(subQuery, fragments[0], Arrays.asList(new FileFragment[]{fragments[1]}));
+
+      // Assign partitions to tasks in a round robin manner.
+      for (Entry<Integer, Map<String, List<IntermediateEntry>>> entry
+          : hashEntries.entrySet()) {
+        addJoinShuffle(subQuery, entry.getKey(), entry.getValue());
+      }
+
+      schedulerContext.setTaskSize((int) Math.ceil((double) bothFetchSize / joinTaskNum));
+      schedulerContext.setEstimatedTaskNum(joinTaskNum);
+    }
+  }
+
+  /**
+   * It creates a number of fragments for all partitions.
+   */
+  public static List<FileFragment> getFragmentsFromPartitionedTable(AbstractStorageManager sm,
+                                                                          ScanNode scan,
+                                                                          TableDesc table) throws IOException {
+    List<FileFragment> fragments = Lists.newArrayList();
+    PartitionedTableScanNode partitionsScan = (PartitionedTableScanNode) scan;
+    fragments.addAll(sm.getSplits(
+        scan.getCanonicalName(), table.getMeta(), table.getSchema(), partitionsScan.getInputPaths()));
+    partitionsScan.setInputPaths(null);
+    return fragments;
+  }
+
+  private static void scheduleLeafTasksWithBroadcastTable(TaskSchedulerContext schedulerContext, SubQuery subQuery,
+                                                          int baseScanId, FileFragment[] fragments) throws IOException {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    ScanNode[] scans = execBlock.getScanNodes();
+    //Preconditions.checkArgument(scans.length == 2, "Must be Join Query");
+
+    for (int i = 0; i < scans.length; i++) {
+      if (i != baseScanId) {
+        scans[i].setBroadcastTable(true);
+      }
+    }
+
+    TableMeta meta;
+    ScanNode scan = scans[baseScanId];
+    TableDesc desc = subQuery.getContext().getTableDescMap().get(scan.getCanonicalName());
+    meta = desc.getMeta();
+
+    Collection<FileFragment> baseFragments;
+    if (scan.getType() == NodeType.PARTITIONS_SCAN) {
+      baseFragments = getFragmentsFromPartitionedTable(subQuery.getStorageManager(), scan, desc);
+    } else {
+      baseFragments = subQuery.getStorageManager().getSplits(scan.getCanonicalName(), meta, desc.getSchema(),
+          desc.getPath());
+    }
+
+    List<FileFragment> broadcastFragments = new ArrayList<FileFragment>();
+    for (int i = 0; i < fragments.length; i++) {
+      if (i != baseScanId) {
+        broadcastFragments.add(fragments[i]);
+      }
+    }
+    SubQuery.scheduleFragments(subQuery, baseFragments, broadcastFragments);
+    schedulerContext.setEstimatedTaskNum(baseFragments.size());
+  }
+
+  private static void addJoinShuffle(SubQuery subQuery, int partitionId,
+                                     Map<String, List<IntermediateEntry>> grouppedPartitions) {
+    Map<String, List<URI>> fetches = new HashMap<String, List<URI>>();
+    for (ExecutionBlock execBlock : subQuery.getMasterPlan().getChilds(subQuery.getId())) {
+      Map<String, List<IntermediateEntry>> requests;
+      if (grouppedPartitions.containsKey(execBlock.getId().toString())) {
+          requests = mergeHashShuffleRequest(grouppedPartitions.get(execBlock.getId().toString()));
+      } else {
+        return;
+      }
+      Set<URI> fetchURIs = TUtil.newHashSet();
+      for (Entry<String, List<IntermediateEntry>> requestPerNode : requests.entrySet()) {
+        Collection<URI> uris = createHashFetchURL(requestPerNode.getKey(),
+            execBlock.getId(),
+            partitionId, HASH_SHUFFLE,
+            requestPerNode.getValue());
+        fetchURIs.addAll(uris);
+      }
+      fetches.put(execBlock.getId().toString(), Lists.newArrayList(fetchURIs));
+    }
+    SubQuery.scheduleFetches(subQuery, fetches);
+  }
+
+  /**
+   * This method merges the partition request associated with the pullserver's address.
+   * It reduces the number of TCP connections.
+   *
+   * @return key: pullserver's address, value: a list of requests
+   */
+  private static Map<String, List<IntermediateEntry>> mergeHashShuffleRequest(List<IntermediateEntry> partitions) {
+    Map<String, List<IntermediateEntry>> mergedPartitions = new HashMap<String, List<IntermediateEntry>>();
+    for (IntermediateEntry partition : partitions) {
+      if (mergedPartitions.containsKey(partition.getPullAddress())) {
+        mergedPartitions.get(partition.getPullAddress()).add(partition);
+      } else {
+        mergedPartitions.put(partition.getPullAddress(), TUtil.newList(partition));
+      }
+    }
+
+    return mergedPartitions;
+  }
+
+  public static void scheduleFragmentsForNonLeafTasks(TaskSchedulerContext schedulerContext,
+                                                      MasterPlan masterPlan, SubQuery subQuery, int maxNum)
+      throws IOException {
+    DataChannel channel = masterPlan.getIncomingChannels(subQuery.getBlock().getId()).get(0);
+    if (channel.getShuffleType() == HASH_SHUFFLE) {
+      scheduleHashShuffledFetches(schedulerContext, masterPlan, subQuery, channel, maxNum);
+    } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
+      scheduleRangeShuffledFetches(schedulerContext, masterPlan, subQuery, channel, maxNum);
+    } else {
+      throw new InternalException("Cannot support partition type");
+    }
+  }
+
+  private static TableStats computeChildBlocksStats(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan,
+                                                    ExecutionBlockId parentBlockId) {
+    List<TableStats> tableStatses = new ArrayList<TableStats>();
+    List<ExecutionBlock> childBlocks = masterPlan.getChilds(parentBlockId);
+    for (ExecutionBlock childBlock : childBlocks) {
+      SubQuery childExecSM = context.getSubQuery(childBlock.getId());
+      tableStatses.add(childExecSM.getResultStats());
+    }
+    return StatisticsUtil.aggregateTableStat(tableStatses);
+  }
+
+  public static void scheduleRangeShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
+                                                  SubQuery subQuery, DataChannel channel, int maxNum)
+      throws IOException {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    ScanNode scan = execBlock.getScanNodes()[0];
+    Path tablePath;
+    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableName());
+
+    ExecutionBlock sampleChildBlock = masterPlan.getChild(subQuery.getId(), 0);
+    SortNode sortNode = PlannerUtil.findTopNode(sampleChildBlock.getPlan(), NodeType.SORT);
+    SortSpec [] sortSpecs = sortNode.getSortKeys();
+    Schema sortSchema = new Schema(channel.getShuffleKeys());
+
+    // calculate the number of maximum query ranges
+    TableStats totalStat = computeChildBlocksStats(subQuery.getContext(), masterPlan, subQuery.getId());
+
+    // If there is an empty table in inner join, it should return zero rows.
+    if (totalStat.getNumBytes() == 0 && totalStat.getColumnStats().size() == 0 ) {
+      return;
+    }
+    TupleRange mergedRange = TupleUtil.columnStatToRange(sortSpecs, sortSchema, totalStat.getColumnStats());
+    RangePartitionAlgorithm partitioner = new UniformRangePartition(mergedRange, sortSpecs);
+    BigDecimal card = partitioner.getTotalCardinality();
+
+    // if the number of the range cardinality is less than the desired number of tasks,
+    // we set the the number of tasks to the number of range cardinality.
+    int determinedTaskNum;
+    if (card.compareTo(new BigDecimal(maxNum)) < 0) {
+      LOG.info("The range cardinality (" + card
+          + ") is less then the desired number of tasks (" + maxNum + ")");
+      determinedTaskNum = card.intValue();
+    } else {
+      determinedTaskNum = maxNum;
+    }
+
+    LOG.info("Try to divide " + mergedRange + " into " + determinedTaskNum +
+        " sub ranges (total units: " + determinedTaskNum + ")");
+    TupleRange [] ranges = partitioner.partition(determinedTaskNum);
+
+    FileFragment dummyFragment = new FileFragment(scan.getTableName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+    SubQuery.scheduleFragment(subQuery, dummyFragment);
+
+    List<String> basicFetchURIs = new ArrayList<String>();
+    List<ExecutionBlock> childBlocks = masterPlan.getChilds(subQuery.getId());
+    for (ExecutionBlock childBlock : childBlocks) {
+      SubQuery childExecSM = subQuery.getContext().getSubQuery(childBlock.getId());
+      for (QueryUnit qu : childExecSM.getQueryUnits()) {
+        for (IntermediateEntry p : qu.getIntermediateData()) {
+          String uri = createBasicFetchUri(p.getPullHost(), p.getPullPort(), childBlock.getId(), p.taskId, p.attemptId);
+          basicFetchURIs.add(uri);
+        }
+      }
+    }
+
+    boolean ascendingFirstKey = sortSpecs[0].isAscending();
+    SortedMap<TupleRange, Collection<URI>> map;
+    if (ascendingFirstKey) {
+      map = new TreeMap<TupleRange, Collection<URI>>();
+    } else {
+      map = new TreeMap<TupleRange, Collection<URI>>(new TupleRange.DescendingTupleRangeComparator());
+    }
+
+    Set<URI> uris;
+    try {
+      RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(sortSchema);
+      for (int i = 0; i < ranges.length; i++) {
+        uris = new HashSet<URI>();
+        for (String uri: basicFetchURIs) {
+          String rangeParam =
+              TupleUtil.rangeToQuery(ranges[i], ascendingFirstKey ? i == (ranges.length - 1) : i == 0, encoder);
+          URI finalUri = URI.create(uri + "&" + rangeParam);
+          uris.add(finalUri);
+        }
+        map.put(ranges[i], uris);
+      }
+
+    } catch (UnsupportedEncodingException e) {
+      LOG.error(e);
+    }
+
+    scheduleFetchesByRoundRobin(subQuery, map, scan.getTableName(), determinedTaskNum);
+
+    schedulerContext.setEstimatedTaskNum(determinedTaskNum);
+  }
+
+  public static void scheduleFetchesByRoundRobin(SubQuery subQuery, Map<?, Collection<URI>> partitions,
+                                                   String tableName, int num) {
+    int i;
+    Map<String, List<URI>>[] fetchesArray = new Map[num];
+    for (i = 0; i < num; i++) {
+      fetchesArray[i] = new HashMap<String, List<URI>>();
+    }
+    i = 0;
+    for (Entry<?, Collection<URI>> entry : partitions.entrySet()) {
+      Collection<URI> value = entry.getValue();
+      TUtil.putCollectionToNestedList(fetchesArray[i++], tableName, value);
+      if (i == num) i = 0;
+    }
+    for (Map<String, List<URI>> eachFetches : fetchesArray) {
+      SubQuery.scheduleFetches(subQuery, eachFetches);
+    }
+  }
+
+  public static String createBasicFetchUri(String hostName, int port,
+                                           ExecutionBlockId childSid,
+                                           int taskId, int attemptId) {
+    String scheme = "http://";
+    StringBuilder sb = new StringBuilder(scheme);
+    sb.append(hostName).append(":").append(port).append("/?")
+        .append("qid=").append(childSid.getQueryId().toString())
+        .append("&sid=").append(childSid.getId())
+        .append("&").append("ta=").append(taskId).append("_").append(attemptId)
+        .append("&").append("p=0")
+        .append("&").append("type=r");
+
+    return sb.toString();
+  }
+
+  public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
+                                                 SubQuery subQuery, DataChannel channel,
+                                                 int maxNum) {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    TableStats totalStat = computeChildBlocksStats(subQuery.getContext(), masterPlan, subQuery.getId());
+
+    if (totalStat.getNumRows() == 0) {
+      return;
+    }
+
+    ScanNode scan = execBlock.getScanNodes()[0];
+    Path tablePath;
+    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableName());
+
+    FileFragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+    List<FileFragment> fragments = new ArrayList<FileFragment>();
+    fragments.add(frag);
+    SubQuery.scheduleFragments(subQuery, fragments);
+
+    Map<String, List<IntermediateEntry>> hashedByHost;
+    Map<Integer, Collection<URI>> finalFetchURI = new HashMap<Integer, Collection<URI>>();
+
+    for (ExecutionBlock block : masterPlan.getChilds(execBlock)) {
+      List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
+      for (QueryUnit tasks : subQuery.getContext().getSubQuery(block.getId()).getQueryUnits()) {
+        if (tasks.getIntermediateData() != null) {
+          partitions.addAll(tasks.getIntermediateData());
+        }
+      }
+      Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);
+      for (Entry<Integer, List<IntermediateEntry>> interm : hashed.entrySet()) {
+        hashedByHost = hashByHost(interm.getValue());
+        for (Entry<String, List<IntermediateEntry>> e : hashedByHost.entrySet()) {
+          Collection<URI> uris = createHashFetchURL(e.getKey(), block.getId(),
+              interm.getKey(), channel.getShuffleType(), e.getValue());
+
+          if (finalFetchURI.containsKey(interm.getKey())) {
+            finalFetchURI.get(interm.getKey()).addAll(uris);
+          } else {
+            finalFetchURI.put(interm.getKey(), TUtil.newList(uris));
+          }
+        }
+      }
+    }
+
+    GroupbyNode groupby = PlannerUtil.findMostBottomNode(subQuery.getBlock().getPlan(), NodeType.GROUP_BY);
+    // get a proper number of tasks
+    int determinedTaskNum = Math.min(maxNum, finalFetchURI.size());
+    LOG.info(subQuery.getId() + ", ScheduleHashShuffledFetches - Max num=" + maxNum + ", finalFetchURI=" + finalFetchURI.size());
+    if (groupby != null && groupby.getGroupingColumns().length == 0) {
+      determinedTaskNum = 1;
+      LOG.info(subQuery.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
+    }
+
+    // set the proper number of tasks to the estimated task num
+    schedulerContext.setEstimatedTaskNum(determinedTaskNum);
+    // divide fetch uris into the the proper number of tasks in a round robin manner.
+    scheduleFetchesByRoundRobin(subQuery, finalFetchURI, scan.getTableName(), determinedTaskNum);
+    LOG.info(subQuery.getId() + ", DeterminedTaskNum : " + determinedTaskNum);
+  }
+
+  public static Collection<URI> createHashFetchURL(String hostAndPort, ExecutionBlockId ebid,
+                                       int partitionId, ShuffleType type, List<IntermediateEntry> entries) {
+    String scheme = "http://";
+    StringBuilder urlPrefix = new StringBuilder(scheme);
+    urlPrefix.append(hostAndPort).append("/?")
+        .append("qid=").append(ebid.getQueryId().toString())
+        .append("&sid=").append(ebid.getId())
+        .append("&p=").append(partitionId)
+        .append("&type=");
+    if (type == HASH_SHUFFLE) {
+      urlPrefix.append("h");
+    } else if (type == RANGE_SHUFFLE) {
+      urlPrefix.append("r");
+    }
+    urlPrefix.append("&ta=");
+
+    // If the get request is longer than 2000 characters,
+    // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long.
+    // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15
+    // The below code transforms a long request to multiple requests.
+    List<String> taskIdsParams = new ArrayList<String>();
+    boolean first = true;
+    StringBuilder taskIdListBuilder = new StringBuilder();
+    for (IntermediateEntry entry: entries) {
+      StringBuilder taskAttemptId = new StringBuilder();
+
+      if (!first) { // when comma is added?
+        taskAttemptId.append(",");
+      } else {
+        first = false;
+      }
+
+      taskAttemptId.append(entry.getTaskId()).append("_").
+          append(entry.getAttemptId());
+      if (taskIdListBuilder.length() + taskAttemptId.length()
+          > HTTP_REQUEST_MAXIMUM_LENGTH) {
+        taskIdsParams.add(taskIdListBuilder.toString());
+        taskIdListBuilder = new StringBuilder(entry.getTaskId() + "_" + entry.getAttemptId());
+      } else {
+        taskIdListBuilder.append(taskAttemptId);
+      }
+    }
+
+    // if the url params remain
+    if (taskIdListBuilder.length() > 0) {
+      taskIdsParams.add(taskIdListBuilder.toString());
+    }
+
+    Collection<URI> fetchURLs = new ArrayList<URI>();
+    for (String param : taskIdsParams) {
+      fetchURLs.add(URI.create(urlPrefix + param));
+    }
+
+    return fetchURLs;
+  }
+
+  public static Map<Integer, List<IntermediateEntry>> hashByKey(List<IntermediateEntry> entries) {
+    Map<Integer, List<IntermediateEntry>> hashed = new HashMap<Integer, List<IntermediateEntry>>();
+    for (IntermediateEntry entry : entries) {
+      if (hashed.containsKey(entry.getPartId())) {
+        hashed.get(entry.getPartId()).add(entry);
+      } else {
+        hashed.put(entry.getPartId(), TUtil.newList(entry));
+      }
+    }
+
+    return hashed;
+  }
+
+  public static Map<String, List<IntermediateEntry>> hashByHost(List<IntermediateEntry> entries) {
+    Map<String, List<IntermediateEntry>> hashed = new HashMap<String, List<IntermediateEntry>>();
+
+    String hostName;
+    for (IntermediateEntry entry : entries) {
+      hostName = entry.getPullHost() + ":" + entry.getPullPort();
+      if (hashed.containsKey(hostName)) {
+        hashed.get(hostName).add(entry);
+      } else {
+        hashed.put(hostName, TUtil.newList(entry));
+      }
+    }
+
+    return hashed;
+  }
+
+  public static SubQuery setShuffleOutputNumForTwoPhase(SubQuery subQuery, final int desiredNum, DataChannel channel) {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    Column[] keys;
+    // if the next query is join,
+    // set the partition number for the current logicalUnit
+    // TODO: the union handling is required when a join has unions as its child
+    MasterPlan masterPlan = subQuery.getMasterPlan();
+    keys = channel.getShuffleKeys();
+    if (!masterPlan.isRoot(subQuery.getBlock()) ) {
+      ExecutionBlock parentBlock = masterPlan.getParent(subQuery.getBlock());
+      if (parentBlock.getPlan().getType() == NodeType.JOIN) {
+        channel.setShuffleOutputNum(desiredNum);
+      }
+    }
+
+    // set the partition number for group by and sort
+    if (channel.getShuffleType() == HASH_SHUFFLE) {
+      if (execBlock.getPlan().getType() == NodeType.GROUP_BY) {
+        keys = channel.getShuffleKeys();
+      }
+    } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
+      if (execBlock.getPlan().getType() == NodeType.SORT) {
+        SortNode sort = (SortNode) execBlock.getPlan();
+        keys = new Column[sort.getSortKeys().length];
+        for (int i = 0; i < keys.length; i++) {
+          keys[i] = sort.getSortKeys()[i].getSortKey();
+        }
+      }
+    }
+    if (keys != null) {
+      if (keys.length == 0) {
+        channel.setShuffleKeys(new Column[]{});
+        channel.setShuffleOutputNum(1);
+      } else {
+        channel.setShuffleKeys(keys);
+        channel.setShuffleOutputNum(desiredNum);
+      }
+    }
+    return subQuery;
+  }
+}


[41/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
new file mode 100644
index 0000000..cb356f8
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -0,0 +1,1516 @@
+/**
+ * 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.parser;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.misc.NotNull;
+import org.antlr.v4.runtime.tree.TerminalNode;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.algebra.Aggregation.GroupType;
+import org.apache.tajo.algebra.LiteralValue.LiteralType;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.engine.parser.SQLParser.*;
+import org.apache.tajo.storage.StorageConstants;
+
+import java.nio.charset.Charset;
+import java.util.*;
+
+import static org.apache.tajo.algebra.Aggregation.GroupElement;
+import static org.apache.tajo.algebra.CreateTable.*;
+import static org.apache.tajo.common.TajoDataTypes.Type;
+import static org.apache.tajo.engine.parser.SQLParser.*;
+
+public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
+  private SQLParser parser;
+
+  public SQLAnalyzer() {
+  }
+
+  public Expr parse(String sql) {
+    ANTLRInputStream input = new ANTLRInputStream(sql);
+    SQLLexer lexer = new SQLLexer(input);
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    this.parser = new SQLParser(tokens);
+    parser.setBuildParseTree(true);
+    parser.removeErrorListeners();
+
+    parser.setErrorHandler(new SQLErrorStrategy());
+    parser.addErrorListener(new SQLErrorListener());
+
+    SqlContext context;
+    try {
+      context = parser.sql();
+    } catch (SQLParseError e) {
+      throw new SQLSyntaxError(e);
+    }
+    return visitSql(context);
+  }
+
+  private static boolean checkIfExist(Object obj) {
+    return obj != null;
+  }
+
+  @Override
+  public Expr visitSql(SqlContext ctx) {
+    Expr statement = visit(ctx.statement());
+    if (checkIfExist(ctx.explain_clause())) {
+      return new Explain(statement);
+    } else {
+      return statement;
+    }
+  }
+
+  @Override
+  public Expr visitNon_join_query_expression(SQLParser.Non_join_query_expressionContext ctx) {
+
+    Expr current = visitNon_join_query_term(ctx.non_join_query_term());
+    if (ctx.getChildCount() == 1) {
+      return current;
+    }
+
+    OpType operatorType;
+    Expr left;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      int idx = i;
+      boolean distinct = true;
+
+      if (ctx.getChild(idx) instanceof TerminalNode) {
+        if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == UNION) {
+          operatorType = OpType.Union;
+        } else {
+          operatorType = OpType.Except;
+        }
+
+        idx++;
+
+        if (ctx.getChild(idx) instanceof TerminalNode) {
+          if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == ALL) {
+            distinct = false;
+          }
+
+          idx++;
+        }
+
+        SQLParser.Query_termContext queryTermContext =
+            (SQLParser.Query_termContext) ctx.getChild(idx);
+        Expr right = visitQuery_term(queryTermContext);
+
+        left = current;
+        current = new SetOperation(operatorType, left, right, distinct);
+
+        i = idx;
+      }
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitNon_join_query_term(Non_join_query_termContext ctx) {
+
+    Expr current = visitNon_join_query_primary(ctx.non_join_query_primary());
+    Expr left;
+
+    for (int i = 1; i < ctx.getChildCount(); ) {
+      int idx = i;
+      boolean distinct = true;
+
+      if (ctx.getChild(idx) instanceof TerminalNode) {
+        if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == INTERSECT) {
+          idx++;
+        }
+
+        if (ctx.getChild(idx) instanceof TerminalNode) {
+          if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == ALL) {
+            distinct = false;
+            idx++;
+          }
+        }
+
+        Query_primaryContext queryPrimaryContext = (Query_primaryContext) ctx.getChild(idx);
+        Expr right = visitQuery_primary(queryPrimaryContext);
+
+        left = current;
+        current = new SetOperation(OpType.Intersect, left, right, distinct);
+
+        i += idx;
+      }
+    }
+
+    return current;
+
+  }
+
+  @Override
+  public Expr visitQuery_specification(SQLParser.Query_specificationContext ctx) {
+    Expr current = null;
+    if (ctx.table_expression() != null) {
+      current = visitFrom_clause(ctx.table_expression().from_clause());
+
+      if (ctx.table_expression().where_clause() != null) {
+        Selection selection = visitWhere_clause(ctx.table_expression().where_clause());
+        selection.setChild(current);
+        current = selection;
+      }
+
+      if (ctx.table_expression().groupby_clause() != null) {
+        Aggregation aggregation = visitGroupby_clause(ctx.table_expression().groupby_clause());
+        aggregation.setChild(current);
+        current = aggregation;
+
+        if (ctx.table_expression().having_clause() != null) {
+          Expr havingCondition = visitBoolean_value_expression(
+              ctx.table_expression().having_clause().boolean_value_expression());
+          Having having = new Having(havingCondition);
+          having.setChild(current);
+          current = having;
+        }
+      }
+
+      if (ctx.table_expression().orderby_clause() != null) {
+        Sort sort = visitOrderby_clause(ctx.table_expression().orderby_clause());
+        sort.setChild(current);
+        current = sort;
+      }
+
+      if (ctx.table_expression().limit_clause() != null) {
+        Limit limit = visitLimit_clause(ctx.table_expression().limit_clause());
+        limit.setChild(current);
+        current = limit;
+      }
+    }
+
+    Projection projection = visitSelect_list(ctx.select_list());
+
+    if (ctx.set_qualifier() != null && ctx.set_qualifier().DISTINCT() != null) {
+      projection.setDistinct();
+    }
+
+    if (current != null) {
+      projection.setChild(current);
+    }
+
+    current = projection;
+
+    return current;
+  }
+
+  /**
+   * <pre>
+   *   select_list
+   *   : select_sublist (COMMA select_sublist)*
+   *   ;
+   * </pre>
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Projection visitSelect_list(SQLParser.Select_listContext ctx) {
+    Projection projection = new Projection();
+    NamedExpr[] targets = new NamedExpr[ctx.select_sublist().size()];
+    for (int i = 0; i < targets.length; i++) {
+      targets[i] = visitSelect_sublist(ctx.select_sublist(i));
+    }
+    projection.setNamedExprs(targets);
+
+    return projection;
+  }
+
+  /**
+   * <pre>
+   *   select_sublist
+   *   : derived_column
+   *   | asterisked_qualifier
+   *   ;
+   * </pre>
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public NamedExpr visitSelect_sublist(SQLParser.Select_sublistContext ctx) {
+    if (ctx.qualified_asterisk() != null) {
+      return visitQualified_asterisk(ctx.qualified_asterisk());
+    } else {
+      return visitDerived_column(ctx.derived_column());
+    }
+  }
+
+  @Override
+  public RelationList visitFrom_clause(SQLParser.From_clauseContext ctx) {
+    Expr[] relations = new Expr[ctx.table_reference_list().table_reference().size()];
+    for (int i = 0; i < relations.length; i++) {
+      relations[i] = visitTable_reference(ctx.table_reference_list().table_reference(i));
+    }
+    return new RelationList(relations);
+  }
+
+  @Override
+  public Selection visitWhere_clause(SQLParser.Where_clauseContext ctx) {
+    return new Selection(visitSearch_condition(ctx.search_condition()));
+  }
+
+  @Override
+  public Aggregation visitGroupby_clause(SQLParser.Groupby_clauseContext ctx) {
+    Aggregation clause = new Aggregation();
+
+    // If grouping group is not empty
+    if (ctx.grouping_element_list().grouping_element().get(0).empty_grouping_set() == null) {
+      int elementSize = ctx.grouping_element_list().grouping_element().size();
+      ArrayList<GroupElement> groups = new ArrayList<GroupElement>(elementSize + 1);
+      ArrayList<Expr> ordinaryExprs = null;
+      int groupSize = 1;
+      groups.add(null);
+
+      for (int i = 0; i < elementSize; i++) {
+        SQLParser.Grouping_elementContext element =
+            ctx.grouping_element_list().grouping_element().get(i);
+        if (element.ordinary_grouping_set() != null) {
+          if (ordinaryExprs == null) {
+            ordinaryExprs = new ArrayList<Expr>();
+          }
+          Collections.addAll(ordinaryExprs, getRowValuePredicandsFromOrdinaryGroupingSet(element.ordinary_grouping_set()));
+        } else if (element.rollup_list() != null) {
+          groupSize++;
+          groups.add(new GroupElement(GroupType.Rollup,
+              getRowValuePredicandsFromOrdinaryGroupingSetList(element.rollup_list().c)));
+        } else if (element.cube_list() != null) {
+          groupSize++;
+          groups.add(new GroupElement(GroupType.Cube,
+              getRowValuePredicandsFromOrdinaryGroupingSetList(element.cube_list().c)));
+        }
+      }
+
+      if (ordinaryExprs != null) {
+        groups.set(0, new GroupElement(GroupType.OrdinaryGroup, ordinaryExprs.toArray(new Expr[ordinaryExprs.size()])));
+        clause.setGroups(groups.subList(0, groupSize).toArray(new GroupElement[groupSize]));
+      } else if (groupSize > 1) {
+        clause.setGroups(groups.subList(1, groupSize).toArray(new GroupElement[groupSize - 1]));
+      }
+    }
+
+    return clause;
+  }
+
+  @Override
+  public Sort visitOrderby_clause(SQLParser.Orderby_clauseContext ctx) {
+    int size = ctx.sort_specifier_list().sort_specifier().size();
+    Sort.SortSpec specs[] = new Sort.SortSpec[size];
+    for (int i = 0; i < size; i++) {
+      SQLParser.Sort_specifierContext specContext = ctx.sort_specifier_list().sort_specifier(i);
+      Expr column = visitRow_value_predicand(specContext.key);
+      specs[i] = new Sort.SortSpec(column);
+      if (specContext.order_specification() != null) {
+        if (specContext.order.DESC() != null) {
+          specs[i].setDescending();
+        }
+      }
+
+      if (specContext.null_ordering() != null) {
+        if (specContext.null_ordering().FIRST() != null) {
+          specs[i].setNullFirst();
+        }
+      }
+    }
+
+    return new Sort(specs);
+  }
+
+  @Override
+  public Limit visitLimit_clause(SQLParser.Limit_clauseContext ctx) {
+    return new Limit(visitNumeric_value_expression(ctx.numeric_value_expression()));
+  }
+
+  @Override
+  public Expr visitJoined_table(SQLParser.Joined_tableContext ctx) {
+    Expr top = visitTable_primary(ctx.table_primary());
+
+    // The following loop builds a left deep join tree.
+    Join join;
+    for (int i = 0; i < ctx.joined_table_primary().size(); i++) {
+      join = visitJoined_table_primary(ctx.joined_table_primary(i));
+      join.setLeft(top);
+      top = join;
+    }
+
+    return top;
+  }
+
+  @Override
+  public Join visitJoined_table_primary(SQLParser.Joined_table_primaryContext ctx) {
+    Join join;
+    if (ctx.CROSS() != null) {
+      join = new Join(JoinType.CROSS);
+    } else if (ctx.UNION() != null) {
+      join = new Join(JoinType.UNION);
+    } else { // qualified join or natural
+      if (ctx.join_type() != null && ctx.join_type().outer_join_type() != null) {
+        Outer_join_type_part2Context outer_join_typeContext = ctx.join_type().outer_join_type()
+            .outer_join_type_part2();
+        if (outer_join_typeContext.FULL() != null) {
+          join = new Join(JoinType.FULL_OUTER);
+        } else if (outer_join_typeContext.LEFT() != null) {
+          join = new Join(JoinType.LEFT_OUTER);
+        } else {
+          join = new Join(JoinType.RIGHT_OUTER);
+        }
+      } else {
+        join = new Join(JoinType.INNER);
+      }
+
+      if (ctx.NATURAL() != null) {
+        join.setNatural();
+      }
+
+      if (ctx.join_specification() != null) { // only for qualified join
+        if (ctx.join_specification().join_condition() != null) {
+          Expr searchCondition = visitSearch_condition(ctx.join_specification().
+              join_condition().search_condition());
+          join.setQual(searchCondition);
+        } else if (ctx.join_specification().named_columns_join() != null) {
+          ColumnReferenceExpr[] columns = getColumnReferences(ctx.join_specification().
+              named_columns_join().column_reference_list());
+          join.setJoinColumns(columns);
+        }
+      }
+    }
+
+    join.setRight(visitTable_primary(ctx.right));
+    return join;
+  }
+
+  private Expr[] getRowValuePredicandsFromOrdinaryGroupingSetList(Ordinary_grouping_set_listContext ctx) {
+    ArrayList<Expr> rowValuePredicands = new ArrayList<Expr>();
+    for (int i = 0; i < ctx.ordinary_grouping_set().size(); i++) {
+      Collections.addAll(rowValuePredicands, getRowValuePredicandsFromOrdinaryGroupingSet(ctx.ordinary_grouping_set(i)));
+    }
+    return rowValuePredicands.toArray(new Expr[rowValuePredicands.size()]);
+  }
+
+  private Expr[] getRowValuePredicandsFromOrdinaryGroupingSet(Ordinary_grouping_setContext ctx) {
+    ArrayList<Expr> rowValuePredicands = new ArrayList<Expr>();
+    if (ctx.row_value_predicand() != null) {
+      rowValuePredicands.add(visitRow_value_predicand(ctx.row_value_predicand()));
+    }
+    if (ctx.row_value_predicand_list() != null) {
+      Collections.addAll(rowValuePredicands, getRowValuePredicands(ctx.row_value_predicand_list()));
+    }
+    return rowValuePredicands.toArray(new Expr[rowValuePredicands.size()]);
+  }
+
+  private Expr[] getRowValuePredicands(Row_value_predicand_listContext ctx) {
+    Expr[] rowValuePredicands = new Expr[ctx.row_value_predicand().size()];
+    for (int i = 0; i < rowValuePredicands.length; i++) {
+      rowValuePredicands[i] = visitRow_value_predicand(ctx.row_value_predicand(i));
+    }
+    return rowValuePredicands;
+  }
+
+  private ColumnReferenceExpr[] getColumnReferences(Column_reference_listContext ctx) {
+    ColumnReferenceExpr[] columnRefs = new ColumnReferenceExpr[ctx.column_reference().size()];
+    for (int i = 0; i < columnRefs.length; i++) {
+      columnRefs[i] = visitColumn_reference(ctx.column_reference(i));
+    }
+    return columnRefs;
+  }
+
+  @Override
+  public Expr visitTable_primary(SQLParser.Table_primaryContext ctx) {
+    if (ctx.table_or_query_name() != null) {
+      Relation relation = new Relation(ctx.table_or_query_name().getText());
+      if (ctx.alias != null) {
+        relation.setAlias(ctx.alias.getText());
+      }
+      return relation;
+    } else if (ctx.derived_table() != null) {
+      return new TablePrimarySubQuery(ctx.name.getText(), visit(ctx.derived_table().table_subquery()));
+    } else {
+      return null;
+    }
+  }
+
+
+  @Override
+  public Expr visitSubquery(SQLParser.SubqueryContext ctx) {
+    return visitQuery_expression(ctx.query_expression());
+  }
+
+  @Override
+  public BetweenPredicate visitBetween_predicate(SQLParser.Between_predicateContext ctx) {
+    Expr predicand = visitRow_value_predicand(ctx.predicand);
+    Expr begin = visitRow_value_predicand(ctx.between_predicate_part_2().begin);
+    Expr end = visitRow_value_predicand(ctx.between_predicate_part_2().end);
+    return new BetweenPredicate(checkIfExist(ctx.between_predicate_part_2().NOT()),
+        checkIfExist(ctx.between_predicate_part_2().SYMMETRIC()), predicand, begin, end);
+  }
+
+  @Override
+  public CaseWhenPredicate visitSimple_case(SQLParser.Simple_caseContext ctx) {
+    Expr leftTerm = visitBoolean_value_expression(ctx.boolean_value_expression());
+    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
+
+    for (int i = 0; i < ctx.simple_when_clause().size(); i++) {
+      Simple_when_clauseContext simpleWhenCtx = ctx.simple_when_clause(i);
+      BinaryOperator bin = new BinaryOperator(OpType.Equals, leftTerm,
+          visitValue_expression(simpleWhenCtx.search_condition().value_expression()));
+      caseWhen.addWhen(bin, buildCaseResult(simpleWhenCtx.result()));
+    }
+    if (ctx.else_clause() != null) {
+      caseWhen.setElseResult(buildCaseResult(ctx.else_clause().result()));
+    }
+    return caseWhen;
+  }
+
+  private Expr buildCaseResult(ResultContext result) {
+    if (result.NULL() != null) {
+      return new NullLiteral();
+    } else {
+      return visitValue_expression(result.value_expression());
+    }
+  }
+
+  @Override
+  public CaseWhenPredicate visitSearched_case(SQLParser.Searched_caseContext ctx) {
+    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
+
+    for (int i = 0; i < ctx.searched_when_clause().size(); i++) {
+      Searched_when_clauseContext searchedWhenCtx = ctx.searched_when_clause(i);
+      caseWhen.addWhen(
+          visitSearch_condition(searchedWhenCtx.search_condition()),
+          buildCaseResult(searchedWhenCtx.result()));
+    }
+    if (ctx.else_clause() != null) {
+      caseWhen.setElseResult(buildCaseResult(ctx.else_clause().result()));
+    }
+    return caseWhen;
+  }
+
+  @Override
+  public Expr visitCommon_value_expression(SQLParser.Common_value_expressionContext ctx) {
+    if (checkIfExist(ctx.NULL())) {
+      return new NullLiteral();
+    } else {
+      return visitChildren(ctx);
+    }
+  }
+
+  @Override
+  public Expr visitParenthesized_value_expression(SQLParser.Parenthesized_value_expressionContext ctx) {
+    return visitValue_expression(ctx.value_expression());
+  }
+
+  @Override
+  public Expr visitBoolean_value_expression(SQLParser.Boolean_value_expressionContext ctx) {
+    Expr current = visitOr_predicate(ctx.or_predicate());
+    return current;
+  }
+
+  @Override
+  public Expr visitOr_predicate(SQLParser.Or_predicateContext ctx) {
+    Expr current = visitAnd_predicate(ctx.and_predicate());
+
+    Expr left;
+    Expr right;
+    for (int i = 0; i < ctx.or_predicate().size(); i++) {
+      left = current;
+      right = visitOr_predicate(ctx.or_predicate(i));
+      current = new BinaryOperator(OpType.Or, left, right);
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitAnd_predicate(SQLParser.And_predicateContext ctx) {
+    Expr current = visitBoolean_factor(ctx.boolean_factor());
+
+    Expr left;
+    Expr right;
+    for (int i = 0; i < ctx.and_predicate().size(); i++) {
+      left = current;
+      right = visitAnd_predicate(ctx.and_predicate(i));
+      current = new BinaryOperator(OpType.And, left, right);
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitBoolean_factor(SQLParser.Boolean_factorContext ctx) {
+    if (ctx.NOT() != null) {
+      return new NotExpr(visitBoolean_test(ctx.boolean_test()));
+    } else {
+      return visitBoolean_test(ctx.boolean_test());
+    }
+  }
+
+  @Override
+  public Expr visitBoolean_test(SQLParser.Boolean_testContext ctx) {
+    if (checkIfExist(ctx.is_clause())) {
+      Is_clauseContext isClauseContext = ctx.is_clause();
+      if (checkIfExist(isClauseContext.NOT())) {
+        if (checkIfExist(ctx.is_clause().truth_value().TRUE())) {
+          return new NotExpr(visitBoolean_primary(ctx.boolean_primary()));
+        } else {
+          return visitBoolean_primary(ctx.boolean_primary());
+        }
+      } else {
+        if (checkIfExist(ctx.is_clause().truth_value().TRUE())) {
+          return visitBoolean_primary(ctx.boolean_primary());
+        } else {
+          return new NotExpr(visitBoolean_primary(ctx.boolean_primary()));
+        }
+      }
+    } else {
+      return visitBoolean_primary(ctx.boolean_primary());
+    }
+  }
+
+  @Override
+  public Expr visitBoolean_primary(SQLParser.Boolean_primaryContext ctx) {
+    if (ctx.predicate() != null) {
+      return visitPredicate(ctx.predicate());
+    } else {
+      return visitBoolean_predicand(ctx.boolean_predicand());
+    }
+  }
+
+  @Override
+  public Expr visitBoolean_predicand(SQLParser.Boolean_predicandContext ctx) {
+    if (checkIfExist(ctx.nonparenthesized_value_expression_primary())) {
+      return visitNonparenthesized_value_expression_primary(ctx.nonparenthesized_value_expression_primary());
+    } else {
+      return visitBoolean_value_expression(ctx.parenthesized_boolean_value_expression().boolean_value_expression());
+    }
+  }
+
+  @Override
+  public Expr visitNonparenthesized_value_expression_primary(
+      SQLParser.Nonparenthesized_value_expression_primaryContext ctx) {
+    return visitChildren(ctx);
+  }
+
+  @Override
+  public Expr visitRow_value_predicand(SQLParser.Row_value_predicandContext ctx) {
+    if (checkIfExist(ctx.row_value_special_case())) {
+      return visitRow_value_special_case(ctx.row_value_special_case());
+    } else {
+      return visitRow_value_constructor_predicand(ctx.row_value_constructor_predicand());
+    }
+  }
+
+  @Override
+  public Expr visitRow_value_constructor_predicand(SQLParser.Row_value_constructor_predicandContext ctx) {
+    if (checkIfExist(ctx.boolean_predicand())) {
+      return visitBoolean_predicand(ctx.boolean_predicand());
+    } else {
+      return visitCommon_value_expression(ctx.common_value_expression());
+    }
+  }
+
+  @Override
+  public BinaryOperator visitComparison_predicate(SQLParser.Comparison_predicateContext ctx) {
+    TerminalNode operator = (TerminalNode) ctx.comp_op().getChild(0);
+    return new BinaryOperator(tokenToExprType(operator.getSymbol().getType()),
+        visitRow_value_predicand(ctx.left),
+        visitRow_value_predicand(ctx.right));
+  }
+
+  @Override
+  public Expr visitNumeric_value_expression(SQLParser.Numeric_value_expressionContext ctx) {
+    Expr current = visitTerm(ctx.term(0));
+
+    Expr left;
+    Expr right;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      left = current;
+      TerminalNode operator = (TerminalNode) ctx.getChild(i++);
+      right = visitTerm((TermContext) ctx.getChild(i));
+
+      if (operator.getSymbol().getType() == PLUS) {
+        current = new BinaryOperator(OpType.Plus, left, right);
+      } else {
+        current = new BinaryOperator(OpType.Minus, left, right);
+      }
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitTerm(SQLParser.TermContext ctx) {
+    Expr current = visitFactor(ctx.factor(0));
+
+    Expr left;
+    Expr right;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      left = current;
+      TerminalNode operator = (TerminalNode) ctx.getChild(i++);
+      right = visitFactor((FactorContext) ctx.getChild(i));
+
+      if (operator.getSymbol().getType() == MULTIPLY) {
+        current = new BinaryOperator(OpType.Multiply, left, right);
+      } else if (operator.getSymbol().getType() == DIVIDE) {
+        current = new BinaryOperator(OpType.Divide, left, right);
+      } else {
+        current = new BinaryOperator(OpType.Modular, left, right);
+      }
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitFactor(SQLParser.FactorContext ctx) {
+    Expr current = visitNumeric_primary(ctx.numeric_primary());
+    if (checkIfExist(ctx.sign()) && checkIfExist(ctx.sign().MINUS())) {
+      current = new SignedExpr(true, current);
+    }
+    return current;
+  }
+
+  @Override
+  public Expr visitNumeric_primary(SQLParser.Numeric_primaryContext ctx) {
+    Expr current = null;
+    if (checkIfExist(ctx.value_expression_primary())) {
+      current = visitValue_expression_primary(ctx.value_expression_primary());
+      for (int i = 0; i < ctx.CAST_EXPRESSION().size(); i++) {
+        current = new CastExpr(current, visitData_type(ctx.cast_target(i).data_type()));
+      }
+    } else if (checkIfExist(ctx.numeric_value_function())) {
+      current = visitNumeric_value_function(ctx.numeric_value_function());
+    }
+
+    return current;
+  }
+
+  public static OpType tokenToExprType(int tokenId) {
+    switch (tokenId) {
+      case SQLParser.UNION:
+        return OpType.Union;
+      case SQLParser.EXCEPT:
+        return OpType.Except;
+      case SQLParser.INTERSECT:
+        return OpType.Intersect;
+
+      case SQLParser.AND:
+        return OpType.And;
+      case SQLParser.OR:
+        return OpType.Or;
+
+      case SQLParser.EQUAL:
+        return OpType.Equals;
+      case SQLParser.NOT_EQUAL:
+        return OpType.NotEquals;
+      case SQLParser.LTH:
+        return OpType.LessThan;
+      case SQLParser.LEQ:
+        return OpType.LessThanOrEquals;
+      case SQLParser.GTH:
+        return OpType.GreaterThan;
+      case SQLParser.GEQ:
+        return OpType.GreaterThanOrEquals;
+
+      case SQLParser.MULTIPLY:
+        return OpType.Multiply;
+      case SQLParser.DIVIDE:
+        return OpType.Divide;
+      case SQLParser.MODULAR:
+        return OpType.Modular;
+      case SQLParser.PLUS:
+        return OpType.Plus;
+      case SQLParser.MINUS:
+        return OpType.Minus;
+
+      default:
+        throw new RuntimeException("Unknown Token Id: " + tokenId);
+    }
+  }
+
+  @Override
+  public InPredicate visitIn_predicate(SQLParser.In_predicateContext ctx) {
+    return new InPredicate(visitChildren(ctx.numeric_value_expression()),
+        visitIn_predicate_value(ctx.in_predicate_value()), ctx.NOT() != null);
+  }
+
+  @Override
+  public Expr visitIn_predicate_value(SQLParser.In_predicate_valueContext ctx) {
+    if (checkIfExist(ctx.in_value_list())) {
+      int size = ctx.in_value_list().row_value_predicand().size();
+      Expr [] exprs = new Expr[size];
+      for (int i = 0; i < size; i++) {
+        exprs[i] = visitRow_value_predicand(ctx.in_value_list().row_value_predicand(i));
+      }
+      return new ValueListExpr(exprs);
+    } else {
+      return new SimpleTableSubQuery(visitChildren(ctx.table_subquery()));
+    }
+  }
+
+  @Override
+  public Expr visitArray(SQLParser.ArrayContext ctx) {
+    int size = ctx.numeric_value_expression().size();
+    Expr[] exprs = new Expr[size];
+    for (int i = 0; i < size; i++) {
+      exprs[i] = visit(ctx.numeric_value_expression(i));
+    }
+    return new ValueListExpr(exprs);
+  }
+
+  @Override
+  public Expr visitPattern_matching_predicate(SQLParser.Pattern_matching_predicateContext ctx) {
+    Expr predicand = visitChildren(ctx.row_value_predicand());
+    Expr pattern = new LiteralValue(stripQuote(ctx.Character_String_Literal().getText()),
+        LiteralType.String);
+
+    if (checkIfExist(ctx.pattern_matcher().negativable_matcher())) {
+      boolean not = ctx.pattern_matcher().NOT() != null;
+      Negativable_matcherContext matcher = ctx.pattern_matcher().negativable_matcher();
+      if (checkIfExist(matcher.LIKE())) {
+        return new PatternMatchPredicate(OpType.LikePredicate, not, predicand, pattern);
+      } else if (checkIfExist(matcher.ILIKE())) {
+        return new PatternMatchPredicate(OpType.LikePredicate, not, predicand, pattern, true);
+      } else if (checkIfExist(matcher.SIMILAR())) {
+        return new PatternMatchPredicate(OpType.SimilarToPredicate, not, predicand, pattern);
+      } else if (checkIfExist(matcher.REGEXP()) || checkIfExist(matcher.RLIKE())) {
+        return new PatternMatchPredicate(OpType.Regexp, not, predicand, pattern);
+      } else {
+        throw new SQLSyntaxError("Unsupported predicate: " + matcher.getText());
+      }
+    } else if (checkIfExist(ctx.pattern_matcher().regex_matcher())) {
+      Regex_matcherContext matcher = ctx.pattern_matcher().regex_matcher();
+      if (checkIfExist(matcher.Similar_To())) {
+        return new PatternMatchPredicate(OpType.Regexp, false, predicand, pattern, false);
+      } else if (checkIfExist(matcher.Not_Similar_To())) {
+        return new PatternMatchPredicate(OpType.Regexp, true, predicand, pattern, false);
+      } else if (checkIfExist(matcher.Similar_To_Case_Insensitive())) {
+        return new PatternMatchPredicate(OpType.Regexp, false, predicand, pattern, true);
+      } else if (checkIfExist(matcher.Not_Similar_To_Case_Insensitive())) {
+        return new PatternMatchPredicate(OpType.Regexp, true, predicand, pattern, true);
+      } else {
+        throw new SQLSyntaxError("Unsupported predicate: " + matcher.getText());
+      }
+    } else {
+      throw new SQLSyntaxError("Unsupported predicate: " + ctx.pattern_matcher().getText());
+    }
+  }
+
+  @Override
+  public IsNullPredicate visitNull_predicate(SQLParser.Null_predicateContext ctx) {
+    Expr predicand = visitRow_value_predicand(ctx.row_value_predicand());
+    return new IsNullPredicate(ctx.NOT() != null, predicand);
+  }
+
+  @Override
+  public ExistsPredicate visitExists_predicate(SQLParser.Exists_predicateContext ctx) {
+    return new ExistsPredicate(new SimpleTableSubQuery(visitTable_subquery(ctx.table_subquery())), ctx.NOT() != null);
+  }
+
+  @Override
+  public ColumnReferenceExpr visitColumn_reference(SQLParser.Column_referenceContext ctx) {
+    ColumnReferenceExpr column = new ColumnReferenceExpr(ctx.name.getText());
+    if (checkIfExist(ctx.db_name)) {
+      column.setQualifier(CatalogUtil.buildFQName(ctx.db_name.getText(), ctx.tb_name.getText()));
+    } else if (ctx.tb_name != null) {
+      column.setQualifier(ctx.tb_name.getText());
+    }
+
+    return column;
+  }
+
+  @Override
+  public LiteralValue visitUnsigned_numeric_literal(@NotNull SQLParser.Unsigned_numeric_literalContext ctx) {
+    if (ctx.NUMBER() != null) {
+      long lValue = Long.parseLong(ctx.getText());
+      if (lValue >= Integer.MIN_VALUE && lValue <= Integer.MAX_VALUE) {
+        return new LiteralValue(ctx.getText(), LiteralType.Unsigned_Integer);
+      } else {
+        return new LiteralValue(ctx.getText(), LiteralType.Unsigned_Large_Integer);
+      }
+    } else {
+      return new LiteralValue(ctx.getText(), LiteralType.Unsigned_Float);
+    }
+  }
+
+  @Override
+  public FunctionExpr visitAggregate_function(SQLParser.Aggregate_functionContext ctx) {
+    if (ctx.COUNT() != null && ctx.MULTIPLY() != null) {
+      return new CountRowsFunctionExpr();
+    } else {
+      return visitGeneral_set_function(ctx.general_set_function());
+    }
+  }
+
+  @Override
+  public FunctionExpr visitGeneral_set_function(SQLParser.General_set_functionContext ctx) {
+    String signature = ctx.set_function_type().getText();
+    boolean distinct = checkIfExist(ctx.set_qualifier()) && checkIfExist(ctx.set_qualifier().DISTINCT());
+    Expr param = visitValue_expression(ctx.value_expression());
+
+    return new GeneralSetFunctionExpr(signature, distinct, param);
+  }
+
+  @Override
+  public FunctionExpr visitRoutine_invocation(SQLParser.Routine_invocationContext ctx) {
+    String signature = ctx.function_name().getText();
+    FunctionExpr function = new FunctionExpr(signature);
+    if (ctx.sql_argument_list() != null) {
+      int numArgs = ctx.sql_argument_list().value_expression().size();
+      Expr[] argument_list = new Expr[numArgs];
+      for (int i = 0; i < numArgs; i++) {
+        argument_list[i] = visitValue_expression(ctx.sql_argument_list().
+            value_expression().get(i));
+      }
+
+      function.setParams(argument_list);
+    }
+    return function;
+  }
+
+  @Override
+  public NamedExpr visitDerived_column(Derived_columnContext ctx) {
+    NamedExpr target = new NamedExpr(visitValue_expression(ctx.value_expression()));
+    if (ctx.as_clause() != null) {
+      target.setAlias(ctx.as_clause().identifier().getText());
+    }
+    return target;
+  }
+
+  @Override
+  public NamedExpr visitQualified_asterisk(Qualified_asteriskContext ctx) {
+    QualifiedAsteriskExpr target = new QualifiedAsteriskExpr();
+    if (ctx.tb_name != null) {
+      target.setQualifier(ctx.tb_name.getText());
+    }
+
+    return new NamedExpr(target);
+  }
+
+  @Override
+  public Expr visitCharacter_string_type(SQLParser.Character_string_typeContext ctx) {
+    return new LiteralValue(stripQuote(ctx.getText()), LiteralType.String);
+  }
+
+  @Override
+  public Expr visitCharacter_value_expression(SQLParser.Character_value_expressionContext ctx) {
+    Expr current = visitCharacter_factor(ctx.character_factor(0));
+
+    Expr left;
+    Expr right;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      left = current;
+      i++; // skip '||' operator
+      right = visitCharacter_factor((Character_factorContext) ctx.getChild(i));
+
+      if (left.getType() == OpType.Literal && right.getType() == OpType.Literal) {
+        current = new LiteralValue(((LiteralValue) left).getValue() + ((LiteralValue) right).getValue(),
+            LiteralType.String);
+      } else {
+        current = new BinaryOperator(OpType.Concatenate, left, right);
+      }
+    }
+
+    return current;
+  }
+
+
+  @Override
+  public Expr visitNumeric_value_function(Numeric_value_functionContext ctx) {
+    if (checkIfExist(ctx.extract_expression())) {
+      return visitExtract_expression(ctx.extract_expression());
+    }
+
+    return null;
+  }
+
+  @Override
+  public Expr visitExtract_expression(Extract_expressionContext ctx) {
+    Expr extractTarget = new LiteralValue(ctx.extract_field_string.getText(), LiteralType.String);
+    Expr extractSource;
+    if (checkIfExist(ctx.extract_source().column_reference())) {
+      extractSource = visitColumn_reference(ctx.extract_source().column_reference());
+    } else if (checkIfExist(ctx.extract_source().datetime_literal())) {
+      extractSource = visitDatetime_literal(ctx.extract_source().datetime_literal());
+    } else {
+      return null;
+    }
+
+    String functionName = "date_part";
+    Expr[] params = new Expr[]{extractTarget, extractSource};
+
+    return new FunctionExpr(functionName, params);
+  }
+
+  @Override
+  public Expr visitTrim_function(SQLParser.Trim_functionContext ctx) {
+    Expr trimSource = visitChildren(ctx.trim_operands().trim_source);
+    String functionName = "trim";
+    if (checkIfExist(ctx.trim_operands().FROM())) {
+      if (checkIfExist(ctx.trim_operands().trim_specification())) {
+        Trim_specificationContext specification = ctx.trim_operands().trim_specification();
+        if (checkIfExist(specification.LEADING())) {
+          functionName = "ltrim";
+        } else if (checkIfExist(specification.TRAILING())) {
+          functionName = "rtrim";
+        } else {
+          functionName = "trim";
+        }
+      }
+    }
+
+    Expr trimCharacters = null;
+    if (checkIfExist(ctx.trim_operands().trim_character)) {
+      trimCharacters = visitCharacter_value_expression(ctx.trim_operands().trim_character);
+    }
+
+    Expr[] params;
+    if (trimCharacters != null) {
+      params = new Expr[]{trimSource, trimCharacters};
+    } else {
+      params = new Expr[]{trimSource};
+    }
+
+    return new FunctionExpr(functionName, params);
+  }
+
+  @Override
+  public Expr visitDatabase_definition(@NotNull SQLParser.Database_definitionContext ctx) {
+    return new CreateDatabase(ctx.identifier().getText(), null, checkIfExist(ctx.if_not_exists()));
+  }
+
+  @Override
+  public Expr visitDrop_database_statement(@NotNull SQLParser.Drop_database_statementContext ctx) {
+    return new DropDatabase(ctx.identifier().getText(), checkIfExist(ctx.if_exists()));
+  }
+
+  @Override
+  public Expr visitCreate_table_statement(SQLParser.Create_table_statementContext ctx) {
+    String tableName = ctx.table_name().getText();
+    CreateTable createTable = new CreateTable(tableName, checkIfExist(ctx.if_not_exists()));
+
+    if (checkIfExist(ctx.EXTERNAL())) {
+      createTable.setExternal();
+
+      ColumnDefinition[] elements = getDefinitions(ctx.table_elements());
+      String fileType = ctx.file_type.getText();
+      String path = stripQuote(ctx.path.getText());
+
+      createTable.setTableElements(elements);
+      createTable.setStorageType(fileType);
+      createTable.setLocation(path);
+    } else {
+      if (checkIfExist(ctx.table_elements())) {
+        ColumnDefinition[] elements = getDefinitions(ctx.table_elements());
+        createTable.setTableElements(elements);
+      }
+
+      if (checkIfExist(ctx.USING())) {
+        String fileType = ctx.file_type.getText();
+        createTable.setStorageType(fileType);
+      }
+
+      if (checkIfExist(ctx.query_expression())) {
+        Expr subquery = visitQuery_expression(ctx.query_expression());
+        createTable.setSubQuery(subquery);
+      }
+    }
+
+    if (checkIfExist(ctx.param_clause())) {
+      Map<String, String> params = escapeTableMeta(getParams(ctx.param_clause()));
+      createTable.setParams(params);
+    }
+
+    if (checkIfExist(ctx.table_partitioning_clauses())) {
+      PartitionMethodDescExpr partitionMethodDesc =
+          parseTablePartitioningClause(ctx.table_partitioning_clauses());
+      createTable.setPartitionMethod(partitionMethodDesc);
+    }
+    return createTable;
+  }
+
+  private ColumnDefinition[] getDefinitions(SQLParser.Table_elementsContext ctx) {
+    int size = ctx.field_element().size();
+    ColumnDefinition[] elements = new ColumnDefinition[size];
+    for (int i = 0; i < size; i++) {
+      String name = ctx.field_element(i).name.getText();
+      DataTypeExpr typeDef = visitData_type(ctx.field_element(i).field_type().data_type());
+      elements[i] = new ColumnDefinition(name, typeDef);
+    }
+
+    return elements;
+  }
+
+  public PartitionMethodDescExpr parseTablePartitioningClause(SQLParser.Table_partitioning_clausesContext ctx) {
+
+    if (checkIfExist(ctx.range_partitions())) { // For Range Partition
+      Range_partitionsContext rangePartitionsContext = ctx.range_partitions();
+      List<Range_value_clauseContext> rangeValueClause = rangePartitionsContext.
+          range_value_clause_list().range_value_clause();
+
+      List<RangePartitionSpecifier> specifiers = Lists.newArrayList();
+
+      for (Range_value_clauseContext rangeValue : rangeValueClause) {
+        if (checkIfExist(rangeValue.MAXVALUE())) { // LESS THAN (MAXVALUE)
+          specifiers.add(new RangePartitionSpecifier(rangeValue.partition_name().getText()));
+        } else { // LESS THAN (expr)
+          specifiers.add(new RangePartitionSpecifier(rangeValue.partition_name().getText(),
+              visitValue_expression(rangeValue.value_expression())));
+        }
+      }
+      return new CreateTable.RangePartition(getColumnReferences(ctx.range_partitions().column_reference_list()),
+          specifiers);
+
+    } else if (checkIfExist(ctx.hash_partitions())) { // For Hash Partition
+      Hash_partitionsContext hashPartitions = ctx.hash_partitions();
+
+      if (checkIfExist(hashPartitions.hash_partitions_by_quantity())) { // PARTITIONS (num)
+        return new HashPartition(getColumnReferences(hashPartitions.column_reference_list()),
+            visitNumeric_value_expression(hashPartitions.hash_partitions_by_quantity().quantity));
+
+      } else { // ( PARTITION part_name , ...)
+        List<CreateTable.PartitionSpecifier> specifiers = Lists.newArrayList();
+        for (Individual_hash_partitionContext partition :
+            hashPartitions.individual_hash_partitions().individual_hash_partition()) {
+          specifiers.add(new CreateTable.PartitionSpecifier(partition.partition_name().getText()));
+        }
+        return new HashPartition(getColumnReferences(hashPartitions.column_reference_list()), specifiers);
+      }
+
+    } else if (checkIfExist(ctx.list_partitions())) { // For List Partition
+      List_partitionsContext listPartitions = ctx.list_partitions();
+      List<List_value_partitionContext> partitions = listPartitions.list_value_clause_list().list_value_partition();
+      List<ListPartitionSpecifier> specifiers = Lists.newArrayList();
+
+      for (List_value_partitionContext listValuePartition : partitions) {
+        int size = listValuePartition.in_value_list().row_value_predicand().size();
+        Expr [] exprs = new Expr[size];
+        for (int i = 0; i < size; i++) {
+          exprs[i] = visitRow_value_predicand(listValuePartition.in_value_list().row_value_predicand(i));
+        }
+        specifiers.add(new ListPartitionSpecifier(listValuePartition.partition_name().getText(),
+            new ValueListExpr(exprs)));
+      }
+      return new ListPartition(getColumnReferences(ctx.list_partitions().column_reference_list()), specifiers);
+
+    } else if (checkIfExist(ctx.column_partitions())) { // For Column Partition (Hive Style)
+      return new CreateTable.ColumnPartition(getDefinitions(ctx.column_partitions().table_elements()), true);
+    } else {
+      throw new SQLSyntaxError("Invalid Partition Type: " + ctx.toStringTree());
+    }
+  }
+
+  @Override
+  public DataTypeExpr visitData_type(SQLParser.Data_typeContext ctx) {
+    SQLParser.Predefined_typeContext predefined_type = ctx.predefined_type();
+
+    DataTypeExpr typeDefinition = null;
+    if (predefined_type.character_string_type() != null) {
+      SQLParser.Character_string_typeContext character_string_type =
+          predefined_type.character_string_type();
+
+      if ((character_string_type.CHARACTER() != null || character_string_type.CHAR() != null) &&
+          character_string_type.VARYING() == null) {
+
+        typeDefinition = new DataTypeExpr(Type.CHAR.name());
+
+        if (character_string_type.type_length() != null) {
+          typeDefinition.setLengthOrPrecision(
+              Integer.parseInt(character_string_type.type_length().NUMBER().getText()));
+        }
+
+      } else if (character_string_type.VARCHAR() != null
+          || character_string_type.VARYING() != null) {
+
+        typeDefinition = new DataTypeExpr(Type.VARCHAR.name());
+
+        if (character_string_type.type_length() != null) {
+          typeDefinition.setLengthOrPrecision(
+              Integer.parseInt(character_string_type.type_length().NUMBER().getText()));
+        }
+
+      } else if (character_string_type.TEXT() != null) {
+        typeDefinition = new DataTypeExpr(Type.TEXT.name());
+      }
+
+    } else if (predefined_type.national_character_string_type() != null) {
+      SQLParser.National_character_string_typeContext nchar_type =
+          predefined_type.national_character_string_type();
+      if ((nchar_type.CHAR() != null || nchar_type.CHARACTER() != null
+          || nchar_type.NCHAR() != null) && nchar_type.VARYING() == null) {
+        typeDefinition = new DataTypeExpr(Type.NCHAR.name());
+      } else if (nchar_type.NVARCHAR() != null || nchar_type.VARYING() != null) {
+        typeDefinition = new DataTypeExpr(Type.NVARCHAR.name());
+      }
+
+      if (nchar_type.type_length() != null) {
+        typeDefinition.setLengthOrPrecision(
+            Integer.parseInt(nchar_type.type_length().NUMBER().getText()));
+      }
+
+    } else if (predefined_type.binary_large_object_string_type() != null) {
+      SQLParser.Binary_large_object_string_typeContext blob_type =
+          predefined_type.binary_large_object_string_type();
+      typeDefinition = new DataTypeExpr(Type.BLOB.name());
+      if (blob_type.type_length() != null) {
+        typeDefinition.setLengthOrPrecision(
+            Integer.parseInt(blob_type.type_length().NUMBER().getText()));
+      }
+    } else if (predefined_type.numeric_type() != null) {
+      // exact number
+      if (predefined_type.numeric_type().exact_numeric_type() != null) {
+        SQLParser.Exact_numeric_typeContext exactType =
+            predefined_type.numeric_type().exact_numeric_type();
+        if (exactType.TINYINT() != null || exactType.INT1() != null) {
+          typeDefinition = new DataTypeExpr(Type.INT1.name());
+        } else if (exactType.INT2() != null || exactType.SMALLINT() != null) {
+          typeDefinition = new DataTypeExpr(Type.INT2.name());
+        } else if (exactType.INT4() != null || exactType.INTEGER() != null ||
+            exactType.INT() != null) {
+          typeDefinition = new DataTypeExpr(Type.INT4.name());
+        } else if (exactType.INT8() != null || exactType.BIGINT() != null) {
+          typeDefinition = new DataTypeExpr(Type.INT8.name());
+        } else if (exactType.NUMERIC() != null) {
+          typeDefinition = new DataTypeExpr(Type.NUMERIC.name());
+        } else if (exactType.DECIMAL() != null || exactType.DEC() != null) {
+          typeDefinition = new DataTypeExpr(Type.NUMERIC.name());
+        }
+
+        if (typeDefinition.getTypeName().equals(Type.NUMERIC.name())) {
+          if (exactType.precision_param() != null) {
+            if (exactType.precision_param().scale != null) {
+              typeDefinition.setScale(
+                  Integer.parseInt(exactType.precision_param().scale.getText()));
+            }
+            typeDefinition.setLengthOrPrecision(
+                Integer.parseInt(exactType.precision_param().precision.getText()));
+          }
+        }
+      } else { // approximate number
+        SQLParser.Approximate_numeric_typeContext approximateType =
+            predefined_type.numeric_type().approximate_numeric_type();
+        if (approximateType.FLOAT() != null || approximateType.FLOAT4() != null
+            || approximateType.REAL() != null) {
+          typeDefinition = new DataTypeExpr(Type.FLOAT4.name());
+        } else if (approximateType.FLOAT8() != null || approximateType.DOUBLE() != null) {
+          typeDefinition = new DataTypeExpr(Type.FLOAT8.name());
+        }
+      }
+    } else if (predefined_type.boolean_type() != null) {
+      typeDefinition = new DataTypeExpr(Type.BOOLEAN.name());
+    } else if (predefined_type.datetime_type() != null) {
+      SQLParser.Datetime_typeContext dateTimeType = predefined_type.datetime_type();
+      if (dateTimeType.DATE() != null) {
+        typeDefinition = new DataTypeExpr(Type.DATE.name());
+      } else if (dateTimeType.TIME(0) != null && dateTimeType.ZONE() == null) {
+        typeDefinition = new DataTypeExpr(Type.TIME.name());
+      } else if ((dateTimeType.TIME(0) != null && dateTimeType.ZONE() != null) ||
+          dateTimeType.TIMETZ() != null) {
+        typeDefinition = new DataTypeExpr(Type.TIMEZ.name());
+      } else if (dateTimeType.TIMESTAMP() != null && dateTimeType.ZONE() == null) {
+        typeDefinition = new DataTypeExpr(Type.TIMESTAMP.name());
+      } else if ((dateTimeType.TIMESTAMP() != null && dateTimeType.ZONE() != null) ||
+          dateTimeType.TIMESTAMPTZ() != null) {
+        typeDefinition = new DataTypeExpr(Type.TIMESTAMPZ.name());
+      }
+    } else if (predefined_type.bit_type() != null) {
+      SQLParser.Bit_typeContext bitType = predefined_type.bit_type();
+      if (bitType.VARBIT() != null || bitType.VARYING() != null) {
+        typeDefinition = new DataTypeExpr(Type.VARBIT.name());
+      } else {
+        typeDefinition = new DataTypeExpr(Type.BIT.name());
+      }
+      if (bitType.type_length() != null) {
+        typeDefinition.setLengthOrPrecision(
+            Integer.parseInt(bitType.type_length().NUMBER().getText()));
+      }
+    } else if (predefined_type.binary_type() != null) {
+      SQLParser.Binary_typeContext binaryType = predefined_type.binary_type();
+      if (binaryType.VARBINARY() != null || binaryType.VARYING() != null) {
+        typeDefinition = new DataTypeExpr(Type.VARBINARY.name());
+      } else {
+        typeDefinition = new DataTypeExpr(Type.BINARY.name());
+      }
+
+      if (binaryType.type_length() != null) {
+        typeDefinition.setLengthOrPrecision(
+            Integer.parseInt(binaryType.type_length().NUMBER().getText()));
+      }
+    } else if (predefined_type.network_type() != null) {
+      typeDefinition = new DataTypeExpr(Type.INET4.name());
+    }
+
+    return typeDefinition;
+  }
+
+  @Override
+  public Expr visitInsert_statement(SQLParser.Insert_statementContext ctx) {
+    Insert insertExpr = new Insert();
+
+    if (ctx.OVERWRITE() != null) {
+      insertExpr.setOverwrite();
+    }
+
+    if (ctx.table_name() != null) {
+      insertExpr.setTableName(ctx.table_name().getText());
+
+      if (ctx.column_name_list() != null) {
+        String[] targetColumns = new String[ctx.column_name_list().identifier().size()];
+        for (int i = 0; i < targetColumns.length; i++) {
+          targetColumns[i] = ctx.column_name_list().identifier().get(i).getText();
+        }
+
+        insertExpr.setTargetColumns(targetColumns);
+      }
+    }
+
+    if (ctx.LOCATION() != null) {
+      insertExpr.setLocation(stripQuote(ctx.path.getText()));
+
+      if (ctx.USING() != null) {
+        insertExpr.setStorageType(ctx.file_type.getText());
+
+        if (ctx.param_clause() != null) {
+          insertExpr.setParams(escapeTableMeta(getParams(ctx.param_clause())));
+        }
+      }
+    }
+
+    insertExpr.setSubQuery(visitQuery_expression(ctx.query_expression()));
+
+    Preconditions.checkState(insertExpr.hasTableName() || insertExpr.hasLocation(),
+        "Either a table name or a location should be given.");
+    Preconditions.checkState(insertExpr.hasTableName() ^ insertExpr.hasLocation(),
+        "A table name and a location cannot coexist.");
+    return insertExpr;
+  }
+
+  @Override
+  public Expr visitDrop_table_statement(SQLParser.Drop_table_statementContext ctx) {
+    return new DropTable(ctx.table_name().getText(), checkIfExist(ctx.if_exists()), checkIfExist(ctx.PURGE()));
+  }
+
+
+  private Map<String, String> getParams(SQLParser.Param_clauseContext ctx) {
+    Map<String, String> params = new HashMap<String, String>();
+    for (int i = 0; i < ctx.param().size(); i++) {
+      params.put(stripQuote(ctx.param(i).key.getText()), stripQuote(ctx.param(i).value.getText()));
+    }
+
+    return params;
+  }
+
+  public Map<String, String> escapeTableMeta(Map<String, String> map) {
+    Map<String, String> params = new HashMap<String, String>();
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      if (entry.getKey().equals(StorageConstants.CSVFILE_DELIMITER)) {
+        params.put(entry.getKey(), escapeDelimiter(entry.getValue()));
+      } else {
+        params.put(entry.getKey(), entry.getValue());
+      }
+    }
+    return params;
+  }
+
+  public static String escapeDelimiter(String value) {
+    try {
+      String delimiter = StringEscapeUtils.unescapeJava(value);
+      delimiter = new String(new byte[]{Byte.valueOf(delimiter).byteValue()}, Charset.defaultCharset());
+      return StringEscapeUtils.escapeJava(delimiter);
+    } catch (NumberFormatException e) {
+    }
+    return value;
+  }
+
+  private static String stripQuote(String str) {
+    return str.substring(1, str.length() - 1);
+  }
+
+  @Override
+  public Expr visitCast_specification(SQLParser.Cast_specificationContext ctx) {
+    Expr operand = visitChildren(ctx.cast_operand());
+    DataTypeExpr castTarget = visitData_type(ctx.cast_target().data_type());
+    return new CastExpr(operand, castTarget);
+  }
+
+  @Override
+  public Expr visitUnsigned_value_specification(@NotNull SQLParser.Unsigned_value_specificationContext ctx) {
+    return visitChildren(ctx);
+  }
+
+  @Override
+  public Expr visitUnsigned_literal(@NotNull SQLParser.Unsigned_literalContext ctx) {
+    if (checkIfExist(ctx.unsigned_numeric_literal())) {
+      return visitUnsigned_numeric_literal(ctx.unsigned_numeric_literal());
+    } else {
+      return visitGeneral_literal(ctx.general_literal());
+    }
+  }
+
+  @Override
+  public Expr visitGeneral_literal(SQLParser.General_literalContext ctx) {
+    if (checkIfExist(ctx.Character_String_Literal())) {
+      return new LiteralValue(stripQuote(ctx.Character_String_Literal().getText()), LiteralType.String);
+    } else if (checkIfExist(ctx.datetime_literal())) {
+      return visitDatetime_literal(ctx.datetime_literal());
+    } else {
+      return new BooleanLiteral(checkIfExist(ctx.boolean_literal().TRUE()));
+    }
+  }
+
+  @Override
+  public Expr visitDatetime_literal(@NotNull SQLParser.Datetime_literalContext ctx) {
+    if (checkIfExist(ctx.time_literal())) {
+      return visitTime_literal(ctx.time_literal());
+    } else if (checkIfExist(ctx.date_literal())) {
+      return visitDate_literal(ctx.date_literal());
+    } else {
+      return visitTimestamp_literal(ctx.timestamp_literal());
+    }
+  }
+
+  @Override
+  public Expr visitTime_literal(SQLParser.Time_literalContext ctx) {
+    String timePart = stripQuote(ctx.time_string.getText());
+    return new TimeLiteral(parseTime(timePart));
+  }
+
+  @Override
+  public Expr visitDate_literal(SQLParser.Date_literalContext ctx) {
+    String datePart = stripQuote(ctx.date_string.getText());
+    return new DateLiteral(parseDate(datePart));
+  }
+
+  @Override
+  public Expr visitTimestamp_literal(SQLParser.Timestamp_literalContext ctx) {
+    String timestampStr = stripQuote(ctx.timestamp_string.getText());
+    String[] parts = timestampStr.split(" ");
+    String datePart = parts[0];
+    String timePart = parts[1];
+    return new TimestampLiteral(parseDate(datePart), parseTime(timePart));
+  }
+
+  private DateValue parseDate(String datePart) {
+    // e.g., 1980-04-01
+    String[] parts = datePart.split("-");
+    return new DateValue(parts[0], parts[1], parts[2]);
+  }
+
+  private TimeValue parseTime(String timePart) {
+    // e.g., 12:01:50.399
+    String[] parts = timePart.split(":");
+
+    TimeValue time;
+    boolean hasFractionOfSeconds = parts[2].indexOf('.') > 0;
+    if (hasFractionOfSeconds) {
+      String[] secondsParts = parts[2].split("\\.");
+      time = new TimeValue(parts[0], parts[1], secondsParts[0]);
+      if (secondsParts.length == 2) {
+        time.setSecondsFraction(secondsParts[1]);
+      }
+    } else {
+      time = new TimeValue(parts[0], parts[1], parts[2]);
+    }
+    return time;
+  }
+
+  @Override
+  public Expr visitAlter_tablespace_statement(@NotNull SQLParser.Alter_tablespace_statementContext ctx) {
+    AlterTablespace alter = new AlterTablespace(ctx.space_name.getText());
+    alter.setLocation(stripQuote(ctx.uri.getText()));
+    return alter;
+  }
+
+  @Override
+  public Expr visitAlter_table_statement(SQLParser.Alter_table_statementContext ctx) {
+
+    final List<Table_nameContext> tables = ctx.table_name();
+
+    final AlterTable alterTable = new AlterTable(tables.get(0).getText());
+
+    if (tables.size() == 2) {
+      alterTable.setNewTableName(tables.get(1).getText());
+    }
+
+    if (checkIfExist(ctx.column_name()) && ctx.column_name().size() == 2) {
+      final List<Column_nameContext> columns = ctx.column_name();
+      alterTable.setColumnName(columns.get(0).getText());
+      alterTable.setNewColumnName(columns.get(1).getText());
+    }
+
+    Field_elementContext field_elementContext = ctx.field_element();
+    if (checkIfExist(field_elementContext)) {
+      final String name = field_elementContext.name.getText();
+      final DataTypeExpr typeDef = visitData_type(field_elementContext.field_type().data_type());
+      final ColumnDefinition columnDefinition = new ColumnDefinition(name, typeDef);
+      alterTable.setAddNewColumn(columnDefinition);
+    }
+
+    alterTable.setAlterTableOpType(determineAlterTableType(ctx));
+
+    return alterTable;
+  }
+
+  private AlterTableOpType determineAlterTableType(SQLParser.Alter_table_statementContext ctx) {
+
+    final int RENAME_MASK = 00000001;
+    final int COLUMN_MASK = 00000010;
+    final int TO_MASK = 00000100;
+    final int ADD_MASK = 00001000;
+
+    int val = 00000000;
+
+    for (int idx = 1; idx < ctx.getChildCount(); idx++) {
+
+      if (ctx.getChild(idx) instanceof TerminalNode) {
+        if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == RENAME) {
+          val = val | RENAME_MASK;
+        }
+        if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == COLUMN) {
+          val = val | COLUMN_MASK;
+        }
+        if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == TO) {
+          val = val | TO_MASK;
+        }
+        if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == ADD) {
+          val = val | ADD_MASK;
+        }
+      }
+    }
+    return evaluateAlterTableOperationTye(val);
+  }
+
+  private AlterTableOpType evaluateAlterTableOperationTye(final int value) {
+
+    switch (value) {
+      case 65:
+        return AlterTableOpType.RENAME_TABLE;
+      case 73:
+        return AlterTableOpType.RENAME_COLUMN;
+      case 520:
+        return AlterTableOpType.ADD_COLUMN;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java
new file mode 100644
index 0000000..1df5f7b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.parser;
+
+import org.antlr.v4.runtime.*;
+import org.apache.commons.lang.StringUtils;
+
+public class SQLErrorListener extends BaseErrorListener {
+  public void syntaxError(Recognizer<?, ?> recognizer,
+                          Object offendingSymbol,
+                          int line, int charPositionInLine,
+                          String msg,
+                          RecognitionException e) {
+    CommonTokenStream tokens = (CommonTokenStream) recognizer.getInputStream();
+    String input = tokens.getTokenSource().getInputStream().toString();
+    Token token = (Token) offendingSymbol;
+    String[] lines = StringUtils.splitPreserveAllTokens(input, '\n');
+    String errorLine = lines[line - 1];
+
+    throw new SQLParseError(token, line, charPositionInLine, msg, errorLine);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java
new file mode 100644
index 0000000..7e1d43c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java
@@ -0,0 +1,66 @@
+/**
+ * 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.parser;
+
+import org.antlr.v4.runtime.*;
+import org.antlr.v4.runtime.misc.NotNull;
+
+public class SQLErrorStrategy extends DefaultErrorStrategy {
+
+  @Override
+  public void reportError(Parser recognizer, RecognitionException e) {
+    // if we've already reported an error and have not matched a token
+    // yet successfully, don't report any errors.
+    if (inErrorRecoveryMode(recognizer)) {
+      return; // don't report spurious errors
+    }
+    beginErrorCondition(recognizer);
+    if (e instanceof NoViableAltException) {
+      reportNoViableAltException(recognizer, (NoViableAltException) e);
+    } else if (e instanceof InputMismatchException) {
+      reportInputMismatchException(recognizer, (InputMismatchException) e);
+    } else if (e instanceof FailedPredicateException) {
+      reportFailedPredicate(recognizer, (FailedPredicateException) e);
+    } else {
+      recognizer.notifyErrorListeners(e.getOffendingToken(), e.getMessage(), e);
+    }
+  }
+
+  protected void reportNoViableAltException(@NotNull Parser recognizer, @NotNull NoViableAltException e) {
+    TokenStream tokens = recognizer.getInputStream();
+    String msg;
+    Token token = e.getStartToken();
+    if (tokens != null) {
+      if (tokens.LT(-1) != null && token.getType() == Token.EOF) {
+        token = tokens.LT(-1);
+      }
+      msg = "syntax error at or near " + getTokenErrorDisplay(token);
+    } else {
+      msg = "no viable alternative at input " + escapeWSAndQuote("<unknown input>");
+    }
+    recognizer.notifyErrorListeners(token, msg, e);
+  }
+
+  protected void reportInputMismatchException(@NotNull Parser recognizer,
+                                              @NotNull InputMismatchException e) {
+    String msg = "mismatched input " + getTokenErrorDisplay(e.getOffendingToken()) +
+        " expecting " + e.getExpectedTokens().toString(recognizer.getTokenNames());
+    recognizer.notifyErrorListeners(e.getOffendingToken(), msg, e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLParseError.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLParseError.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLParseError.java
new file mode 100644
index 0000000..2106b53
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLParseError.java
@@ -0,0 +1,107 @@
+/**
+ * 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.parser;
+
+
+import org.antlr.v4.runtime.Token;
+import org.apache.commons.lang.StringUtils;
+
+public class SQLParseError extends RuntimeException {
+  private String header;
+  private String errorLine;
+  private int charPositionInLine;
+  private int line;
+  private Token offendingToken;
+  private String detailedMessage;
+
+  public SQLParseError(Token offendingToken,
+                       int line, int charPositionInLine,
+                       String msg,
+                       String errorLine) {
+    super(msg);
+    this.offendingToken = offendingToken;
+    this.charPositionInLine = charPositionInLine;
+    this.line = line;
+    this.errorLine = errorLine;
+    this.header = msg;
+  }
+
+  @Override
+  public String getMessage() {
+    if (detailedMessage == null) {
+      if (offendingToken != null) {
+        detailedMessage = getDetailedMessageWithLocation();
+      } else {
+        StringBuilder sb = new StringBuilder();
+        sb.append("ERROR: ").append(header).append("\n");
+        sb.append("LINE: ").append(errorLine);
+        detailedMessage = sb.toString();
+      }
+    }
+
+    return detailedMessage;
+  }
+
+  public String getMessageHeader(){
+    return this.header;
+  }
+
+  private String getDetailedMessageWithLocation() {
+    StringBuilder sb = new StringBuilder();
+    int displayLimit = 80;
+    String queryPrefix = "LINE " + line + ":" + charPositionInLine + " ";
+    String prefixPadding = StringUtils.repeat(" ", queryPrefix.length());
+    String locationString;
+
+    int tokenLength = offendingToken.getStopIndex() - offendingToken.getStartIndex() + 1;
+    if(tokenLength > 0){
+      locationString = StringUtils.repeat(" ", charPositionInLine) + StringUtils.repeat("^", tokenLength);
+    } else {
+      locationString = StringUtils.repeat(" ", charPositionInLine) + "^";
+    }
+
+    sb.append("ERROR: ").append(header).append("\n");
+    sb.append(queryPrefix);
+
+    if (errorLine.length() > displayLimit) {
+      int padding = (displayLimit / 2);
+
+      String ellipsis = " ... ";
+      int startPos = locationString.length() - padding - 1;
+      if (startPos <= 0) {
+        startPos = 0;
+        sb.append(errorLine.substring(startPos, displayLimit)).append(ellipsis).append("\n");
+        sb.append(prefixPadding).append(locationString);
+      } else if (errorLine.length() - (locationString.length() + padding) <= 0) {
+        startPos = errorLine.length() - displayLimit - 1;
+        sb.append(ellipsis).append(errorLine.substring(startPos)).append("\n");
+        sb.append(prefixPadding).append(StringUtils.repeat(" ", ellipsis.length()))
+            .append(locationString.substring(startPos));
+      } else {
+        sb.append(ellipsis).append(errorLine.substring(startPos, startPos + displayLimit)).append(ellipsis).append("\n");
+        sb.append(prefixPadding).append(StringUtils.repeat(" ", ellipsis.length()))
+            .append(locationString.substring(startPos));
+      }
+    } else {
+      sb.append(errorLine).append("\n");
+      sb.append(prefixPadding).append(locationString);
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java
new file mode 100644
index 0000000..d565509
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java
@@ -0,0 +1,51 @@
+/**
+ * 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.parser;
+
+
+import org.apache.tajo.engine.exception.InvalidQueryException;
+
+public class SQLSyntaxError extends InvalidQueryException {
+  private static final long serialVersionUID = 5388279335175632066L;
+
+  private String errorMessage;
+  private String detailedMessage;
+  private SQLParseError parseError;
+
+  public SQLSyntaxError(String errorMessage) {
+    this.errorMessage = errorMessage;
+  }
+
+  public SQLSyntaxError(SQLParseError e) {
+    this.errorMessage = e.getMessageHeader();
+    this.parseError = e;
+  }
+
+  @Override
+  public String getMessage() {
+    if (detailedMessage == null) {
+      if (parseError != null) {
+        detailedMessage = parseError.getMessage();
+      } else {
+        detailedMessage = String.format("ERROR: %s\n", errorMessage);
+      }
+    }
+    return detailedMessage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
new file mode 100644
index 0000000..aa94801
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
@@ -0,0 +1,110 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.algebra.*;
+
+import java.util.Stack;
+
+public interface AlgebraVisitor<CONTEXT, RESULT> {
+  // Relational Operators
+  RESULT visitProjection(CONTEXT ctx, Stack<Expr> stack, Projection expr) throws PlanningException;
+  RESULT visitLimit(CONTEXT ctx, Stack<Expr> stack, Limit expr) throws PlanningException;
+  RESULT visitSort(CONTEXT ctx, Stack<Expr> stack, Sort expr) throws PlanningException;
+  RESULT visitHaving(CONTEXT ctx, Stack<Expr> stack, Having expr) throws PlanningException;
+  RESULT visitGroupBy(CONTEXT ctx, Stack<Expr> stack, Aggregation expr) throws PlanningException;
+  RESULT visitJoin(CONTEXT ctx, Stack<Expr> stack, Join expr) throws PlanningException;
+  RESULT visitFilter(CONTEXT ctx, Stack<Expr> stack, Selection expr) throws PlanningException;
+  RESULT visitUnion(CONTEXT ctx, Stack<Expr> stack, SetOperation expr) throws PlanningException;
+  RESULT visitExcept(CONTEXT ctx, Stack<Expr> stack, SetOperation expr) throws PlanningException;
+  RESULT visitIntersect(CONTEXT ctx, Stack<Expr> stack, SetOperation expr) throws PlanningException;
+  RESULT visitSimpleTableSubQuery(CONTEXT ctx, Stack<Expr> stack, SimpleTableSubQuery expr) throws PlanningException;
+  RESULT visitTableSubQuery(CONTEXT ctx, Stack<Expr> stack, TablePrimarySubQuery expr) throws PlanningException;
+  RESULT visitRelationList(CONTEXT ctx, Stack<Expr> stack, RelationList expr) throws PlanningException;
+  RESULT visitRelation(CONTEXT ctx, Stack<Expr> stack, Relation expr) throws PlanningException;
+  RESULT visitScalarSubQuery(CONTEXT ctx, Stack<Expr> stack, ScalarSubQuery expr) throws PlanningException;
+  RESULT visitExplain(CONTEXT ctx, Stack<Expr> stack, Explain expr) throws PlanningException;
+
+  // Data definition language
+  RESULT visitCreateDatabase(CONTEXT ctx, Stack<Expr> stack, CreateDatabase expr) throws PlanningException;
+  RESULT visitDropDatabase(CONTEXT ctx, Stack<Expr> stack, DropDatabase expr) throws PlanningException;
+  RESULT visitCreateTable(CONTEXT ctx, Stack<Expr> stack, CreateTable expr) throws PlanningException;
+  RESULT visitDropTable(CONTEXT ctx, Stack<Expr> stack, DropTable expr) throws PlanningException;
+  RESULT visitAlterTablespace(CONTEXT ctx, Stack<Expr> stack, AlterTablespace expr) throws PlanningException;
+  RESULT visitAlterTable(CONTEXT ctx, Stack<Expr> stack, AlterTable expr) throws PlanningException;
+
+    // Insert or Update
+  RESULT visitInsert(CONTEXT ctx, Stack<Expr> stack, Insert expr) throws PlanningException;
+
+  // Logical operators
+  RESULT visitAnd(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitOr(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitNot(CONTEXT ctx, Stack<Expr> stack, NotExpr expr) throws PlanningException;
+
+  // comparison predicates
+  RESULT visitEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitNotEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitLessThan(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitLessThanOrEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitGreaterThan(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitGreaterThanOrEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+
+  // Other Predicates
+  RESULT visitBetween(CONTEXT ctx, Stack<Expr> stack, BetweenPredicate expr) throws PlanningException;
+  RESULT visitCaseWhen(CONTEXT ctx, Stack<Expr> stack, CaseWhenPredicate expr) throws PlanningException;
+  RESULT visitIsNullPredicate(CONTEXT ctx, Stack<Expr> stack, IsNullPredicate expr) throws PlanningException;
+  RESULT visitInPredicate(CONTEXT ctx, Stack<Expr> stack, InPredicate expr) throws PlanningException;
+  RESULT visitValueListExpr(CONTEXT ctx, Stack<Expr> stack, ValueListExpr expr) throws PlanningException;
+  RESULT visitExistsPredicate(CONTEXT ctx, Stack<Expr> stack, ExistsPredicate expr) throws PlanningException;
+
+  // String Operator or Pattern Matching Predicates
+  RESULT visitLikePredicate(CONTEXT ctx, Stack<Expr> stack, PatternMatchPredicate expr) throws PlanningException;
+  RESULT visitSimilarToPredicate(CONTEXT ctx, Stack<Expr> stack, PatternMatchPredicate expr) throws PlanningException;
+  RESULT visitRegexpPredicate(CONTEXT ctx, Stack<Expr> stack, PatternMatchPredicate expr) throws PlanningException;
+  RESULT visitConcatenate(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+
+  // arithmetic operators
+  RESULT visitPlus(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitMinus(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitMultiply(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitDivide(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+  RESULT visitModular(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException;
+
+  // other expressions
+  RESULT visitSign(CONTEXT ctx, Stack<Expr> stack, SignedExpr expr) throws PlanningException;
+  RESULT visitColumnReference(CONTEXT ctx, Stack<Expr> stack, ColumnReferenceExpr expr) throws PlanningException;
+  RESULT visitTargetExpr(CONTEXT ctx, Stack<Expr> stack, NamedExpr expr) throws PlanningException;
+  RESULT visitFunction(CONTEXT ctx, Stack<Expr> stack, FunctionExpr expr) throws PlanningException;
+  RESULT visitQualifiedAsterisk(CONTEXT ctx, Stack<Expr> stack, QualifiedAsteriskExpr expr) throws PlanningException;
+
+  // set functions
+  RESULT visitCountRowsFunction(CONTEXT ctx, Stack<Expr> stack, CountRowsFunctionExpr expr) throws PlanningException;
+  RESULT visitGeneralSetFunction(CONTEXT ctx, Stack<Expr> stack, GeneralSetFunctionExpr expr)
+      throws PlanningException;
+
+  // Literal
+  RESULT visitCastExpr(CONTEXT ctx, Stack<Expr> stack, CastExpr expr) throws PlanningException;
+
+  RESULT visitDataType(CONTEXT ctx, Stack<Expr> stack, DataTypeExpr expr) throws PlanningException;
+  RESULT visitLiteral(CONTEXT ctx, Stack<Expr> stack, LiteralValue expr) throws PlanningException;
+  RESULT visitNullLiteral(CONTEXT ctx, Stack<Expr> stack, NullLiteral expr) throws PlanningException;
+  RESULT visitTimestampLiteral(CONTEXT ctx, Stack<Expr> stack, TimestampLiteral expr) throws PlanningException;
+  RESULT visitTimeLiteral(CONTEXT ctx, Stack<Expr> stack, TimeLiteral expr) throws PlanningException;
+  RESULT visitDateLiteral(CONTEXT ctx, Stack<Expr> stack, DateLiteral expr) throws PlanningException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlterTablespaceNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlterTablespaceNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlterTablespaceNode.java
new file mode 100644
index 0000000..d64a89d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/AlterTablespaceNode.java
@@ -0,0 +1,103 @@
+/**
+ * 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.planner;
+
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.algebra.AlterTablespaceSetType;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.LogicalNodeVisitor;
+import org.apache.tajo.engine.planner.logical.NodeType;
+
+public class AlterTablespaceNode extends LogicalNode implements Cloneable {
+
+  @Expose private String tablespaceName;
+  @Expose private AlterTablespaceSetType setType;
+  @Expose private String uri;
+
+
+  public AlterTablespaceNode(int pid) {
+    super(pid, NodeType.ALTER_TABLESPACE);
+  }
+
+  public String getTablespaceName() {
+    return tablespaceName;
+  }
+
+  public void setTablespaceName(String tablespaceName) {
+    this.tablespaceName = tablespaceName;
+  }
+
+  public AlterTablespaceSetType getSetType() {
+    return setType;
+  }
+
+  public String getLocation() {
+    return uri;
+  }
+
+  public void setLocation(String uri) {
+    this.setType = AlterTablespaceSetType.LOCATION;
+    this.uri = uri;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof AlterTablespaceNode) {
+      AlterTablespaceNode other = (AlterTablespaceNode) obj;
+      return super.equals(other);
+    } else {
+      return false;
+    }
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(tablespaceName, setType, uri);
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    AlterTablespaceNode alter = (AlterTablespaceNode) super.clone();
+    alter.tablespaceName = tablespaceName;
+    alter.setType = setType;
+    alter.uri = uri;
+    return alter;
+  }
+
+  @Override
+  public String toString() {
+    return "AlterTablespace (space=" + tablespaceName + ")";
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+}


[32/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
new file mode 100644
index 0000000..c422b49
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
@@ -0,0 +1,782 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.planner.logical.SortNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.tajo.storage.RawFile.RawFileAppender;
+import static org.apache.tajo.storage.RawFile.RawFileScanner;
+
+/**
+ * This external sort algorithm can be characterized by the followings:
+ *
+ * <ul>
+ *   <li>in-memory sort if input data size fits a sort buffer</li>
+ *   <li>k-way merge sort if input data size exceeds the size of sort buffer</li>
+ *   <li>parallel merge</li>
+ *   <li>final merge avoidance</li>
+ *   <li>Unbalance merge if needed</li>
+ * </ul>
+ */
+public class ExternalSortExec extends SortExec {
+  /** Class logger */
+  private static final Log LOG = LogFactory.getLog(ExternalSortExec.class);
+
+  private SortNode plan;
+  private final TableMeta meta;
+  /** the defaultFanout of external sort */
+  private final int defaultFanout;
+  /** It's the size of in-memory table. If memory consumption exceeds it, store the memory table into a disk. */
+  private long sortBufferBytesNum;
+  /** the number of available cores */
+  private final int allocatedCoreNum;
+  /** If there are available multiple cores, it tries parallel merge. */
+  private ExecutorService executorService;
+  /** used for in-memory sort of each chunk. */
+  private List<Tuple> inMemoryTable;
+  /** temporal dir */
+  private final Path sortTmpDir;
+  /** It enables round-robin disks allocation */
+  private final LocalDirAllocator localDirAllocator;
+  /** local file system */
+  private final RawLocalFileSystem localFS;
+  /** final output files which are used for cleaning */
+  private List<Path> finalOutputFiles = null;
+  /** for directly merging sorted inputs */
+  private List<Path> mergedInputPaths = null;
+
+  ///////////////////////////////////////////////////
+  // transient variables
+  ///////////////////////////////////////////////////
+  /** already sorted or not */
+  private boolean sorted = false;
+  /** a flag to point whether sorted data resides in memory or not */
+  private boolean memoryResident = true;
+  /** the final result */
+  private Scanner result;
+  /** total bytes of input data */
+  private long sortAndStoredBytes;
+
+  private ExternalSortExec(final TaskAttemptContext context, final AbstractStorageManager sm, final SortNode plan)
+      throws PhysicalPlanningException {
+    super(context, plan.getInSchema(), plan.getOutSchema(), null, plan.getSortKeys());
+
+    this.plan = plan;
+    this.meta = CatalogUtil.newTableMeta(StoreType.ROWFILE);
+
+    this.defaultFanout = context.getConf().getIntVar(ConfVars.EXECUTOR_EXTERNAL_SORT_FANOUT);
+    if (defaultFanout < 2) {
+      throw new PhysicalPlanningException(ConfVars.EXECUTOR_EXTERNAL_SORT_FANOUT.varname + " cannot be lower than 2");
+    }
+    // TODO - sort buffer and core num should be changed to use the allocated container resource.
+    this.sortBufferBytesNum = context.getConf().getLongVar(ConfVars.EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE) * 1048576L;
+    this.allocatedCoreNum = context.getConf().getIntVar(ConfVars.EXECUTOR_EXTERNAL_SORT_THREAD_NUM);
+    this.executorService = Executors.newFixedThreadPool(this.allocatedCoreNum);
+    this.inMemoryTable = new ArrayList<Tuple>(100000);
+
+    this.sortTmpDir = getExecutorTmpDir();
+    localDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
+    localFS = new RawLocalFileSystem();
+  }
+
+  public ExternalSortExec(final TaskAttemptContext context,
+                          final AbstractStorageManager sm, final SortNode plan,
+                          final CatalogProtos.FragmentProto[] fragments) throws PhysicalPlanningException {
+    this(context, sm, plan);
+
+    mergedInputPaths = TUtil.newList();
+    for (CatalogProtos.FragmentProto proto : fragments) {
+      FileFragment fragment = FragmentConvertor.convert(FileFragment.class, proto);
+      mergedInputPaths.add(fragment.getPath());
+    }
+  }
+
+  public ExternalSortExec(final TaskAttemptContext context,
+                          final AbstractStorageManager sm, final SortNode plan, final PhysicalExec child)
+      throws IOException {
+    this(context, sm, plan);
+    setChild(child);
+  }
+
+  @VisibleForTesting
+  public void setSortBufferBytesNum(int sortBufferBytesNum) {
+    this.sortBufferBytesNum = sortBufferBytesNum;
+  }
+
+  public void init() throws IOException {
+    inputStats = new TableStats();
+    super.init();
+  }
+
+  public SortNode getPlan() {
+    return this.plan;
+  }
+
+  /**
+   * Sort a tuple block and store them into a chunk file
+   */
+  private Path sortAndStoreChunk(int chunkId, List<Tuple> tupleBlock)
+      throws IOException {
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.RAW);
+    int rowNum = tupleBlock.size();
+
+    long sortStart = System.currentTimeMillis();
+    Collections.sort(tupleBlock, getComparator());
+    long sortEnd = System.currentTimeMillis();
+
+    long chunkWriteStart = System.currentTimeMillis();
+    Path outputPath = getChunkPathForWrite(0, chunkId);
+    final RawFileAppender appender = new RawFileAppender(context.getConf(), inSchema, meta, outputPath);
+    appender.init();
+    for (Tuple t : tupleBlock) {
+      appender.addTuple(t);
+    }
+    appender.close();
+    tupleBlock.clear();
+    long chunkWriteEnd = System.currentTimeMillis();
+
+
+    info(LOG, "Chunk #" + chunkId + " sort and written (" +
+        FileUtil.humanReadableByteCount(appender.getOffset(), false) + " bytes, " + rowNum + " rows, " +
+        ", sort time: " + (sortEnd - sortStart) + " msec, " +
+        "write time: " + (chunkWriteEnd - chunkWriteStart) + " msec)");
+    return outputPath;
+  }
+
+  /**
+   * It divides all tuples into a number of chunks, then sort for each chunk.
+   *
+   * @return All paths of chunks
+   * @throws java.io.IOException
+   */
+  private List<Path> sortAndStoreAllChunks() throws IOException {
+    Tuple tuple;
+    long memoryConsumption = 0;
+    List<Path> chunkPaths = TUtil.newList();
+
+    int chunkId = 0;
+    long runStartTime = System.currentTimeMillis();
+    while ((tuple = child.next()) != null) { // partition sort start
+      Tuple vtuple = new VTuple(tuple);
+      inMemoryTable.add(vtuple);
+      memoryConsumption += MemoryUtil.calculateMemorySize(vtuple);
+
+      if (memoryConsumption > sortBufferBytesNum) {
+        long runEndTime = System.currentTimeMillis();
+        info(LOG, chunkId + " run loading time: " + (runEndTime - runStartTime) + " msec");
+        runStartTime = runEndTime;
+
+        info(LOG, "Memory consumption exceeds " + sortBufferBytesNum + " bytes");
+        memoryResident = false;
+
+        chunkPaths.add(sortAndStoreChunk(chunkId, inMemoryTable));
+
+        memoryConsumption = 0;
+        chunkId++;
+
+        // When the volume of sorting data once exceed the size of sort buffer,
+        // the total progress of this external sort is divided into two parts.
+        // In contrast, if the data fits in memory, the progress is only one part.
+        //
+        // When the progress is divided into two parts, the first part sorts tuples on memory and stores them
+        // into a chunk. The second part merges stored chunks into fewer chunks, and it continues until the number
+        // of merged chunks is fewer than the default fanout.
+        //
+        // The fact that the code reach here means that the first chunk has been just stored.
+        // That is, the progress was divided into two parts.
+        // So, it multiply the progress of the children operator and 0.5f.
+        progress = child.getProgress() * 0.5f;
+      }
+    }
+
+    if (inMemoryTable.size() > 0) { // if there are at least one or more input tuples
+      if (!memoryResident) { // check if data exceeds a sort buffer. If so, it store the remain data into a chunk.
+        if (inMemoryTable.size() > 0) {
+          long start = System.currentTimeMillis();
+          int rowNum = inMemoryTable.size();
+          chunkPaths.add(sortAndStoreChunk(chunkId, inMemoryTable));
+          long end = System.currentTimeMillis();
+          info(LOG, "Last Chunk #" + chunkId + " " + rowNum + " rows written (" + (end - start) + " msec)");
+        }
+      } else { // this case means that all data does not exceed a sort buffer
+        Collections.sort(inMemoryTable, getComparator());
+      }
+    }
+
+    // get total loaded (or stored) bytes and total row numbers
+    TableStats childTableStats = child.getInputStats();
+    if (childTableStats != null) {
+      sortAndStoredBytes = childTableStats.getNumBytes();
+    }
+    return chunkPaths;
+  }
+
+  /**
+   * Get a local path from all temporal paths in round-robin manner.
+   */
+  private synchronized Path getChunkPathForWrite(int level, int chunkId) throws IOException {
+    return localDirAllocator.getLocalPathForWrite(sortTmpDir + "/" + level +"_" + chunkId, context.getConf());
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+
+    if (!sorted) { // if not sorted, first sort all data
+
+      // if input files are given, it starts merging directly.
+      if (mergedInputPaths != null) {
+        try {
+          this.result = externalMergeAndSort(mergedInputPaths);
+        } catch (Exception e) {
+          throw new PhysicalPlanningException(e);
+        }
+      } else {
+        // Try to sort all data, and store them as multiple chunks if memory exceeds
+        long startTimeOfChunkSplit = System.currentTimeMillis();
+        List<Path> chunks = sortAndStoreAllChunks();
+        long endTimeOfChunkSplit = System.currentTimeMillis();
+        info(LOG, "Chunks creation time: " + (endTimeOfChunkSplit - startTimeOfChunkSplit) + " msec");
+
+        if (memoryResident) { // if all sorted data reside in a main-memory table.
+          this.result = new MemTableScanner();
+        } else { // if input data exceeds main-memory at least once
+
+          try {
+            this.result = externalMergeAndSort(chunks);
+          } catch (Exception e) {
+            throw new PhysicalPlanningException(e);
+          }
+
+        }
+      }
+
+      sorted = true;
+      result.init();
+
+      // if loaded and sorted, we assume that it proceeds the half of one entire external sort operation.
+      progress = 0.5f;
+    }
+
+    return result.next();
+  }
+
+  private int calculateFanout(int remainInputChunks, int intputNum, int outputNum, int startIdx) {
+    int computedFanout = Math.min(remainInputChunks, defaultFanout);
+
+    // Why should we detect an opportunity for unbalanced merge?
+    //
+    // Assume that a fanout is given by 8 and there are 10 chunks.
+    // If we firstly merge 3 chunks into one chunk, there remain only 8 chunks.
+    // Then, we can just finish the merge phase even though we don't complete merge phase on all chunks.
+    if (checkIfCanBeUnbalancedMerged(intputNum - (startIdx + computedFanout), outputNum + 1)) {
+      int candidateFanout = computedFanout;
+      while(checkIfCanBeUnbalancedMerged(intputNum - (startIdx + candidateFanout), outputNum + 1)) {
+        candidateFanout--;
+      }
+      int beforeFanout = computedFanout;
+      if (computedFanout > candidateFanout + 1) {
+        computedFanout = candidateFanout + 1;
+        info(LOG, "Fanout reduced for unbalanced merge: " + beforeFanout + " -> " + computedFanout);
+      }
+    }
+
+    return computedFanout;
+  }
+
+  private Scanner externalMergeAndSort(List<Path> chunks)
+      throws IOException, ExecutionException, InterruptedException {
+    int level = 0;
+    final List<Path> inputFiles = TUtil.newList(chunks);
+    final List<Path> outputFiles = TUtil.newList();
+    int remainRun = inputFiles.size();
+    int chunksSize = chunks.size();
+
+    long mergeStart = System.currentTimeMillis();
+
+    // continue until the remain runs are larger than defaultFanout
+    while (remainRun > defaultFanout) {
+
+      // reset outChunkId
+      int remainInputRuns = inputFiles.size();
+      int outChunkId = 0;
+      int outputFileNum = 0;
+      List<Future> futures = TUtil.newList();
+      // the number of files being merged in threads.
+      List<Integer> numberOfMergingFiles = TUtil.newList();
+
+      for (int startIdx = 0; startIdx < inputFiles.size();) {
+
+        // calculate proper fanout
+        int fanout = calculateFanout(remainInputRuns, inputFiles.size(), outputFileNum, startIdx);
+        // how many files are merged in ith thread?
+        numberOfMergingFiles.add(fanout);
+        // launch a merger runner
+        futures.add(executorService.submit(
+            new KWayMergerCaller(level, outChunkId++, inputFiles, startIdx, fanout, false)));
+        outputFileNum++;
+
+        startIdx += fanout;
+        remainInputRuns = inputFiles.size() - startIdx;
+
+        // If unbalanced merge is available, it finishes the merge phase earlier.
+        if (checkIfCanBeUnbalancedMerged(remainInputRuns, outputFileNum)) {
+          info(LOG, "Unbalanced merge possibility detected: number of remain input (" + remainInputRuns
+              + ") and output files (" + outputFileNum + ") <= " + defaultFanout);
+
+          List<Path> switched = TUtil.newList();
+          // switch the remain inputs to the next outputs
+          for (int j = startIdx; j < inputFiles.size(); j++) {
+            switched.add(inputFiles.get(j));
+          }
+          inputFiles.removeAll(switched);
+          outputFiles.addAll(switched);
+
+          break;
+        }
+      }
+
+      // wait for all sort runners
+      int finishedMerger = 0;
+      int index = 0;
+      for (Future<Path> future : futures) {
+        outputFiles.add(future.get());
+        // Getting the number of merged files
+        finishedMerger += numberOfMergingFiles.get(index++);
+        // progress = (# number of merged files / total number of files) * 0.5;
+        progress = ((float)finishedMerger/(float)chunksSize) * 0.5f;
+      }
+
+      // delete merged intermediate files
+      for (Path path : inputFiles) {
+        localFS.delete(path, true);
+      }
+      info(LOG, inputFiles.size() + " merged intermediate files deleted");
+
+      // switch input files to output files, and then clear outputFiles
+      inputFiles.clear();
+      inputFiles.addAll(outputFiles);
+      remainRun = inputFiles.size();
+      outputFiles.clear();
+      level++;
+    }
+
+    long mergeEnd = System.currentTimeMillis();
+    info(LOG, "Total merge time: " + (mergeEnd - mergeStart) + " msec");
+
+    // final result
+    finalOutputFiles = inputFiles;
+
+    result = createFinalMerger(inputFiles);
+    return result;
+  }
+
+  /**
+   * Merge Thread
+   */
+  private class KWayMergerCaller implements Callable<Path> {
+    final int level;
+    final int nextRunId;
+    final List<Path> inputFiles;
+    final int startIdx;
+    final int mergeFanout;
+    final boolean updateInputStats;
+
+    public KWayMergerCaller(final int level, final int nextRunId, final List<Path> inputFiles,
+                            final int startIdx, final int mergeFanout, final boolean updateInputStats) {
+      this.level = level;
+      this.nextRunId = nextRunId;
+      this.inputFiles = inputFiles;
+      this.startIdx = startIdx;
+      this.mergeFanout = mergeFanout;
+      this.updateInputStats = updateInputStats;
+    }
+
+    @Override
+    public Path call() throws Exception {
+      final Path outputPath = getChunkPathForWrite(level + 1, nextRunId);
+      info(LOG, mergeFanout + " files are being merged to an output file " + outputPath.getName());
+      long mergeStartTime = System.currentTimeMillis();
+      final RawFileAppender output = new RawFileAppender(context.getConf(), inSchema, meta, outputPath);
+      output.init();
+      final Scanner merger = createKWayMerger(inputFiles, startIdx, mergeFanout);
+      merger.init();
+      Tuple mergeTuple;
+      while((mergeTuple = merger.next()) != null) {
+        output.addTuple(mergeTuple);
+      }
+      merger.close();
+      output.close();
+      long mergeEndTime = System.currentTimeMillis();
+      info(LOG, outputPath.getName() + " is written to a disk. ("
+          + FileUtil.humanReadableByteCount(output.getOffset(), false)
+          + " bytes, " + (mergeEndTime - mergeStartTime) + " msec)");
+      return outputPath;
+    }
+  }
+
+  /**
+   * It checks if unbalanced merge is possible.
+   */
+  private boolean checkIfCanBeUnbalancedMerged(int remainInputNum, int outputNum) {
+    return (remainInputNum + outputNum) <= defaultFanout;
+  }
+
+  /**
+   * Create a merged file scanner or k-way merge scanner.
+   */
+  private Scanner createFinalMerger(List<Path> inputs) throws IOException {
+    if (inputs.size() == 1) {
+      this.result = getFileScanner(inputs.get(0));
+    } else {
+      this.result = createKWayMerger(inputs, 0, inputs.size());
+    }
+    return result;
+  }
+
+  private Scanner getFileScanner(Path path) throws IOException {
+    return new RawFileScanner(context.getConf(), plan.getInSchema(), meta, path);
+  }
+
+  private Scanner createKWayMerger(List<Path> inputs, final int startChunkId, final int num) throws IOException {
+    final Scanner [] sources = new Scanner[num];
+    for (int i = 0; i < num; i++) {
+      sources[i] = getFileScanner(inputs.get(startChunkId + i));
+    }
+
+    return createKWayMergerInternal(sources, 0, num);
+  }
+
+  private Scanner createKWayMergerInternal(final Scanner [] sources, final int startIdx, final int num)
+      throws IOException {
+    if (num > 1) {
+      final int mid = (int) Math.ceil((float)num / 2);
+      return new PairWiseMerger(
+          createKWayMergerInternal(sources, startIdx, mid),
+          createKWayMergerInternal(sources, startIdx + mid, num - mid));
+    } else {
+      return sources[startIdx];
+    }
+  }
+
+  private class MemTableScanner implements Scanner {
+    Iterator<Tuple> iterator;
+
+    // for input stats
+    float scannerProgress;
+    int numRecords;
+    int totalRecords;
+    TableStats scannerTableStats;
+
+    @Override
+    public void init() throws IOException {
+      iterator = inMemoryTable.iterator();
+
+      totalRecords = inMemoryTable.size();
+      scannerProgress = 0.0f;
+      numRecords = 0;
+
+      // it will be returned as the final stats
+      scannerTableStats = new TableStats();
+      scannerTableStats.setNumBytes(sortAndStoredBytes);
+      scannerTableStats.setReadBytes(sortAndStoredBytes);
+      scannerTableStats.setNumRows(totalRecords);
+    }
+
+    @Override
+    public Tuple next() throws IOException {
+      if (iterator.hasNext()) {
+        numRecords++;
+        return iterator.next();
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public void reset() throws IOException {
+      init();
+    }
+
+    @Override
+    public void close() throws IOException {
+      iterator = null;
+      scannerProgress = 1.0f;
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return false;
+    }
+
+    @Override
+    public void setTarget(Column[] targets) {
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public void setSearchCondition(Object expr) {
+    }
+
+    @Override
+    public boolean isSplittable() {
+      return false;
+    }
+
+    @Override
+    public Schema getSchema() {
+      return null;
+    }
+
+    @Override
+    public float getProgress() {
+      if (iterator != null && numRecords > 0) {
+        return (float)numRecords / (float)totalRecords;
+
+      } else { // if an input is empty
+        return scannerProgress;
+      }
+    }
+
+    @Override
+    public TableStats getInputStats() {
+      return scannerTableStats;
+    }
+  }
+
+  /**
+   * Two-way merger scanner that reads two input sources and outputs one output tuples sorted in some order.
+   */
+  private class PairWiseMerger implements Scanner {
+    private Scanner leftScan;
+    private Scanner rightScan;
+
+    private Tuple leftTuple;
+    private Tuple rightTuple;
+
+    private final Comparator<Tuple> comparator = getComparator();
+
+    private float mergerProgress;
+    private TableStats mergerInputStats;
+
+    public PairWiseMerger(Scanner leftScanner, Scanner rightScanner) throws IOException {
+      this.leftScan = leftScanner;
+      this.rightScan = rightScanner;
+    }
+
+    @Override
+    public void init() throws IOException {
+      leftScan.init();
+      rightScan.init();
+
+      leftTuple = leftScan.next();
+      rightTuple = rightScan.next();
+
+      mergerInputStats = new TableStats();
+      mergerProgress = 0.0f;
+    }
+
+    public Tuple next() throws IOException {
+      Tuple outTuple;
+      if (leftTuple != null && rightTuple != null) {
+        if (comparator.compare(leftTuple, rightTuple) < 0) {
+          outTuple = leftTuple;
+          leftTuple = leftScan.next();
+        } else {
+          outTuple = rightTuple;
+          rightTuple = rightScan.next();
+        }
+        return outTuple;
+      }
+
+      if (leftTuple == null) {
+        outTuple = rightTuple;
+        rightTuple = rightScan.next();
+      } else {
+        outTuple = leftTuple;
+        leftTuple = leftScan.next();
+      }
+      return outTuple;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      leftScan.reset();
+      rightScan.reset();
+      init();
+    }
+
+    public void close() throws IOException {
+      IOUtils.cleanup(LOG, leftScan, rightScan);
+      getInputStats();
+      leftScan = null;
+      rightScan = null;
+      mergerProgress = 1.0f;
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return false;
+    }
+
+    @Override
+    public void setTarget(Column[] targets) {
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public void setSearchCondition(Object expr) {
+    }
+
+    @Override
+    public boolean isSplittable() {
+      return false;
+    }
+
+    @Override
+    public Schema getSchema() {
+      return inSchema;
+    }
+
+    @Override
+    public float getProgress() {
+      if (leftScan == null) {
+        return mergerProgress;
+      }
+      return leftScan.getProgress() * 0.5f + rightScan.getProgress() * 0.5f;
+    }
+
+    @Override
+    public TableStats getInputStats() {
+      if (leftScan == null) {
+        return mergerInputStats;
+      }
+      TableStats leftInputStats = leftScan.getInputStats();
+      mergerInputStats.setNumBytes(0);
+      mergerInputStats.setReadBytes(0);
+      mergerInputStats.setNumRows(0);
+
+      if (leftInputStats != null) {
+        mergerInputStats.setNumBytes(leftInputStats.getNumBytes());
+        mergerInputStats.setReadBytes(leftInputStats.getReadBytes());
+        mergerInputStats.setNumRows(leftInputStats.getNumRows());
+      }
+
+      TableStats rightInputStats = rightScan.getInputStats();
+      if (rightInputStats != null) {
+        mergerInputStats.setNumBytes(mergerInputStats.getNumBytes() + rightInputStats.getNumBytes());
+        mergerInputStats.setReadBytes(mergerInputStats.getReadBytes() + rightInputStats.getReadBytes());
+        mergerInputStats.setNumRows(mergerInputStats.getNumRows() + rightInputStats.getNumRows());
+      }
+
+      return mergerInputStats;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (result != null) {
+      result.close();
+      try {
+        inputStats = (TableStats)result.getInputStats().clone();
+      } catch (CloneNotSupportedException e) {
+        LOG.warn(e.getMessage());
+      }
+      result = null;
+    }
+
+    if (finalOutputFiles != null) {
+      for (Path path : finalOutputFiles) {
+        localFS.delete(path, true);
+      }
+    }
+
+    if(inMemoryTable != null){
+      inMemoryTable.clear();
+      inMemoryTable = null;
+    }
+
+    if(executorService != null){
+      executorService.shutdown();
+      executorService = null;
+    }
+
+    plan = null;
+    super.close();
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    if (result != null) {
+      result.reset();
+      progress = 0.5f;
+    }
+  }
+
+  @Override
+  public float getProgress() {
+    if (result != null) {
+      return progress + result.getProgress() * 0.5f;
+    } else {
+      return progress;
+    }
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    if (result != null) {
+      return result.getInputStats();
+    } else {
+      return inputStats;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java
new file mode 100644
index 0000000..a31ad90
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java
@@ -0,0 +1,115 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * This is the hash-based GroupBy Operator.
+ */
+public class HashAggregateExec extends AggregationExec {
+  private Tuple tuple = null;
+  private Map<Tuple, FunctionContext[]> hashTable;
+  private boolean computed = false;
+  private Iterator<Entry<Tuple, FunctionContext []>> iterator = null;
+
+  public HashAggregateExec(TaskAttemptContext ctx, GroupbyNode plan, PhysicalExec subOp) throws IOException {
+    super(ctx, plan, subOp);
+    hashTable = new HashMap<Tuple, FunctionContext []>(100000);
+    this.tuple = new VTuple(plan.getOutSchema().size());
+  }
+
+  private void compute() throws IOException {
+    Tuple tuple;
+    Tuple keyTuple;
+    while((tuple = child.next()) != null && !context.isStopped()) {
+      keyTuple = new VTuple(groupingKeyIds.length);
+      // build one key tuple
+      for(int i = 0; i < groupingKeyIds.length; i++) {
+        keyTuple.put(i, tuple.get(groupingKeyIds[i]));
+      }
+      
+      if(hashTable.containsKey(keyTuple)) {
+        FunctionContext [] contexts = hashTable.get(keyTuple);
+        for(int i = 0; i < aggFunctions.length; i++) {
+          aggFunctions[i].merge(contexts[i], inSchema, tuple);
+        }
+      } else { // if the key occurs firstly
+        FunctionContext contexts [] = new FunctionContext[aggFunctionsNum];
+        for(int i = 0; i < aggFunctionsNum; i++) {
+          contexts[i] = aggFunctions[i].newContext();
+          aggFunctions[i].merge(contexts[i], inSchema, tuple);
+        }
+        hashTable.put(keyTuple, contexts);
+      }
+    }
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if(!computed) {
+      compute();
+      iterator = hashTable.entrySet().iterator();
+      computed = true;
+    }
+
+    FunctionContext [] contexts;
+
+    if (iterator.hasNext()) {
+      Entry<Tuple, FunctionContext []> entry = iterator.next();
+      Tuple keyTuple = entry.getKey();
+      contexts =  entry.getValue();
+
+      int tupleIdx = 0;
+      for (; tupleIdx < groupingKeyNum; tupleIdx++) {
+        tuple.put(tupleIdx, keyTuple.get(tupleIdx));
+      }
+      for (int funcIdx = 0; funcIdx < aggFunctionsNum; funcIdx++, tupleIdx++) {
+        tuple.put(tupleIdx, aggFunctions[funcIdx].terminate(contexts[funcIdx]));
+      }
+
+      return tuple;
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {    
+    iterator = hashTable.entrySet().iterator();
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    hashTable.clear();
+    hashTable = null;
+    iterator = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashBasedColPartitionStoreExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashBasedColPartitionStoreExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashBasedColPartitionStoreExec.java
new file mode 100644
index 0000000..df32d0b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashBasedColPartitionStoreExec.java
@@ -0,0 +1,132 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.planner.logical.StoreTableNode;
+import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.StorageManagerFactory;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This class is a physical operator to store at column partitioned table.
+ */
+public class HashBasedColPartitionStoreExec extends ColPartitionStoreExec {
+  private static Log LOG = LogFactory.getLog(HashBasedColPartitionStoreExec.class);
+
+  private final Map<String, Appender> appenderMap = new HashMap<String, Appender>();
+
+  public HashBasedColPartitionStoreExec(TaskAttemptContext context, StoreTableNode plan, PhysicalExec child)
+      throws IOException {
+    super(context, plan, child);
+  }
+
+  public void init() throws IOException {
+    super.init();
+  }
+
+  private Appender getAppender(String partition) throws IOException {
+    Appender appender = appenderMap.get(partition);
+
+    if (appender == null) {
+      Path dataFile = getDataFile(partition);
+      FileSystem fs = dataFile.getFileSystem(context.getConf());
+
+      if (fs.exists(dataFile.getParent())) {
+        LOG.info("Path " + dataFile.getParent() + " already exists!");
+      } else {
+        fs.mkdirs(dataFile.getParent());
+        LOG.info("Add subpartition path directory :" + dataFile.getParent());
+      }
+
+      if (fs.exists(dataFile)) {
+        LOG.info("File " + dataFile + " already exists!");
+        FileStatus status = fs.getFileStatus(dataFile);
+        LOG.info("File size: " + status.getLen());
+      }
+
+      appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta, outSchema, dataFile);
+      appender.enableStats();
+      appender.init();
+      appenderMap.put(partition, appender);
+    } else {
+      appender = appenderMap.get(partition);
+    }
+    return appender;
+  }
+
+  /* (non-Javadoc)
+   * @see PhysicalExec#next()
+   */
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple;
+    StringBuilder sb = new StringBuilder();
+    while((tuple = child.next()) != null) {
+      // set subpartition directory name
+      sb.delete(0, sb.length());
+      if (keyIds != null) {
+        for(int i = 0; i < keyIds.length; i++) {
+          Datum datum = tuple.get(keyIds[i]);
+          if(i > 0)
+            sb.append("/");
+          sb.append(keyNames[i]).append("=");
+          sb.append(datum.asChars());
+        }
+      }
+
+      // add tuple
+      Appender appender = getAppender(sb.toString());
+      appender.addTuple(tuple);
+    }
+
+    List<TableStats> statSet = new ArrayList<TableStats>();
+    for (Map.Entry<String, Appender> entry : appenderMap.entrySet()) {
+      Appender app = entry.getValue();
+      app.flush();
+      app.close();
+      statSet.add(app.getStats());
+    }
+
+    // Collect and aggregated statistics data
+    TableStats aggregated = StatisticsUtil.aggregateTableStat(statSet);
+    context.setResultStats(aggregated);
+
+    return null;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    // nothing to do
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java
new file mode 100644
index 0000000..65ebe2f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java
@@ -0,0 +1,252 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.*;
+
+
+public class HashFullOuterJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  protected JoinNode plan;
+  protected EvalNode joinQual;
+
+  protected List<Column[]> joinKeyPairs;
+
+  // temporal tuples and states for nested loop join
+  protected boolean first = true;
+  protected FrameTuple frameTuple;
+  protected Tuple outTuple = null;
+  protected Map<Tuple, List<Tuple>> tupleSlots;
+  protected Iterator<Tuple> iterator = null;
+  protected Tuple leftTuple;
+  protected Tuple leftKeyTuple;
+
+  protected int [] leftKeyList;
+  protected int [] rightKeyList;
+
+  protected boolean finished = false;
+  protected boolean shouldGetLeftTuple = true;
+
+  // projection
+  protected final Projector projector;
+
+  private int rightNumCols;
+  private int leftNumCols;
+  private Map<Tuple, Boolean> matched;
+
+  public HashFullOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer,
+                               PhysicalExec inner) {
+    super(context, SchemaUtil.merge(outer.getSchema(), inner.getSchema()),
+        plan.getOutSchema(), outer, inner);
+    this.plan = plan;
+    this.joinQual = plan.getJoinQual();
+    this.tupleSlots = new HashMap<Tuple, List<Tuple>>(10000);
+
+    // this hashmap mirrors the evolution of the tupleSlots, with the same keys. For each join key,
+    // we have a boolean flag, initially false (whether this join key had at least one match on the left operand)
+    this.matched = new HashMap<Tuple, Boolean>(10000);
+
+    // HashJoin only can manage equi join key pairs.
+    this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, outer.getSchema(), inner.getSchema(),
+        false);
+
+    leftKeyList = new int[joinKeyPairs.size()];
+    rightKeyList = new int[joinKeyPairs.size()];
+
+    for (int i = 0; i < joinKeyPairs.size(); i++) {
+      leftKeyList[i] = outer.getSchema().getColumnId(joinKeyPairs.get(i)[0].getQualifiedName());
+    }
+
+    for (int i = 0; i < joinKeyPairs.size(); i++) {
+      rightKeyList[i] = inner.getSchema().getColumnId(joinKeyPairs.get(i)[1].getQualifiedName());
+    }
+
+    // for projection
+    this.projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+    leftKeyTuple = new VTuple(leftKeyList.length);
+
+    leftNumCols = outer.getSchema().size();
+    rightNumCols = inner.getSchema().size();
+  }
+
+  protected void getKeyLeftTuple(final Tuple outerTuple, Tuple keyTuple) {
+    for (int i = 0; i < leftKeyList.length; i++) {
+      keyTuple.put(i, outerTuple.get(leftKeyList[i]));
+    }
+  }
+
+  public Tuple getNextUnmatchedRight() {
+
+    List<Tuple> newValue;
+    Tuple returnedTuple;
+    // get a keyTUple from the matched hashmap with a boolean false value
+    for(Tuple aKeyTuple : matched.keySet()) {
+      if(matched.get(aKeyTuple) == false) {
+        newValue = tupleSlots.get(aKeyTuple);
+        returnedTuple = newValue.remove(0);
+        tupleSlots.put(aKeyTuple, newValue);
+
+        // after taking the last element from the list in tupleSlots, set flag true in matched as well
+        if(newValue.isEmpty()){
+          matched.put(aKeyTuple, true);
+        }
+
+        return returnedTuple;
+      }
+    }
+    return null;
+  }
+
+  public Tuple next() throws IOException {
+    if (first) {
+      loadRightToHashTable();
+    }
+
+    Tuple rightTuple;
+    boolean found = false;
+
+    while(!finished) {
+      if (shouldGetLeftTuple) { // initially, it is true.
+        // getting new outer
+        leftTuple = leftChild.next(); // it comes from a disk
+        if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed.
+          // in this stage we can begin outputing tuples from the right operand (which were before in tupleSlots) null padded on the left side
+          Tuple unmatchedRightTuple = getNextUnmatchedRight();
+          if( unmatchedRightTuple == null) {
+            finished = true;
+            outTuple = null;
+            return null;
+          } else {
+            Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(leftNumCols);
+            frameTuple.set(nullPaddedTuple, unmatchedRightTuple);
+            projector.eval(frameTuple, outTuple);
+
+            return outTuple;
+          }
+        }
+
+        // getting corresponding right
+        getKeyLeftTuple(leftTuple, leftKeyTuple); // get a left key tuple
+        List<Tuple> rightTuples = tupleSlots.get(leftKeyTuple);
+        if (rightTuples != null) { // found right tuples on in-memory hash table.
+          iterator = rightTuples.iterator();
+          shouldGetLeftTuple = false;
+        } else {
+          //this left tuple doesn't have a match on the right.But full outer join => we should keep it anyway
+          //output a tuple with the nulls padded rightTuple
+          Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols);
+          frameTuple.set(leftTuple, nullPaddedTuple);
+          projector.eval(frameTuple, outTuple);
+          // we simulate we found a match, which is exactly the null padded one
+          shouldGetLeftTuple = true;
+          return outTuple;
+        }
+      }
+
+      // getting a next right tuple on in-memory hash table.
+      rightTuple = iterator.next();
+      frameTuple.set(leftTuple, rightTuple); // evaluate a join condition on both tuples
+
+      if (joinQual.eval(inSchema, frameTuple).isTrue()) { // if both tuples are joinable
+        projector.eval(frameTuple, outTuple);
+        found = true;
+        getKeyLeftTuple(leftTuple, leftKeyTuple);
+        matched.put(leftKeyTuple, true);
+      }
+
+      if (!iterator.hasNext()) { // no more right tuples for this hash key
+        shouldGetLeftTuple = true;
+      }
+
+      if (found) {
+        break;
+      }
+    }
+    return outTuple;
+  }
+
+  protected void loadRightToHashTable() throws IOException {
+    Tuple tuple;
+    Tuple keyTuple;
+
+    while ((tuple = rightChild.next()) != null) {
+      keyTuple = new VTuple(joinKeyPairs.size());
+      for (int i = 0; i < rightKeyList.length; i++) {
+        keyTuple.put(i, tuple.get(rightKeyList[i]));
+      }
+
+      List<Tuple> newValue = tupleSlots.get(keyTuple);
+      if (newValue != null) {
+        newValue.add(tuple);
+      } else {
+        newValue = new ArrayList<Tuple>();
+        newValue.add(tuple);
+        tupleSlots.put(keyTuple, newValue);
+        matched.put(keyTuple,false);
+      }
+    }
+    first = false;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+
+    tupleSlots.clear();
+    first = true;
+
+    finished = false;
+    iterator = null;
+    shouldGetLeftTuple = true;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    tupleSlots.clear();
+    matched.clear();
+    tupleSlots = null;
+    matched = null;
+    iterator = null;
+    plan = null;
+    joinQual = null;
+  }
+
+  public JoinNode getPlan() {
+    return this.plan;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java
new file mode 100644
index 0000000..dea0340
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java
@@ -0,0 +1,202 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.*;
+
+public class HashJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  protected JoinNode plan;
+  protected EvalNode joinQual;
+
+  protected List<Column[]> joinKeyPairs;
+
+  // temporal tuples and states for nested loop join
+  protected boolean first = true;
+  protected FrameTuple frameTuple;
+  protected Tuple outTuple = null;
+  protected Map<Tuple, List<Tuple>> tupleSlots;
+  protected Iterator<Tuple> iterator = null;
+  protected Tuple leftTuple;
+  protected Tuple leftKeyTuple;
+
+  protected int [] leftKeyList;
+  protected int [] rightKeyList;
+
+  protected boolean finished = false;
+  protected boolean shouldGetLeftTuple = true;
+
+  // projection
+  protected final Projector projector;
+
+  public HashJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec,
+      PhysicalExec rightExec) {
+    super(context, SchemaUtil.merge(leftExec.getSchema(), rightExec.getSchema()), plan.getOutSchema(),
+        leftExec, rightExec);
+    this.plan = plan;
+    this.joinQual = plan.getJoinQual();
+    this.tupleSlots = new HashMap<Tuple, List<Tuple>>(100000);
+
+    // HashJoin only can manage equi join key pairs.
+    this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, leftExec.getSchema(),
+        rightExec.getSchema(), false);
+
+    leftKeyList = new int[joinKeyPairs.size()];
+    rightKeyList = new int[joinKeyPairs.size()];
+
+    for (int i = 0; i < joinKeyPairs.size(); i++) {
+      leftKeyList[i] = leftExec.getSchema().getColumnId(joinKeyPairs.get(i)[0].getQualifiedName());
+    }
+
+    for (int i = 0; i < joinKeyPairs.size(); i++) {
+      rightKeyList[i] = rightExec.getSchema().getColumnId(joinKeyPairs.get(i)[1].getQualifiedName());
+    }
+
+    // for projection
+    this.projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+    leftKeyTuple = new VTuple(leftKeyList.length);
+  }
+
+  protected void getKeyLeftTuple(final Tuple outerTuple, Tuple keyTuple) {
+    for (int i = 0; i < leftKeyList.length; i++) {
+      keyTuple.put(i, outerTuple.get(leftKeyList[i]));
+    }
+  }
+
+  long scanStartTime = 0;
+  public Tuple next() throws IOException {
+    if (first) {
+      loadRightToHashTable();
+      scanStartTime = System.currentTimeMillis();
+    }
+
+    Tuple rightTuple;
+    boolean found = false;
+
+    while(!finished) {
+      if (shouldGetLeftTuple) { // initially, it is true.
+        // getting new outer
+        leftTuple = leftChild.next(); // it comes from a disk
+        if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed.
+          finished = true;
+          return null;
+        }
+
+        // getting corresponding right
+        getKeyLeftTuple(leftTuple, leftKeyTuple); // get a left key tuple
+        List<Tuple> rightTuples = tupleSlots.get(leftKeyTuple);
+        if (rightTuples != null) { // found right tuples on in-memory hash table.
+          iterator = rightTuples.iterator();
+          shouldGetLeftTuple = false;
+        } else {
+          shouldGetLeftTuple = true;
+          continue;
+        }
+      }
+
+      // getting a next right tuple on in-memory hash table.
+      rightTuple = iterator.next();
+      frameTuple.set(leftTuple, rightTuple); // evaluate a join condition on both tuples
+      if (joinQual.eval(inSchema, frameTuple).isTrue()) { // if both tuples are joinable
+        projector.eval(frameTuple, outTuple);
+        found = true;
+      }
+
+      if (!iterator.hasNext()) { // no more right tuples for this hash key
+        shouldGetLeftTuple = true;
+      }
+
+      if (found) {
+        break;
+      }
+    }
+
+    return new VTuple(outTuple);
+  }
+
+  protected void loadRightToHashTable() throws IOException {
+    Tuple tuple;
+    Tuple keyTuple;
+
+    while ((tuple = rightChild.next()) != null) {
+      keyTuple = new VTuple(joinKeyPairs.size());
+      for (int i = 0; i < rightKeyList.length; i++) {
+        keyTuple.put(i, tuple.get(rightKeyList[i]));
+      }
+
+      List<Tuple> newValue = tupleSlots.get(keyTuple);
+
+      if (newValue != null) {
+        newValue.add(tuple);
+      } else {
+        newValue = new ArrayList<Tuple>();
+        newValue.add(tuple);
+        tupleSlots.put(keyTuple, newValue);
+      }
+    }
+
+    first = false;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+
+    tupleSlots.clear();
+    first = true;
+
+    finished = false;
+    iterator = null;
+    shouldGetLeftTuple = true;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    if (tupleSlots != null) {
+      tupleSlots.clear();
+      tupleSlots = null;
+    }
+
+    iterator = null;
+    plan = null;
+    joinQual = null;
+  }
+
+  public JoinNode getPlan() {
+    return this.plan;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java
new file mode 100644
index 0000000..50a1438
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Prepare a hash table of the NOT IN side of the join. Scan the FROM side table.
+ * For each tuple of the FROM side table, it tries to find a matched tuple from the hash table for the NOT INT side.
+ * If not found, it returns the tuple of the FROM side table with null padding.
+ */
+public class HashLeftAntiJoinExec extends HashJoinExec {
+  private Tuple rightNullTuple;
+
+  public HashLeftAntiJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec fromSideChild,
+                              PhysicalExec notInSideChild) {
+    super(context, plan, fromSideChild, notInSideChild);
+    // NUll Tuple
+    rightNullTuple = new VTuple(leftChild.outColumnNum);
+    for (int i = 0; i < leftChild.outColumnNum; i++) {
+      rightNullTuple.put(i, NullDatum.get());
+    }
+  }
+
+  /**
+   * The End of Tuple (EOT) condition is true only when no more tuple in the left relation (on disk).
+   * next() method finds the first unmatched tuple from both tables.
+   *
+   * For each left tuple, next() tries to find the right tuple from the hash table. If there is no hash bucket
+   * in the hash table. It returns a tuple. If next() find the hash bucket in the hash table, it reads tuples in
+   * the found bucket sequentially. If it cannot find tuple in the bucket, it returns a tuple.
+   *
+   * @return The tuple which is unmatched to a given join condition.
+   * @throws IOException
+   */
+  public Tuple next() throws IOException {
+    if (first) {
+      loadRightToHashTable();
+    }
+
+    Tuple rightTuple;
+    boolean notFound;
+
+    while(!finished) {
+
+      // getting new outer
+      leftTuple = leftChild.next(); // it comes from a disk
+      if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed.
+        finished = true;
+        return null;
+      }
+
+      // Try to find a hash bucket in in-memory hash table
+      getKeyLeftTuple(leftTuple, leftKeyTuple);
+      List<Tuple> rightTuples = tupleSlots.get(leftKeyTuple);
+      if (rightTuples != null) {
+        // if found, it gets a hash bucket from the hash table.
+        iterator = rightTuples.iterator();
+      } else {
+        // if not found, it returns a tuple.
+        frameTuple.set(leftTuple, rightNullTuple);
+        projector.eval(frameTuple, outTuple);
+        return outTuple;
+      }
+
+      // Reach here only when a hash bucket is found. Then, it checks all tuples in the found bucket.
+      // If it finds a matched tuple, it escapes the loop for all tuples in the hash bucket.
+      notFound = true;
+      while (notFound && iterator.hasNext()) {
+        rightTuple = iterator.next();
+        frameTuple.set(leftTuple, rightTuple);
+        if (joinQual.eval(inSchema, frameTuple).isTrue()) { // if the matched one is found
+          notFound = false;
+        }
+      }
+
+      if (notFound) { // if there is no matched tuple
+        frameTuple.set(leftTuple, rightNullTuple);
+        projector.eval(frameTuple, outTuple);
+        break;
+      }
+    }
+
+    return outTuple;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java
new file mode 100644
index 0000000..849dc38
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java
@@ -0,0 +1,212 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.*;
+
+
+public class HashLeftOuterJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  protected JoinNode plan;
+  protected EvalNode joinQual;
+
+  protected List<Column[]> joinKeyPairs;
+
+  // temporal tuples and states for nested loop join
+  protected boolean first = true;
+  protected FrameTuple frameTuple;
+  protected Tuple outTuple = null;
+  protected Map<Tuple, List<Tuple>> tupleSlots;
+  protected Iterator<Tuple> iterator = null;
+  protected Tuple leftTuple;
+  protected Tuple leftKeyTuple;
+
+  protected int [] leftKeyList;
+  protected int [] rightKeyList;
+
+  protected boolean finished = false;
+  protected boolean shouldGetLeftTuple = true;
+
+  // projection
+  protected Projector projector;
+
+  private int rightNumCols;
+  private static final Log LOG = LogFactory.getLog(HashLeftOuterJoinExec.class);
+
+  public HashLeftOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftChild,
+                               PhysicalExec rightChild) {
+    super(context, SchemaUtil.merge(leftChild.getSchema(), rightChild.getSchema()),
+        plan.getOutSchema(), leftChild, rightChild);
+    this.plan = plan;
+    this.joinQual = plan.getJoinQual();
+    this.tupleSlots = new HashMap<Tuple, List<Tuple>>(10000);
+
+    // HashJoin only can manage equi join key pairs.
+    this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, leftChild.getSchema(),
+        rightChild.getSchema(), false);
+
+    leftKeyList = new int[joinKeyPairs.size()];
+    rightKeyList = new int[joinKeyPairs.size()];
+
+    for (int i = 0; i < joinKeyPairs.size(); i++) {
+      leftKeyList[i] = leftChild.getSchema().getColumnId(joinKeyPairs.get(i)[0].getQualifiedName());
+    }
+
+    for (int i = 0; i < joinKeyPairs.size(); i++) {
+      rightKeyList[i] = rightChild.getSchema().getColumnId(joinKeyPairs.get(i)[1].getQualifiedName());
+    }
+
+    // for projection
+    this.projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+    leftKeyTuple = new VTuple(leftKeyList.length);
+
+    rightNumCols = rightChild.getSchema().size();
+  }
+
+  protected void getKeyLeftTuple(final Tuple outerTuple, Tuple keyTuple) {
+    for (int i = 0; i < leftKeyList.length; i++) {
+      keyTuple.put(i, outerTuple.get(leftKeyList[i]));
+    }
+  }
+
+  public Tuple next() throws IOException {
+    if (first) {
+      loadRightToHashTable();
+    }
+
+    Tuple rightTuple;
+    boolean found = false;
+
+    while(!finished) {
+
+      if (shouldGetLeftTuple) { // initially, it is true.
+        // getting new outer
+        leftTuple = leftChild.next(); // it comes from a disk
+        if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed.
+          finished = true;
+          return null;
+        }
+
+        // getting corresponding right
+        getKeyLeftTuple(leftTuple, leftKeyTuple); // get a left key tuple
+        List<Tuple> rightTuples = tupleSlots.get(leftKeyTuple);
+        if (rightTuples != null) { // found right tuples on in-memory hash table.
+          iterator = rightTuples.iterator();
+          shouldGetLeftTuple = false;
+        } else {
+          // this left tuple doesn't have a match on the right, and output a tuple with the nulls padded rightTuple
+          Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols);
+          frameTuple.set(leftTuple, nullPaddedTuple);
+          projector.eval(frameTuple, outTuple);
+          // we simulate we found a match, which is exactly the null padded one
+          shouldGetLeftTuple = true;
+          return outTuple;
+        }
+      }
+
+      // getting a next right tuple on in-memory hash table.
+      rightTuple = iterator.next();
+      frameTuple.set(leftTuple, rightTuple); // evaluate a join condition on both tuples
+      if (joinQual.eval(inSchema, frameTuple).isTrue()) { // if both tuples are joinable
+        projector.eval(frameTuple, outTuple);
+        found = true;
+      }
+
+      if (!iterator.hasNext()) { // no more right tuples for this hash key
+        shouldGetLeftTuple = true;
+      }
+
+      if (found) {
+        break;
+      }
+    }
+
+    return outTuple;
+  }
+
+  protected void loadRightToHashTable() throws IOException {
+    Tuple tuple;
+    Tuple keyTuple;
+
+    while ((tuple = rightChild.next()) != null) {
+      keyTuple = new VTuple(joinKeyPairs.size());
+      for (int i = 0; i < rightKeyList.length; i++) {
+        keyTuple.put(i, tuple.get(rightKeyList[i]));
+      }
+
+      List<Tuple> newValue = tupleSlots.get(keyTuple);
+      if (newValue != null) {
+        newValue.add(tuple);
+      } else {
+        newValue = new ArrayList<Tuple>();
+        newValue.add(tuple);
+        tupleSlots.put(keyTuple, newValue);
+      }
+    }
+    first = false;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+
+    tupleSlots.clear();
+    first = true;
+
+    finished = false;
+    iterator = null;
+    shouldGetLeftTuple = true;
+  }
+
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    tupleSlots.clear();
+    tupleSlots = null;
+    iterator = null;
+    plan = null;
+    joinQual = null;
+    projector = null;
+  }
+
+  public JoinNode getPlan() {
+    return this.plan;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java
new file mode 100644
index 0000000..4fbb5e4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java
@@ -0,0 +1,107 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Prepare a hash table of the NOT IN side of the join. Scan the FROM side table.
+ * For each tuple of the FROM side table, it tries to find a matched tuple from the hash table for the NOT INT side.
+ * If found, it returns the tuple of the FROM side table.
+ */
+public class HashLeftSemiJoinExec extends HashJoinExec {
+  private Tuple rightNullTuple;
+
+  public HashLeftSemiJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec fromSideChild,
+                              PhysicalExec inSideChild) {
+    super(context, plan, fromSideChild, inSideChild);
+    // NUll Tuple
+    rightNullTuple = new VTuple(leftChild.outColumnNum);
+    for (int i = 0; i < leftChild.outColumnNum; i++) {
+      rightNullTuple.put(i, NullDatum.get());
+    }
+  }
+
+  /**
+   * The End of Tuple (EOT) condition is true only when no more tuple in the left relation (on disk).
+   * next() method finds the first unmatched tuple from both tables.
+   *
+   * For each left tuple on the disk, next() tries to find at least one matched tuple from the hash table.
+   *
+   * In more detail, until there is a hash bucket matched to the left tuple in the hash table, it continues to traverse
+   * the left tuples. If next() finds the matched bucket in the hash table, it finds any matched tuple in the bucket.
+   * If found, it returns the composite tuple immediately without finding more matched tuple in the bucket.
+   *
+   * @return The tuple which is firstly matched to a given join condition.
+   * @throws java.io.IOException
+   */
+  public Tuple next() throws IOException {
+    if (first) {
+      loadRightToHashTable();
+    }
+
+    Tuple rightTuple;
+    boolean notFound;
+
+    while(!finished) {
+
+      // getting new outer
+      leftTuple = leftChild.next(); // it comes from a disk
+      if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed.
+        finished = true;
+        return null;
+      }
+
+      // Try to find a hash bucket in in-memory hash table
+      getKeyLeftTuple(leftTuple, leftKeyTuple);
+      List<Tuple> rightTuples = tupleSlots.get(leftKeyTuple);
+      if (rightTuples != null) {
+        // if found, it gets a hash bucket from the hash table.
+        iterator = rightTuples.iterator();
+      } else {
+        continue;
+      }
+
+      // Reach here only when a hash bucket is found. Then, it checks all tuples in the found bucket.
+      // If it finds any matched tuple, it returns the tuple immediately.
+      notFound = true;
+      while (notFound && iterator.hasNext()) {
+        rightTuple = iterator.next();
+        frameTuple.set(leftTuple, rightTuple);
+        if (joinQual.eval(inSchema, frameTuple).isTrue()) { // if the matched one is found
+          notFound = false;
+          projector.eval(frameTuple, outTuple);
+        }
+      }
+
+      if (!notFound) { // if there is no matched tuple
+        break;
+      }
+    }
+
+    return outTuple;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java
new file mode 100644
index 0000000..3ae53d9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java
@@ -0,0 +1,49 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+public class HashPartitioner extends Partitioner {
+  private final Tuple keyTuple;
+  
+  public HashPartitioner(final int [] keys, final int numPartitions) {
+    super(keys, numPartitions);
+    this.keyTuple = new VTuple(partitionKeyIds.length);
+  }
+  
+  @Override
+  public int getPartition(Tuple tuple) {
+    // In outer join, partition number can be zero because of empty tables.
+    // So, we should return zero for this case.
+    if (numPartitions == 0)
+      return 0;
+
+    // build one key tuple
+    for (int i = 0; i < partitionKeyIds.length; i++) {
+      keyTuple.put(i, tuple.get(partitionKeyIds[i]));
+    }
+    return (keyTuple.hashCode() & Integer.MAX_VALUE) %
+        (numPartitions == 32 ? numPartitions-1 : numPartitions);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java
new file mode 100644
index 0000000..678b745
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java
@@ -0,0 +1,160 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.engine.planner.logical.ShuffleFileWriteNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <code>HashShuffleFileWriteExec</code> is a physical executor to store intermediate data into a number of
+ * file outputs associated with shuffle keys. The file outputs are stored on local disks.
+ */
+public final class HashShuffleFileWriteExec extends UnaryPhysicalExec {
+  private static Log LOG = LogFactory.getLog(HashShuffleFileWriteExec.class);
+  private ShuffleFileWriteNode plan;
+  private final TableMeta meta;
+  private Partitioner partitioner;
+  private final Path storeTablePath;
+  private Map<Integer, Appender> appenderMap = new HashMap<Integer, Appender>();
+  private final int numShuffleOutputs;
+  private final int [] shuffleKeyIds;
+  
+  public HashShuffleFileWriteExec(TaskAttemptContext context, final AbstractStorageManager sm,
+                                  final ShuffleFileWriteNode plan, final PhysicalExec child) throws IOException {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+    Preconditions.checkArgument(plan.hasShuffleKeys());
+    this.plan = plan;
+    if (plan.hasOptions()) {
+      this.meta = CatalogUtil.newTableMeta(plan.getStorageType(), plan.getOptions());
+    } else {
+      this.meta = CatalogUtil.newTableMeta(plan.getStorageType());
+    }
+    // about the shuffle
+    this.numShuffleOutputs = this.plan.getNumOutputs();
+    int i = 0;
+    this.shuffleKeyIds = new int [this.plan.getShuffleKeys().length];
+    for (Column key : this.plan.getShuffleKeys()) {
+      shuffleKeyIds[i] = inSchema.getColumnId(key.getQualifiedName());
+      i++;
+    }
+    this.partitioner = new HashPartitioner(shuffleKeyIds, numShuffleOutputs);
+    storeTablePath = new Path(context.getWorkDir(), "output");
+  }
+
+  @Override
+  public void init() throws IOException {
+    super.init();
+    FileSystem fs = new RawLocalFileSystem();
+    fs.mkdirs(storeTablePath);
+  }
+  
+  private Appender getAppender(int partId) throws IOException {
+    Appender appender = appenderMap.get(partId);
+
+    if (appender == null) {
+      Path dataFile = getDataFile(partId);
+      FileSystem fs = dataFile.getFileSystem(context.getConf());
+      if (fs.exists(dataFile)) {
+        LOG.info("File " + dataFile + " already exists!");
+        FileStatus status = fs.getFileStatus(dataFile);
+        LOG.info("File size: " + status.getLen());
+      }
+      appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta, outSchema, dataFile);
+      appender.enableStats();
+      appender.init();
+      appenderMap.put(partId, appender);
+    } else {
+      appender = appenderMap.get(partId);
+    }
+
+    return appender;
+  }
+
+  private Path getDataFile(int partId) {
+    return StorageUtil.concatPath(storeTablePath, ""+partId);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple;
+    Appender appender;
+    int partId;
+    while ((tuple = child.next()) != null) {
+      partId = partitioner.getPartition(tuple);
+      appender = getAppender(partId);
+      appender.addTuple(tuple);
+    }
+    
+    List<TableStats> statSet = new ArrayList<TableStats>();
+    for (Map.Entry<Integer, Appender> entry : appenderMap.entrySet()) {
+      int partNum = entry.getKey();
+      Appender app = entry.getValue();
+      app.flush();
+      app.close();
+      statSet.add(app.getStats());
+      if (app.getStats().getNumRows() > 0) {
+        context.addShuffleFileOutput(partNum, getDataFile(partNum).getName());
+      }
+    }
+    
+    // Collect and aggregated statistics data
+    TableStats aggregated = StatisticsUtil.aggregateTableStat(statSet);
+    context.setResultStats(aggregated);
+    
+    return null;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    // nothing to do   
+  }
+
+  @Override
+  public void close() throws IOException{
+    super.close();
+    if (appenderMap != null) {
+      appenderMap.clear();
+      appenderMap = null;
+    }
+
+    partitioner = null;
+    plan = null;
+
+    progress = 1.0f;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java
new file mode 100644
index 0000000..0418f65
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java
@@ -0,0 +1,50 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.logical.HavingNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class HavingExec extends UnaryPhysicalExec  {
+  private final EvalNode qual;
+
+  public HavingExec(TaskAttemptContext context,
+                    HavingNode plan,
+                    PhysicalExec child) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+
+    this.qual = plan.getQual();
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple;
+    while ((tuple = child.next()) != null) {
+      if (qual.eval(inSchema, tuple).isTrue()) {
+        return tuple;
+      }
+    }
+
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/JoinTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/JoinTupleComparator.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/JoinTupleComparator.java
new file mode 100644
index 0000000..0d4c47b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/JoinTupleComparator.java
@@ -0,0 +1,87 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.Tuple;
+
+import java.util.Comparator;
+
+/**
+ * The Comparator class for Outer and Inner Tuples
+ *
+ * @see org.apache.tajo.storage.Tuple
+ */
+public class JoinTupleComparator implements Comparator<Tuple> {
+  private int numSortKey;
+  private final int[] outerSortKeyIds;
+  private final int[] innerSortKeyIds;
+
+  private Datum outer;
+  private Datum inner;
+  private int compVal;
+
+  public JoinTupleComparator(Schema leftschema, Schema rightschema, SortSpec[][] sortKeys) {
+    Preconditions.checkArgument(sortKeys.length == 2,
+        "The two of the sortspecs must be given, but " + sortKeys.length + " sortkeys are given.");
+    Preconditions.checkArgument(sortKeys[0].length == sortKeys[1].length,
+        "The number of both side sortkeys must be equals, but they are different: "
+            + sortKeys[0].length + " and " + sortKeys[1].length);
+
+    this.numSortKey = sortKeys[0].length; // because it is guaranteed that the number of sortspecs are equals
+    this.outerSortKeyIds = new int[numSortKey];
+    this.innerSortKeyIds = new int[numSortKey];
+
+    for (int i = 0; i < numSortKey; i++) {
+      this.outerSortKeyIds[i] = leftschema.getColumnId(sortKeys[0][i].getSortKey().getQualifiedName());
+      this.innerSortKeyIds[i] = rightschema.getColumnId(sortKeys[1][i].getSortKey().getQualifiedName());
+    }
+  }
+
+  @Override
+  public int compare(Tuple outerTuple, Tuple innerTuple) {
+    for (int i = 0; i < numSortKey; i++) {
+      outer = outerTuple.get(outerSortKeyIds[i]);
+      inner = innerTuple.get(innerSortKeyIds[i]);
+
+      if (outer instanceof NullDatum || inner instanceof NullDatum) {
+        if (!outer.equals(inner)) {
+          if (outer instanceof NullDatum) {
+            compVal = 1;
+          } else if (inner instanceof NullDatum) {
+            compVal = -1;
+          }
+        } else {
+          compVal = 0;
+        }
+      } else {
+        compVal = outer.compareTo(inner);
+      }
+
+      if (compVal < 0 || compVal > 0) {
+        return compVal;
+      }
+    }
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/LimitExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/LimitExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/LimitExec.java
new file mode 100644
index 0000000..d736c25
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/LimitExec.java
@@ -0,0 +1,55 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.planner.logical.LimitNode;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public class LimitExec extends UnaryPhysicalExec {
+  private final long fetchFirstNum;
+  private long fetchCount;
+
+  public LimitExec(TaskAttemptContext context, Schema inSchema,
+                   Schema outSchema, PhysicalExec child, LimitNode limit) {
+    super(context, inSchema, outSchema, child);
+    this.fetchFirstNum = limit.getFetchFirstNum();
+    this.fetchCount = 0;
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple = child.next();
+    fetchCount++;
+
+    if (fetchCount > fetchFirstNum || tuple == null) {
+      return null;
+    }
+
+    return tuple;
+  }
+
+  public void rescan() throws IOException {
+    super.init();
+    fetchCount = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MemSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MemSortExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MemSortExec.java
new file mode 100644
index 0000000..9f4f20a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MemSortExec.java
@@ -0,0 +1,89 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.engine.planner.logical.SortNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+public class MemSortExec extends SortExec {
+  private SortNode plan;
+  private List<Tuple> tupleSlots;
+  private boolean sorted = false;
+  private Iterator<Tuple> iterator;
+  
+  public MemSortExec(final TaskAttemptContext context,
+                     SortNode plan, PhysicalExec child) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child, plan.getSortKeys());
+    this.plan = plan;
+  }
+
+  public void init() throws IOException {
+    super.init();
+    this.tupleSlots = new ArrayList<Tuple>(1000);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+
+    if (!sorted) {
+      Tuple tuple;
+      while ((tuple = child.next()) != null) {
+        tupleSlots.add(new VTuple(tuple));
+      }
+      
+      Collections.sort(tupleSlots, getComparator());
+      this.iterator = tupleSlots.iterator();
+      sorted = true;
+    }
+    
+    if (iterator.hasNext()) {
+      return this.iterator.next();
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+    this.iterator = tupleSlots.iterator();
+    sorted = true;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    tupleSlots.clear();
+    tupleSlots = null;
+    iterator = null;
+    plan = null;
+  }
+
+  public SortNode getPlan() {
+    return this.plan;
+  }
+}


[13/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all-min.js
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all-min.js b/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all-min.js
new file mode 100644
index 0000000..48b4cc5
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all-min.js
@@ -0,0 +1 @@
+jsPlumbUtil={isArray:function(b){return Object.prototype.toString.call(b)==="[object Array]"},isString:function(a){return typeof a==="string"},isBoolean:function(a){return typeof a==="boolean"},isObject:function(a){return Object.prototype.toString.call(a)==="[object Object]"},isDate:function(a){return Object.prototype.toString.call(a)==="[object Date]"},isFunction:function(a){return Object.prototype.toString.call(a)==="[object Function]"},clone:function(d){if(this.isString(d)){return new String(d)}else{if(this.isBoolean(d)){return new Boolean(d)}else{if(this.isDate(d)){return new Date(d.getTime())}else{if(this.isFunction(d)){return d}else{if(this.isArray(d)){var c=[];for(var e=0;e<d.length;e++){c.push(this.clone(d[e]))}return c}else{if(this.isObject(d)){var c={};for(var e in d){c[e]=this.clone(d[e])}return c}else{return d}}}}}}},merge:function(e,d){var k=this.clone(e);for(var h in d){if(k[h]==null||this.isString(d[h])||this.isBoolean(d[h])){k[h]=d[h]}else{if(this.isArray(d[h])&&this
 .isArray(k[h])){var f=[];f.push.apply(f,k[h]);f.push.apply(f,d[h]);k[h]=f}else{if(this.isObject(k[h])&&this.isObject(d[h])){for(var g in d[h]){k[h][g]=d[h][g]}}}}}return k},convertStyle:function(b,a){if("transparent"===b){return b}var g=b,f=function(h){return h.length==1?"0"+h:h},c=function(h){return f(Number(h).toString(16))},d=/(rgb[a]?\()(.*)(\))/;if(b.match(d)){var e=b.match(d)[2].split(",");g="#"+c(e[0])+c(e[1])+c(e[2]);if(!a&&e.length==4){g=g+c(e[3])}}return g},gradient:function(b,a){b=jsPlumbUtil.isArray(b)?b:[b.x,b.y];a=jsPlumbUtil.isArray(a)?a:[a.x,a.y];return(a[1]-b[1])/(a[0]-b[0])},normal:function(b,a){return -1/jsPlumbUtil.gradient(b,a)},lineLength:function(b,a){b=jsPlumbUtil.isArray(b)?b:[b.x,b.y];a=jsPlumbUtil.isArray(a)?a:[a.x,a.y];return Math.sqrt(Math.pow(a[1]-b[1],2)+Math.pow(a[0]-b[0],2))},segment:function(b,a){b=jsPlumbUtil.isArray(b)?b:[b.x,b.y];a=jsPlumbUtil.isArray(a)?a:[a.x,a.y];if(a[0]>b[0]){return(a[1]>b[1])?2:1}else{return(a[1]>b[1])?3:4}},intersects:funct
 ion(f,e){var c=f.x,a=f.x+f.w,k=f.y,h=f.y+f.h,d=e.x,b=e.x+e.w,i=e.y,g=e.y+e.h;return((c<=d&&d<=a)&&(k<=i&&i<=h))||((c<=b&&b<=a)&&(k<=i&&i<=h))||((c<=d&&d<=a)&&(k<=g&&g<=h))||((c<=b&&d<=a)&&(k<=g&&g<=h))||((d<=c&&c<=b)&&(i<=k&&k<=g))||((d<=a&&a<=b)&&(i<=k&&k<=g))||((d<=c&&c<=b)&&(i<=h&&h<=g))||((d<=a&&c<=b)&&(i<=h&&h<=g))},segmentMultipliers:[null,[1,-1],[1,1],[-1,1],[-1,-1]],inverseSegmentMultipliers:[null,[-1,-1],[-1,1],[1,1],[1,-1]],pointOnLine:function(a,e,b){var d=jsPlumbUtil.gradient(a,e),i=jsPlumbUtil.segment(a,e),h=b>0?jsPlumbUtil.segmentMultipliers[i]:jsPlumbUtil.inverseSegmentMultipliers[i],c=Math.atan(d),f=Math.abs(b*Math.sin(c))*h[1],g=Math.abs(b*Math.cos(c))*h[0];return{x:a.x+g,y:a.y+f}},perpendicularLineTo:function(c,d,e){var b=jsPlumbUtil.gradient(c,d),f=Math.atan(-1/b),g=e/2*Math.sin(f),a=e/2*Math.cos(f);return[{x:d.x+a,y:d.y+g},{x:d.x-a,y:d.y-g}]},findWithFunction:function(b,d){if(b){for(var c=0;c<b.length;c++){if(d(b[c])){return c}}}return -1},indexOf:function(a,b){r
 eturn jsPlumbUtil.findWithFunction(a,function(c){return c==b})},removeWithFunction:function(c,d){var b=jsPlumbUtil.findWithFunction(c,d);if(b>-1){c.splice(b,1)}return b!=-1},remove:function(b,c){var a=jsPlumbUtil.indexOf(b,c);if(a>-1){b.splice(a,1)}return a!=-1},addWithFunction:function(c,b,a){if(jsPlumbUtil.findWithFunction(c,a)==-1){c.push(b)}},addToList:function(d,b,c){var a=d[b];if(a==null){a=[],d[b]=a}a.push(c);return a},EventGenerator:function(){var c={},b=this;var a=["ready"];this.bind=function(d,e){jsPlumbUtil.addToList(c,d,e);return b};this.fire=function(g,h,d){if(c[g]){for(var f=0;f<c[g].length;f++){if(jsPlumbUtil.findWithFunction(a,function(i){return i===g})!=-1){c[g][f](h,d)}else{try{c[g][f](h,d)}catch(k){jsPlumbUtil.log("jsPlumb: fire failed for event "+g+" : "+k)}}}}return b};this.unbind=function(d){if(d){delete c[d]}else{c={}}return b};this.getListener=function(d){return c[d]}},logEnabled:true,log:function(){if(jsPlumbUtil.logEnabled&&typeof console!="undefined"){try{
 var b=arguments[arguments.length-1];console.log(b)}catch(a){}}},group:function(a){if(jsPlumbUtil.logEnabled&&typeof console!="undefined"){console.group(a)}},groupEnd:function(a){if(jsPlumbUtil.logEnabled&&typeof console!="undefined"){console.groupEnd(a)}},time:function(a){if(jsPlumbUtil.logEnabled&&typeof console!="undefined"){console.time(a)}},timeEnd:function(a){if(jsPlumbUtil.logEnabled&&typeof console!="undefined"){console.timeEnd(a)}}};(function(){var b=!!document.createElement("canvas").getContext,a=!!window.SVGAngle||document.implementation.hasFeature("http://www.w3.org/TR/SVG11/feature#BasicStructure","1.1"),d=function(){if(d.vml==undefined){var f=document.body.appendChild(document.createElement("div"));f.innerHTML='<v:shape id="vml_flag1" adj="1" />';var e=f.firstChild;e.style.behavior="url(#default#VML)";d.vml=e?typeof e.adj=="object":true;f.parentNode.removeChild(f)}return d.vml};var c=function(i){var h={},g=[],f={},e={};this.register=function(n){var m=jsPlumb.CurrentLibr
 ary;n=m.getElementObject(n);var p=i.getId(n),l=m.getDOMElement(n),k=m.getOffset(n);if(!h[p]){h[p]=n;g.push(n);f[p]={}}var o=function(u,q){if(u){for(var r=0;r<u.childNodes.length;r++){if(u.childNodes[r].nodeType!=3){var t=m.getElementObject(u.childNodes[r]),v=i.getId(t,null,true);if(v&&e[v]&&e[v]>0){var s=m.getOffset(t);f[p][v]={id:v,offset:{left:s.left-k.left,top:s.top-k.top}}}o(u.childNodes[r])}}}};o(l)};this.updateOffsets=function(o){var r=jsPlumb.CurrentLibrary,m=r.getElementObject(o),l=i.getId(m),n=f[l],k=r.getOffset(m);if(n){for(var q in n){var s=r.getElementObject(q),p=r.getOffset(s);f[l][q]={id:q,offset:{left:p.left-k.left,top:p.top-k.top}}}}};this.endpointAdded=function(m){var r=jsPlumb.CurrentLibrary,u=document.body,k=i.getId(m),t=r.getDOMElement(m),l=t.parentNode,o=l==u;e[k]=e[k]?e[k]+1:1;while(l!=u){var q=i.getId(l,null,true);if(q&&h[q]){var w=-1,s=r.getElementObject(l),n=r.getOffset(s);if(f[q][k]==null){var v=jsPlumb.CurrentLibrary.getOffset(m);f[q][k]={id:k,offset:{left
 :v.left-n.left,top:v.top-n.top}}}break}l=l.parentNode}};this.endpointDeleted=function(l){if(e[l.elementId]){e[l.elementId]--;if(e[l.elementId]<=0){for(var k in f){delete f[k][l.elementId]}}}};this.getElementsForDraggable=function(k){return f[k]};this.reset=function(){h={};g=[];f={};e={}}};if(!window.console){window.console={time:function(){},timeEnd:function(){},group:function(){},groupEnd:function(){},log:function(){}}}window.jsPlumbAdapter={headless:false,appendToRoot:function(e){document.body.appendChild(e)},getRenderModes:function(){return["canvas","svg","vml"]},isRenderModeAvailable:function(e){return{canvas:b,svg:a,vml:d()}[e]},getDragManager:function(e){return new c(e)},setRenderMode:function(i){var h;if(i){i=i.toLowerCase();var f=this.isRenderModeAvailable("canvas"),e=this.isRenderModeAvailable("svg"),g=this.isRenderModeAvailable("vml");if(i==="svg"){if(e){h="svg"}else{if(f){h="canvas"}else{if(g){h="vml"}}}}else{if(i==="canvas"&&f){h="canvas"}else{if(g){h="vml"}}}}return h}}
 })();(function(){var g=jsPlumbUtil.findWithFunction,G=jsPlumbUtil.indexOf,A=jsPlumbUtil.removeWithFunction,k=jsPlumbUtil.remove,r=jsPlumbUtil.addWithFunction,i=jsPlumbUtil.addToList,l=jsPlumbUtil.isArray,z=jsPlumbUtil.isString,u=jsPlumbUtil.isObject;var v=null,c=function(H,I){return n.CurrentLibrary.getAttribute(C(H),I)},d=function(I,J,H){n.CurrentLibrary.setAttribute(C(I),J,H)},y=function(I,H){n.CurrentLibrary.addClass(C(I),H)},h=function(I,H){return n.CurrentLibrary.hasClass(C(I),H)},m=function(I,H){n.CurrentLibrary.removeClass(C(I),H)},C=function(H){return n.CurrentLibrary.getElementObject(H)},s=function(I,H){var K=n.CurrentLibrary.getOffset(C(I));if(H!=null){var J=H.getZoom();return{left:K.left/J,top:K.top/J}}else{return K}},a=function(H){return n.CurrentLibrary.getSize(C(H))},o=jsPlumbUtil.log,F=jsPlumbUtil.group,f=jsPlumbUtil.groupEnd,E=jsPlumbUtil.time,t=jsPlumbUtil.timeEnd,p=function(){return""+(new Date()).getTime()},B=function(aa){var U=this,ab=arguments,S=false,M=aa.param
 eters||{},K=U.idPrefix,X=K+(new Date()).getTime(),W=null,ac=null;U._jsPlumb=aa._jsPlumb;U.getId=function(){return X};U.tooltip=aa.tooltip;U.hoverClass=aa.hoverClass||U._jsPlumb.Defaults.HoverClass||n.Defaults.HoverClass;jsPlumbUtil.EventGenerator.apply(this);this.clone=function(){var ad=new Object();U.constructor.apply(ad,ab);return ad};this.getParameter=function(ad){return M[ad]},this.getParameters=function(){return M},this.setParameter=function(ad,ae){M[ad]=ae},this.setParameters=function(ad){M=ad},this.overlayPlacements=[];var L=aa.beforeDetach;this.isDetachAllowed=function(ad){var ae=U._jsPlumb.checkCondition("beforeDetach",ad);if(L){try{ae=L(ad)}catch(af){o("jsPlumb: beforeDetach callback failed",af)}}return ae};var O=aa.beforeDrop;this.isDropAllowed=function(ai,af,ag,ad,ae){var ah=U._jsPlumb.checkCondition("beforeDrop",{sourceId:ai,targetId:af,scope:ag,connection:ad,dropEndpoint:ae});if(O){try{ah=O({sourceId:ai,targetId:af,scope:ag,connection:ad,dropEndpoint:ae})}catch(aj){o("
 jsPlumb: beforeDrop callback failed",aj)}}return ah};var Y=function(){if(W&&ac){var ad={};n.extend(ad,W);n.extend(ad,ac);delete U.hoverPaintStyle;if(ad.gradient&&W.fillStyle){delete ad.gradient}ac=ad}};this.setPaintStyle=function(ad,ae){W=ad;U.paintStyleInUse=W;Y();if(!ae){U.repaint()}};this.getPaintStyle=function(){return W};this.setHoverPaintStyle=function(ad,ae){ac=ad;Y();if(!ae){U.repaint()}};this.getHoverPaintStyle=function(){return ac};this.setHover=function(ad,af,ae){if(!U._jsPlumb.currentlyDragging&&!U._jsPlumb.isHoverSuspended()){S=ad;if(U.hoverClass!=null&&U.canvas!=null){if(ad){J.addClass(U.canvas,U.hoverClass)}else{J.removeClass(U.canvas,U.hoverClass)}}if(ac!=null){U.paintStyleInUse=ad?ac:W;ae=ae||p();U.repaint({timestamp:ae,recalc:false})}if(U.getAttachedElements&&!af){Z(ad,p(),U)}}};this.isHover=function(){return S};var V=null;this.setZIndex=function(ad){V=ad};this.getZIndex=function(){return V};var J=n.CurrentLibrary,I=["click","dblclick","mouseenter","mouseout","mous
 emove","mousedown","mouseup","contextmenu"],T={mouseout:"mouseexit"},N=function(af,ag,ae){var ad=T[ae]||ae;J.bind(af,ae,function(ah){ag.fire(ad,ag,ah)})},R=function(af,ae){var ad=T[ae]||ae;J.unbind(af,ae)};this.attachListeners=function(ae,af){for(var ad=0;ad<I.length;ad++){N(ae,af,I[ad])}};var Z=function(ah,ag,ad){var af=U.getAttachedElements();if(af){for(var ae=0;ae<af.length;ae++){if(!ad||ad!=af[ae]){af[ae].setHover(ah,true,ag)}}}};this.reattachListenersForElement=function(ae){if(arguments.length>1){for(var ad=0;ad<I.length;ad++){R(ae,I[ad])}for(var ad=1;ad<arguments.length;ad++){U.attachListeners(ae,arguments[ad])}}};var H=[],P=function(ad){return ad==null?null:ad.split(" ")},Q=function(ae){if(U.getDefaultType){var ag=U.getTypeDescriptor();var af=jsPlumbUtil.merge({},U.getDefaultType());for(var ad=0;ad<H.length;ad++){af=jsPlumbUtil.merge(af,U._jsPlumb.getType(H[ad],ag))}U.applyType(af);if(!ae){U.repaint()}}};U.setType=function(ad,ae){H=P(ad)||[];Q(ae)};U.getType=function(){return
  H};U.hasType=function(ad){return jsPlumbUtil.indexOf(H,ad)!=-1};U.addType=function(af,ag){var ae=P(af),ah=false;if(ae!=null){for(var ad=0;ad<ae.length;ad++){if(!U.hasType(ae[ad])){H.push(ae[ad]);ah=true}}if(ah){Q(ag)}}};U.removeType=function(ag,ah){var ae=P(ag),ai=false,af=function(ak){var aj=jsPlumbUtil.indexOf(H,ak);if(aj!=-1){H.splice(aj,1);return true}return false};if(ae!=null){for(var ad=0;ad<ae.length;ad++){ai=af(ae[ad])||ai}if(ai){Q(ah)}}};U.toggleType=function(ag,ah){var af=P(ag);if(af!=null){for(var ae=0;ae<af.length;ae++){var ad=jsPlumbUtil.indexOf(H,af[ae]);if(ad!=-1){H.splice(ad,1)}else{H.push(af[ae])}}Q(ah)}};this.applyType=function(ae){U.setPaintStyle(ae.paintStyle);U.setHoverPaintStyle(ae.hoverPaintStyle);if(ae.parameters){for(var ad in ae.parameters){U.setParameter(ad,ae.parameters[ad])}}}},x=function(M){B.apply(this,arguments);var S=this;this.overlays=[];var K=function(X){var V=null;if(l(X)){var U=X[0],W=n.extend({component:S,_jsPlumb:S._jsPlumb},X[1]);if(X.length=
 =3){n.extend(W,X[2])}V=new n.Overlays[S._jsPlumb.getRenderMode()][U](W);if(W.events){for(var T in W.events){V.bind(T,W.events[T])}}}else{if(X.constructor==String){V=new n.Overlays[S._jsPlumb.getRenderMode()][X]({component:S,_jsPlumb:S._jsPlumb})}else{V=X}}S.overlays.push(V)},L=function(X){var T=S.defaultOverlayKeys||[],W=X.overlays,U=function(Y){return S._jsPlumb.Defaults[Y]||n.Defaults[Y]||[]};if(!W){W=[]}for(var V=0;V<T.length;V++){W.unshift.apply(W,U(T[V]))}return W};var I=L(M);if(I){for(var O=0;O<I.length;O++){K(I[O])}}var H=function(V){var T=-1;for(var U=0;U<S.overlays.length;U++){if(V===S.overlays[U].id){T=U;break}}return T};this.addOverlay=function(T,U){K(T);if(!U){S.repaint()}};this.getOverlay=function(U){var T=H(U);return T>=0?S.overlays[T]:null};this.getOverlays=function(){return S.overlays};this.hideOverlay=function(U){var T=S.getOverlay(U);if(T){T.hide()}};this.hideOverlays=function(){for(var T=0;T<S.overlays.length;T++){S.overlays[T].hide()}};this.showOverlay=function(U
 ){var T=S.getOverlay(U);if(T){T.show()}};this.showOverlays=function(){for(var T=0;T<S.overlays.length;T++){S.overlays[T].show()}};this.removeAllOverlays=function(){for(var T=0;T<S.overlays.length;T++){if(S.overlays[T].cleanup){S.overlays[T].cleanup()}}S.overlays.splice(0,S.overlays.length);S.repaint()};this.removeOverlay=function(U){var T=H(U);if(T!=-1){var V=S.overlays[T];if(V.cleanup){V.cleanup()}S.overlays.splice(T,1)}};this.removeOverlays=function(){for(var T=0;T<arguments.length;T++){S.removeOverlay(arguments[T])}};var J="__label",R=function(V){var T={cssClass:V.cssClass,labelStyle:this.labelStyle,id:J,component:S,_jsPlumb:S._jsPlumb},U=n.extend(T,V);return new n.Overlays[S._jsPlumb.getRenderMode()].Label(U)};if(M.label){var P=M.labelLocation||S.defaultLabelLocation||0.5,Q=M.labelStyle||S._jsPlumb.Defaults.LabelStyle||n.Defaults.LabelStyle;this.overlays.push(R({label:M.label,location:P,labelStyle:Q}))}this.setLabel=function(T){var U=S.getOverlay(J);if(!U){var V=T.constructor==S
 tring||T.constructor==Function?{label:T}:T;U=R(V);this.overlays.push(U)}else{if(T.constructor==String||T.constructor==Function){U.setLabel(T)}else{if(T.label){U.setLabel(T.label)}if(T.location){U.setLocation(T.location)}}}if(!S._jsPlumb.isSuspendDrawing()){S.repaint()}};this.getLabel=function(){var T=S.getOverlay(J);return T!=null?T.getLabel():null};this.getLabelOverlay=function(){return S.getOverlay(J)};var N=this.applyType;this.applyType=function(U){N(U);S.removeAllOverlays();if(U.overlays){for(var T=0;T<U.overlays.length;T++){S.addOverlay(U.overlays[T],true)}}}},D=function(J,H,I){J.bind("click",function(K,L){H.fire("click",H,L)});J.bind("dblclick",function(K,L){H.fire("dblclick",H,L)});J.bind("contextmenu",function(K,L){H.fire("contextmenu",H,L)});J.bind("mouseenter",function(K,L){if(!H.isHover()){I(true);H.fire("mouseenter",H,L)}});J.bind("mouseexit",function(K,L){if(H.isHover()){I(false);H.fire("mouseexit",H,L)}})};var e=0,b=function(){var H=e+1;e++;return H};var w=function(I){
 this.Defaults={Anchor:"BottomCenter",Anchors:[null,null],ConnectionsDetachable:true,ConnectionOverlays:[],Connector:"Bezier",ConnectorZIndex:null,Container:null,DragOptions:{},DropOptions:{},Endpoint:"Dot",EndpointOverlays:[],Endpoints:[null,null],EndpointStyle:{fillStyle:"#456"},EndpointStyles:[null,null],EndpointHoverStyle:null,EndpointHoverStyles:[null,null],HoverPaintStyle:null,LabelStyle:{color:"black"},LogEnabled:false,Overlays:[],MaxConnections:1,PaintStyle:{lineWidth:8,strokeStyle:"#456"},ReattachConnections:false,RenderMode:"svg",Scope:"jsPlumb_DefaultScope"};if(I){n.extend(this.Defaults,I)}this.logEnabled=this.Defaults.LogEnabled;var a4={},al={};this.registerConnectionType=function(bA,bz){a4[bA]=n.extend({},bz)};this.registerConnectionTypes=function(bA){for(var bz in bA){a4[bz]=n.extend({},bA[bz])}};this.registerEndpointType=function(bA,bz){al[bA]=n.extend({},bz)};this.registerEndpointTypes=function(bA){for(var bz in bA){al[bz]=n.extend({},bA[bz])}};this.getType=function(b
 A,bz){return bz==="connection"?a4[bA]:al[bA]};jsPlumbUtil.EventGenerator.apply(this);var br=this,aO=b(),aR=br.bind,aE={},ad=1;this.setZoom=function(bA,bz){ad=bA;if(bz){br.repaintEverything()}};this.getZoom=function(){return ad};for(var aD in this.Defaults){aE[aD]=this.Defaults[aD]}this.bind=function(bA,bz){if("ready"===bA&&K){bz()}else{aR.apply(br,[bA,bz])}};br.importDefaults=function(bA){for(var bz in bA){br.Defaults[bz]=bA[bz]}};br.restoreDefaults=function(){br.Defaults=n.extend({},aE)};var O=null,bj=null,K=false,a0={},aV={},aW={},aj={},bu={},bk={},bq={},by=[],ag=[],R=this.Defaults.Scope,Y=null,W=function(bC,bA,bB){var bz=bC[bA];if(bz==null){bz=[];bC[bA]=bz}bz.push(bB);return bz},aX=function(bA,bz){if(br.Defaults.Container){n.CurrentLibrary.appendElement(bA,br.Defaults.Container)}else{if(!bz){jsPlumbAdapter.appendToRoot(bA)}else{n.CurrentLibrary.appendElement(bA,bz)}}},aF=1,an=function(){return""+aF++},aL=function(bz){return bz._nodes?bz._nodes:bz},be=function(bB,bD,bC){if(!jsPlum
 bAdapter.headless&&!a9){var bE=c(bB,"id"),bz=br.dragManager.getElementsForDraggable(bE);if(bC==null){bC=p()}br.anchorManager.redraw(bE,bD,bC);if(bz){for(var bA in bz){br.anchorManager.redraw(bz[bA].id,bD,bC,bz[bA].offset)}}}},aJ=function(bA,bC){var bD=null;if(l(bA)){bD=[];for(var bz=0;bz<bA.length;bz++){var bB=C(bA[bz]),bE=c(bB,"id");bD.push(bC(bB,bE))}}else{var bB=C(bA),bE=c(bB,"id");bD=bC(bB,bE)}return bD},ax=function(bz){return aW[bz]},bg=function(bD,bz,bG){if(!jsPlumbAdapter.headless){var bI=bz==null?false:bz,bE=n.CurrentLibrary;if(bI){if(bE.isDragSupported(bD)&&!bE.isAlreadyDraggable(bD)){var bH=bG||br.Defaults.DragOptions||n.Defaults.DragOptions;bH=n.extend({},bH);var bF=bE.dragEvents.drag,bA=bE.dragEvents.stop,bC=bE.dragEvents.start;bH[bC]=ap(bH[bC],function(){br.setHoverSuspended(true)});bH[bF]=ap(bH[bF],function(){var bJ=bE.getUIPosition(arguments,br.getZoom());be(bD,bJ);y(bD,"jsPlumb_dragged")});bH[bA]=ap(bH[bA],function(){var bJ=bE.getUIPosition(arguments,br.getZoom());be
 (bD,bJ);m(bD,"jsPlumb_dragged");br.setHoverSuspended(false)});var bB=H(bD);bq[bB]=true;var bI=bq[bB];bH.disabled=bI==null?false:!bI;bE.initDraggable(bD,bH,false);br.dragManager.register(bD)}}}},aC=function(bF,bA){var bz=n.extend({sourceIsNew:true,targetIsNew:true},bF);if(bA){n.extend(bz,bA)}if(bz.source&&bz.source.endpoint){bz.sourceEndpoint=bz.source}if(bz.source&&bz.target.endpoint){bz.targetEndpoint=bz.target}if(bF.uuids){bz.sourceEndpoint=ax(bF.uuids[0]);bz.targetEndpoint=ax(bF.uuids[1])}if(bz.sourceEndpoint&&bz.sourceEndpoint.isFull()){o(br,"could not add connection; source endpoint is full");return}if(bz.targetEndpoint&&bz.targetEndpoint.isFull()){o(br,"could not add connection; target endpoint is full");return}if(bz.sourceEndpoint){bz.sourceIsNew=false}if(bz.targetEndpoint){bz.targetIsNew=false}if(!bz.type&&bz.sourceEndpoint){bz.type=bz.sourceEndpoint.connectionType}if(bz.sourceEndpoint&&bz.sourceEndpoint.connectorOverlays){bz.overlays=bz.overlays||[];for(var bD=0;bD<bz.sourc
 eEndpoint.connectorOverlays.length;bD++){bz.overlays.push(bz.sourceEndpoint.connectorOverlays[bD])}}bz.tooltip=bF.tooltip;if(!bz.tooltip&&bz.sourceEndpoint&&bz.sourceEndpoint.connectorTooltip){bz.tooltip=bz.sourceEndpoint.connectorTooltip}if(bz.target&&!bz.target.endpoint&&!bz.targetEndpoint&&!bz.newConnection){var bE=H(bz.target),bG=a1[bE],bB=aK[bE];if(bG){if(!am[bE]){return}var bC=bB!=null?bB:br.addEndpoint(bz.target,bG);if(bl[bE]){aK[bE]=bC}bz.targetEndpoint=bC;bC._makeTargetCreator=true;bz.targetIsNew=true}}if(bz.source&&!bz.source.endpoint&&!bz.sourceEndpoint&&!bz.newConnection){var bE=H(bz.source),bG=aB[bE],bB=a6[bE];if(bG){if(!ae[bE]){return}var bC=bB!=null?bB:br.addEndpoint(bz.source,bG);if(bc[bE]){a6[bE]=bC}bz.sourceEndpoint=bC;bz.sourceIsNew=true}}return bz},af=function(bD){var bC=br.Defaults.ConnectionType||br.getDefaultConnectionType(),bB=br.Defaults.EndpointType||ah,bA=n.CurrentLibrary.getParent;if(bD.container){bD.parent=bD.container}else{if(bD.sourceEndpoint){bD.paren
 t=bD.sourceEndpoint.parent}else{if(bD.source.constructor==bB){bD.parent=bD.source.parent}else{bD.parent=bA(bD.source)}}}bD._jsPlumb=br;var bz=new bC(bD);bz.id="con_"+an();bw("click","click",bz);bw("dblclick","dblclick",bz);bw("contextmenu","contextmenu",bz);return bz},bx=function(bB,bC,bz){bC=bC||{};if(!bB.suspendedEndpoint){W(a0,bB.scope,bB)}if(!bC.doNotFireConnectionEvent&&bC.fireEvent!==false){var bA={connection:bB,source:bB.source,target:bB.target,sourceId:bB.sourceId,targetId:bB.targetId,sourceEndpoint:bB.endpoints[0],targetEndpoint:bB.endpoints[1]};br.fire("jsPlumbConnection",bA,bz);br.fire("connection",bA,bz)}br.anchorManager.newConnection(bB);be(bB.source)},bw=function(bz,bA,bB){bB.bind(bz,function(bD,bC){br.fire(bA,bB,bC)})},ay=function(bB){if(bB.container){return bB.container}else{var bz=n.CurrentLibrary.getTagName(bB.source),bA=n.CurrentLibrary.getParent(bB.source);if(bz&&bz.toLowerCase()==="td"){return n.CurrentLibrary.getParent(bA)}else{return bA}}},aH=function(bB){var 
 bA=br.Defaults.EndpointType||ah;bB.parent=ay(bB);bB._jsPlumb=br;var bz=new bA(bB);bz.id="ep_"+an();bw("click","endpointClick",bz);bw("dblclick","endpointDblClick",bz);bw("contextmenu","contextmenu",bz);if(!jsPlumbAdapter.headless){br.dragManager.endpointAdded(bB.source)}return bz},V=function(bB,bE,bD){var bz=aV[bB];if(bz&&bz.length){for(var bC=0;bC<bz.length;bC++){for(var bA=0;bA<bz[bC].connections.length;bA++){var bF=bE(bz[bC].connections[bA]);if(bF){return}}if(bD){bD(bz[bC])}}}},Z=function(bA){for(var bz in aV){V(bz,bA)}},aw=function(bz,bA){if(bz!=null&&bz.parentNode!=null){bz.parentNode.removeChild(bz)}},aZ=function(bB,bA){for(var bz=0;bz<bB.length;bz++){aw(bB[bz],bA)}},bo=function(bA,bz){return aJ(bA,function(bB,bC){bq[bC]=bz;if(n.CurrentLibrary.isDragSupported(bB)){n.CurrentLibrary.setDraggable(bB,bz)}})},a7=function(bB,bC,bz){bC=bC==="block";var bA=null;if(bz){if(bC){bA=function(bE){bE.setVisible(true,true,true)}}else{bA=function(bE){bE.setVisible(false,true,true)}}}var bD=c(b
 B,"id");V(bD,function(bF){if(bC&&bz){var bE=bF.sourceId===bD?1:0;if(bF.endpoints[bE].isVisible()){bF.setVisible(true)}}else{bF.setVisible(bC)}},bA)},bm=function(bz){return aJ(bz,function(bB,bA){var bC=bq[bA]==null?false:bq[bA];bC=!bC;bq[bA]=bC;n.CurrentLibrary.setDraggable(bB,bC);return bC})},aS=function(bz,bB){var bA=null;if(bB){bA=function(bC){var bD=bC.isVisible();bC.setVisible(!bD)}}V(bz,function(bD){var bC=bD.isVisible();bD.setVisible(!bC)},bA)},X=function(bE){var bC=bE.timestamp,bz=bE.recalc,bD=bE.offset,bA=bE.elId;if(a9&&!bC){bC=bd}if(!bz){if(bC&&bC===bu[bA]){return aj[bA]}}if(bz||!bD){var bB=C(bA);if(bB!=null){ag[bA]=a(bB);aj[bA]=s(bB,br);bu[bA]=bC}}else{aj[bA]=bD;if(ag[bA]==null){var bB=C(bA);if(bB!=null){ag[bA]=a(bB)}}}if(aj[bA]&&!aj[bA].right){aj[bA].right=aj[bA].left+ag[bA][0];aj[bA].bottom=aj[bA].top+ag[bA][1];aj[bA].width=ag[bA][0];aj[bA].height=ag[bA][1];aj[bA].centerx=aj[bA].left+(aj[bA].width/2);aj[bA].centery=aj[bA].top+(aj[bA].height/2)}return aj[bA]},aQ=function(
 bz){var bA=aj[bz];if(!bA){bA=X({elId:bz})}return{o:bA,s:ag[bz]}},H=function(bz,bA,bC){var bB=C(bz);var bD=c(bB,"id");if(!bD||bD=="undefined"){if(arguments.length==2&&arguments[1]!=undefined){bD=bA}else{if(arguments.length==1||(arguments.length==3&&!arguments[2])){bD="jsPlumb_"+aO+"_"+an()}}if(!bC){d(bB,"id",bD)}}return bD},ap=function(bB,bz,bA){bB=bB||function(){};bz=bz||function(){};return function(){var bC=null;try{bC=bz.apply(this,arguments)}catch(bD){o(br,"jsPlumb function failed : "+bD)}if(bA==null||(bC!==bA)){try{bB.apply(this,arguments)}catch(bD){o(br,"wrapped function failed : "+bD)}}return bC}};this.connectorClass="_jsPlumb_connector";this.endpointClass="_jsPlumb_endpoint";this.overlayClass="_jsPlumb_overlay";this.Anchors={};this.Connectors={canvas:{},svg:{},vml:{}};this.Endpoints={canvas:{},svg:{},vml:{}};this.Overlays={canvas:{},svg:{},vml:{}};this.addClass=function(bA,bz){return n.CurrentLibrary.addClass(bA,bz)};this.removeClass=function(bA,bz){return n.CurrentLibrary.re
 moveClass(bA,bz)};this.hasClass=function(bA,bz){return n.CurrentLibrary.hasClass(bA,bz)};this.addEndpoint=function(bC,bD,bM){bM=bM||{};var bB=n.extend({},bM);n.extend(bB,bD);bB.endpoint=bB.endpoint||br.Defaults.Endpoint||n.Defaults.Endpoint;bB.paintStyle=bB.paintStyle||br.Defaults.EndpointStyle||n.Defaults.EndpointStyle;bC=aL(bC);var bE=[],bH=bC.length&&bC.constructor!=String?bC:[bC];for(var bF=0;bF<bH.length;bF++){var bK=C(bH[bF]),bA=H(bK);bB.source=bK;X({elId:bA,timestamp:bd});var bJ=aH(bB);if(bB.parentAnchor){bJ.parentAnchor=bB.parentAnchor}W(aV,bA,bJ);var bI=aj[bA],bG=ag[bA];var bL=bJ.anchor.compute({xy:[bI.left,bI.top],wh:bG,element:bJ,timestamp:bd});var bz={anchorLoc:bL,timestamp:bd};if(a9){bz.recalc=false}bJ.paint(bz);bE.push(bJ)}return bE.length==1?bE[0]:bE};this.addEndpoints=function(bD,bA,bz){var bC=[];for(var bB=0;bB<bA.length;bB++){var bE=br.addEndpoint(bD,bA[bB],bz);if(l(bE)){Array.prototype.push.apply(bC,bE)}else{bC.push(bE)}}return bC};this.animate=function(bB,bA,bz){
 var bC=C(bB),bF=c(bB,"id");bz=bz||{};var bE=n.CurrentLibrary.dragEvents.step;var bD=n.CurrentLibrary.dragEvents.complete;bz[bE]=ap(bz[bE],function(){br.repaint(bF)});bz[bD]=ap(bz[bD],function(){br.repaint(bF)});n.CurrentLibrary.animate(bC,bA,bz)};this.checkCondition=function(bB,bD){var bz=br.getListener(bB),bC=true;if(bz&&bz.length>0){try{for(var bA=0;bA<bz.length;bA++){bC=bC&&bz[bA](bD)}}catch(bE){o(br,"cannot check condition ["+bB+"]"+bE)}}return bC};this.checkASyncCondition=function(bB,bD,bC,bA){var bz=br.getListener(bB);if(bz&&bz.length>0){try{bz[0](bD,bC,bA)}catch(bE){o(br,"cannot asynchronously check condition ["+bB+"]"+bE)}}};this.connect=function(bC,bA){var bz=aC(bC,bA),bB;if(bz){if(bz.deleteEndpointsOnDetach==null){bz.deleteEndpointsOnDetach=true}bB=af(bz);bx(bB,bz)}return bB};this.deleteEndpoint=function(bA){var bF=(typeof bA=="string")?aW[bA]:bA;if(bF){var bC=bF.getUuid();if(bC){aW[bC]=null}bF.detachAll();if(bF.endpoint.cleanup){bF.endpoint.cleanup()}aZ(bF.endpoint.getDis
 playElements());br.anchorManager.deleteEndpoint(bF);for(var bE in aV){var bz=aV[bE];if(bz){var bD=[];for(var bB=0;bB<bz.length;bB++){if(bz[bB]!=bF){bD.push(bz[bB])}}aV[bE]=bD}}if(!jsPlumbAdapter.headless){br.dragManager.endpointDeleted(bF)}}};this.deleteEveryEndpoint=function(){br.setSuspendDrawing(true);for(var bB in aV){var bz=aV[bB];if(bz&&bz.length){for(var bA=0;bA<bz.length;bA++){br.deleteEndpoint(bz[bA])}}}aV={};aW={};br.setSuspendDrawing(false,true)};var ba=function(bC,bE,bz){var bB=br.Defaults.ConnectionType||br.getDefaultConnectionType(),bA=bC.constructor==bB,bD=bA?{connection:bC,source:bC.source,target:bC.target,sourceId:bC.sourceId,targetId:bC.targetId,sourceEndpoint:bC.endpoints[0],targetEndpoint:bC.endpoints[1]}:bC;if(bE){br.fire("jsPlumbConnectionDetached",bD,bz);br.fire("connectionDetached",bD,bz)}br.anchorManager.connectionDetached(bD)},a8=function(bz){br.fire("connectionDrag",bz)},aT=function(bz){br.fire("connectionDragStop",bz)};this.detach=function(){if(arguments.
 length==0){return}var bD=br.Defaults.ConnectionType||br.getDefaultConnectionType(),bE=arguments[0].constructor==bD,bC=arguments.length==2?bE?(arguments[1]||{}):arguments[0]:arguments[0],bH=(bC.fireEvent!==false),bB=bC.forceDetach,bA=bE?arguments[0]:bC.connection;if(bA){if(bB||(bA.isDetachAllowed(bA)&&bA.endpoints[0].isDetachAllowed(bA)&&bA.endpoints[1].isDetachAllowed(bA))){if(bB||br.checkCondition("beforeDetach",bA)){bA.endpoints[0].detach(bA,false,true,bH)}}}else{var bz=n.extend({},bC);if(bz.uuids){ax(bz.uuids[0]).detachFrom(ax(bz.uuids[1]),bH)}else{if(bz.sourceEndpoint&&bz.targetEndpoint){bz.sourceEndpoint.detachFrom(bz.targetEndpoint)}else{var bG=H(bz.source),bF=H(bz.target);V(bG,function(bI){if((bI.sourceId==bG&&bI.targetId==bF)||(bI.targetId==bG&&bI.sourceId==bF)){if(br.checkCondition("beforeDetach",bI)){bI.endpoints[0].detach(bI,false,true,bH)}}})}}}};this.detachAllConnections=function(bB,bC){bC=bC||{};bB=C(bB);var bD=c(bB,"id"),bz=aV[bD];if(bz&&bz.length){for(var bA=0;bA<bz.
 length;bA++){bz[bA].detachAll(bC.fireEvent)}}};this.detachEveryConnection=function(bB){bB=bB||{};for(var bC in aV){var bz=aV[bC];if(bz&&bz.length){for(var bA=0;bA<bz.length;bA++){bz[bA].detachAll(bB.fireEvent)}}}a0={}};this.draggable=function(bB,bz){if(typeof bB=="object"&&bB.length){for(var bA=0;bA<bB.length;bA++){var bC=C(bB[bA]);if(bC){bg(bC,true,bz)}}}else{if(bB._nodes){for(var bA=0;bA<bB._nodes.length;bA++){var bC=C(bB._nodes[bA]);if(bC){bg(bC,true,bz)}}}else{var bC=C(bB);if(bC){bg(bC,true,bz)}}}};this.extend=function(bA,bz){return n.CurrentLibrary.extend(bA,bz)};this.getDefaultEndpointType=function(){return ah};this.getDefaultConnectionType=function(){return az};var bt=function(bD,bC,bA,bz){for(var bB=0;bB<bD.length;bB++){bD[bB][bC].apply(bD[bB],bA)}return bz(bD)},T=function(bD,bC,bA){var bz=[];for(var bB=0;bB<bD.length;bB++){bz.push([bD[bB][bC].apply(bD[bB],bA),bD[bB]])}return bz},aq=function(bB,bA,bz){return function(){return bt(bB,bA,arguments,bz)}},aA=function(bA,bz){retur
 n function(){return T(bA,bz,arguments)}},bv=function(bz,bC){var bB=[];if(bz){if(typeof bz=="string"){if(bz==="*"){return bz}bB.push(bz)}else{if(bC){bB=bz}else{for(var bA=0;bA<bz.length;bA++){bB.push(H(C(bz[bA])))}}}}return bB},aI=function(bB,bA,bz){if(bB==="*"){return true}return bB.length>0?G(bB,bA)!=-1:!bz};this.getConnections=function(bI,bA){if(!bI){bI={}}else{if(bI.constructor==String){bI={scope:bI}}}var bH=bI.scope||br.getDefaultScope(),bG=bv(bH,true),bz=bv(bI.source),bE=bv(bI.target),bD=(!bA&&bG.length>1)?{}:[],bJ=function(bL,bM){if(!bA&&bG.length>1){var bK=bD[bL];if(bK==null){bK=[];bD[bL]=bK}bK.push(bM)}else{bD.push(bM)}};for(var bC in a0){if(aI(bG,bC)){for(var bB=0;bB<a0[bC].length;bB++){var bF=a0[bC][bB];if(aI(bz,bF.sourceId)&&aI(bE,bF.targetId)){bJ(bC,bF)}}}}return bD};var L=function(bz,bA){return function(bC){for(var bB=0;bB<bz.length;bB++){bC(bz[bB])}return bA(bz)}},P=function(bz){return function(bA){return bz[bA]}};var Q=function(bz,bA){return{setHover:aq(bz,"setHover",
 bA),removeAllOverlays:aq(bz,"removeAllOverlays",bA),setLabel:aq(bz,"setLabel",bA),addOverlay:aq(bz,"addOverlay",bA),removeOverlay:aq(bz,"removeOverlay",bA),removeOverlays:aq(bz,"removeOverlays",bA),showOverlay:aq(bz,"showOverlay",bA),hideOverlay:aq(bz,"hideOverlay",bA),showOverlays:aq(bz,"showOverlays",bA),hideOverlays:aq(bz,"hideOverlays",bA),setPaintStyle:aq(bz,"setPaintStyle",bA),setHoverPaintStyle:aq(bz,"setHoverPaintStyle",bA),setParameter:aq(bz,"setParameter",bA),setParameters:aq(bz,"setParameters",bA),setVisible:aq(bz,"setVisible",bA),setZIndex:aq(bz,"setZIndex",bA),repaint:aq(bz,"repaint",bA),addType:aq(bz,"addType",bA),toggleType:aq(bz,"toggleType",bA),removeType:aq(bz,"removeType",bA),getLabel:aA(bz,"getLabel"),getOverlay:aA(bz,"getOverlay"),isHover:aA(bz,"isHover"),getParameter:aA(bz,"getParameter"),getParameters:aA(bz,"getParameters"),getPaintStyle:aA(bz,"getPaintStyle"),getHoverPaintStyle:aA(bz,"getHoverPaintStyle"),isVisible:aA(bz,"isVisible"),getZIndex:aA(bz,"getZInde
 x"),hasType:aA(bz,"hasType"),getType:aA(bz,"getType"),length:bz.length,each:L(bz,bA),get:P(bz)}};var aN=function(bA){var bz=Q(bA,aN);return n.CurrentLibrary.extend(bz,{setDetachable:aq(bA,"setDetachable",aN),setReattach:aq(bA,"setReattach",aN),setConnector:aq(bA,"setConnector",aN),detach:function(){for(var bB=0;bB<bA.length;bB++){br.detach(bA[bB])}},isDetachable:aA(bA,"isDetachable"),isReattach:aA(bA,"isReattach")})};var bf=function(bA){var bz=Q(bA,bf);return n.CurrentLibrary.extend(bz,{setEnabled:aq(bA,"setEnabled",bf),isEnabled:aA(bA,"isEnabled"),detachAll:function(){for(var bB=0;bB<bA.length;bB++){bA[bB].detachAll()}},"delete":function(){for(var bB=0;bB<bA.length;bB++){br.deleteEndpoint(bA[bB])}}})};this.select=function(bz){bz=bz||{};bz.scope=bz.scope||"*";var bA=br.getConnections(bz,true);return aN(bA)};this.selectEndpoints=function(bH){bH=bH||{};bH.scope=bH.scope||"*";var bM=!bH.element&&!bH.source&&!bH.target,bA=bM?"*":bv(bH.element),bB=bM?"*":bv(bH.source),bJ=bM?"*":bv(bH.tar
 get),bO=bv(bH.scope,true);var bP=[];for(var bF in aV){var bD=aI(bA,bF,true),bz=aI(bB,bF,true),bE=bB!="*",bK=aI(bJ,bF,true),bC=bJ!="*";if(bD||bz||bK){inner:for(var bI=0;bI<aV[bF].length;bI++){var bL=aV[bF][bI];if(aI(bO,bL.scope,true)){var bN=(bE&&bB.length>0&&!bL.isSource),bG=(bC&&bJ.length>0&&!bL.isTarget);if(bN||bG){continue inner}bP.push(bL)}}}}return bf(bP)};this.getAllConnections=function(){return a0};this.getDefaultScope=function(){return R};this.getEndpoint=ax;this.getEndpoints=function(bz){return aV[H(bz)]};this.getId=H;this.getOffset=function(bA){var bz=aj[bA];return X({elId:bA})};this.getSelector=function(bz){return n.CurrentLibrary.getSelector(bz)};this.getSize=function(bA){var bz=ag[bA];if(!bz){X({elId:bA})}return ag[bA]};this.appendElement=aX;var aU=false;this.isHoverSuspended=function(){return aU};this.setHoverSuspended=function(bz){aU=bz};var aY=function(bz){return function(){return jsPlumbAdapter.isRenderModeAvailable(bz)}};this.isCanvasAvailable=aY("canvas");this.isS
 VGAvailable=aY("svg");this.isVMLAvailable=aY("vml");this.hide=function(bz,bA){a7(bz,"none",bA)};this.idstamp=an;this.init=function(){if(!K){br.setRenderMode(br.Defaults.RenderMode);var bz=function(bA){n.CurrentLibrary.bind(document,bA,function(bG){if(!br.currentlyDragging&&Y==n.CANVAS){for(var bF in a0){var bH=a0[bF];for(var bD=0;bD<bH.length;bD++){var bC=bH[bD].connector[bA](bG);if(bC){return}}}for(var bE in aV){var bB=aV[bE];for(var bD=0;bD<bB.length;bD++){if(bB[bD].endpoint[bA](bG)){return}}}}})};bz("click");bz("dblclick");bz("mousemove");bz("mousedown");bz("mouseup");bz("contextmenu");K=true;br.fire("ready")}};this.log=O;this.jsPlumbUIComponent=B;this.makeAnchor=function(){if(arguments.length==0){return null}var bE=arguments[0],bB=arguments[1],bA=arguments[2],bC=null;if(bE.compute&&bE.getOrientation){return bE}else{if(typeof bE=="string"){bC=n.Anchors[arguments[0]]({elementId:bB,jsPlumbInstance:br})}else{if(l(bE)){if(l(bE[0])||z(bE[0])){if(bE.length==2&&z(bE[0])&&u(bE[1])){var b
 z=n.extend({elementId:bB,jsPlumbInstance:br},bE[1]);bC=n.Anchors[bE[0]](bz)}else{bC=new au(bE,null,bB)}}else{var bD={x:bE[0],y:bE[1],orientation:(bE.length>=4)?[bE[2],bE[3]]:[0,0],offsets:(bE.length==6)?[bE[4],bE[5]]:[0,0],elementId:bB};bC=new ab(bD);bC.clone=function(){return new ab(bD)}}}}}if(!bC.id){bC.id="anchor_"+an()}return bC};this.makeAnchors=function(bC,bA,bz){var bD=[];for(var bB=0;bB<bC.length;bB++){if(typeof bC[bB]=="string"){bD.push(n.Anchors[bC[bB]]({elementId:bA,jsPlumbInstance:bz}))}else{if(l(bC[bB])){bD.push(br.makeAnchor(bC[bB],bA,bz))}}}return bD};this.makeDynamicAnchor=function(bz,bA){return new au(bz,bA)};var a1={},aK={},bl={},at={},ac=function(bz,bA){bz.paintStyle=bz.paintStyle||br.Defaults.EndpointStyles[bA]||br.Defaults.EndpointStyle||n.Defaults.EndpointStyles[bA]||n.Defaults.EndpointStyle;bz.hoverPaintStyle=bz.hoverPaintStyle||br.Defaults.EndpointHoverStyles[bA]||br.Defaults.EndpointHoverStyle||n.Defaults.EndpointHoverStyles[bA]||n.Defaults.EndpointHoverStyl
 e;bz.anchor=bz.anchor||br.Defaults.Anchors[bA]||br.Defaults.Anchor||n.Defaults.Anchors[bA]||n.Defaults.Anchor;bz.endpoint=bz.endpoint||br.Defaults.Endpoints[bA]||br.Defaults.Endpoint||n.Defaults.Endpoints[bA]||n.Defaults.Endpoint};this.makeTarget=function(bC,bD,bJ){var bA=n.extend({_jsPlumb:br},bJ);n.extend(bA,bD);ac(bA,1);var bH=n.CurrentLibrary,bI=bA.scope||br.Defaults.Scope,bE=!(bA.deleteEndpointsOnDetach===false),bB=bA.maxConnections||-1,bz=bA.onMaxConnections;_doOne=function(bO){var bM=H(bO);a1[bM]=bA;bl[bM]=bA.uniqueEndpoint,at[bM]=bB,am[bM]=true,proxyComponent=new B(bA);var bL=n.extend({},bA.dropOptions||{}),bK=function(){var bR=n.CurrentLibrary.getDropEvent(arguments),bT=br.select({target:bM}).length;br.currentlyDragging=false;var b3=C(bH.getDragObject(arguments)),bS=c(b3,"dragId"),b1=c(b3,"originalScope"),bY=bk[bS],bQ=bY.endpoints[0],bP=bA.endpoint?n.extend({},bA.endpoint):{};if(!am[bM]||at[bM]>0&&bT>=at[bM]){if(bz){bz({element:bO,connection:bY},bR)}return false}bQ.anchor.l
 ocked=false;if(b1){bH.setDragScope(b3,b1)}var bW=proxyComponent.isDropAllowed(bY.sourceId,H(bO),bY.scope,bY,null);if(bY.endpointsToDeleteOnDetach){if(bQ===bY.endpointsToDeleteOnDetach[0]){bY.endpointsToDeleteOnDetach[0]=null}else{if(bQ===bY.endpointsToDeleteOnDetach[1]){bY.endpointsToDeleteOnDetach[1]=null}}}if(bY.suspendedEndpoint){bY.targetId=bY.suspendedEndpoint.elementId;bY.target=bH.getElementObject(bY.suspendedEndpoint.elementId);bY.endpoints[1]=bY.suspendedEndpoint}if(bW){bQ.detach(bY,false,true,false);var b2=aK[bM]||br.addEndpoint(bO,bA);if(bA.uniqueEndpoint){aK[bM]=b2}b2._makeTargetCreator=true;if(b2.anchor.positionFinder!=null){var bZ=bH.getUIPosition(arguments,br.getZoom()),bV=s(bO,br),b0=a(bO),bU=b2.anchor.positionFinder(bZ,bV,b0,b2.anchor.constructorParams);b2.anchor.x=bU[0];b2.anchor.y=bU[1]}var bX=br.connect({source:bQ,target:b2,scope:b1,previousConnection:bY,container:bY.parent,deleteEndpointsOnDetach:bE,doNotFireConnectionEvent:bQ.endpointWillMoveAfterConnection});i
 f(bY.endpoints[1]._makeTargetCreator&&bY.endpoints[1].connections.length<2){br.deleteEndpoint(bY.endpoints[1])}if(bE){bX.endpointsToDeleteOnDetach=[bQ,b2]}bX.repaint()}else{if(bY.suspendedEndpoint){if(bY.isReattach()){bY.setHover(false);bY.floatingAnchorIndex=null;bY.suspendedEndpoint.addConnection(bY);br.repaint(bQ.elementId)}else{bQ.detach(bY,false,true,true,bR)}}}};var bN=bH.dragEvents.drop;bL.scope=bL.scope||bI;bL[bN]=ap(bL[bN],bK);bH.initDroppable(bO,bL,true)};bC=aL(bC);var bG=bC.length&&bC.constructor!=String?bC:[bC];for(var bF=0;bF<bG.length;bF++){_doOne(C(bG[bF]))}return br};this.unmakeTarget=function(bA,bB){bA=n.CurrentLibrary.getElementObject(bA);var bz=H(bA);if(!bB){delete a1[bz];delete bl[bz];delete at[bz];delete am[bz]}return br};this.makeTargets=function(bB,bC,bz){for(var bA=0;bA<bB.length;bA++){br.makeTarget(bB[bA],bC,bz)}};var aB={},a6={},bc={},ae={},J={},N={},am={};this.makeSource=function(bD,bE,bI){var bB=n.extend({},bI);n.extend(bB,bE);ac(bB,0);var bH=n.CurrentLib
 rary,bC=bB.maxConnections||-1,bA=bB.onMaxConnections,bz=function(bQ){var bK=H(bQ),bR=bB.parent,bJ=bR!=null?br.getId(bH.getElementObject(bR)):bK;aB[bJ]=bB;bc[bJ]=bB.uniqueEndpoint;ae[bJ]=true;var bL=bH.dragEvents.stop,bP=bH.dragEvents.drag,bS=n.extend({},bB.dragOptions||{}),bN=bS.drag,bT=bS.stop,bU=null,bO=false;N[bJ]=bC;bS.scope=bS.scope||bB.scope;bS[bP]=ap(bS[bP],function(){if(bN){bN.apply(this,arguments)}bO=false});bS[bL]=ap(bS[bL],function(){if(bT){bT.apply(this,arguments)}br.currentlyDragging=false;if(bU.connections.length==0){br.deleteEndpoint(bU)}else{bH.unbind(bU.canvas,"mousedown");var bW=bB.anchor||br.Defaults.Anchor,bX=bU.anchor,bZ=bU.connections[0];bU.anchor=br.makeAnchor(bW,bK,br);if(bB.parent){var bY=bH.getElementObject(bB.parent);if(bY){var bV=bU.elementId;var b0=bB.container||br.Defaults.Container||n.Defaults.Container;bU.setElement(bY,b0);bU.endpointWillMoveAfterConnection=false;br.anchorManager.rehomeEndpoint(bV,bY);bZ.previousConnection=null;A(a0[bZ.scope],function
 (b1){return b1.id===bZ.id});br.anchorManager.connectionDetached({sourceId:bZ.sourceId,targetId:bZ.targetId,connection:bZ});bx(bZ)}}bU.repaint();br.repaint(bU.elementId);br.repaint(bZ.targetId)}});var bM=function(bZ){if(!ae[bJ]){return}var bX=br.select({source:bJ}).length;if(N[bJ]>=0&&bX>=N[bJ]){if(bA){bA({element:bQ,maxConnections:bC},bZ)}return false}if(bE.filter){var bV=bE.filter(bH.getOriginalEvent(bZ),bQ);if(bV===false){return}}var b3=X({elId:bK});var b2=((bZ.pageX||bZ.page.x)-b3.left)/b3.width,b1=((bZ.pageY||bZ.page.y)-b3.top)/b3.height,b7=b2,b6=b1;if(bB.parent){var b0=bH.getElementObject(bB.parent),bY=H(b0);b3=X({elId:bY});b7=((bZ.pageX||bZ.page.x)-b3.left)/b3.width,b6=((bZ.pageY||bZ.page.y)-b3.top)/b3.height}var b5={};n.extend(b5,bB);b5.isSource=true;b5.anchor=[b2,b1,0,0];b5.parentAnchor=[b7,b6,0,0];b5.dragOptions=bS;if(bB.parent){var bW=b5.container||br.Defaults.Container||n.Defaults.Container;if(bW){b5.container=bW}else{b5.container=n.CurrentLibrary.getParent(bB.parent)}}bU
 =br.addEndpoint(bK,b5);bO=true;bU.endpointWillMoveAfterConnection=bB.parent!=null;bU.endpointWillMoveTo=bB.parent?bH.getElementObject(bB.parent):null;var b4=function(){if(bO){br.deleteEndpoint(bU)}};br.registerListener(bU.canvas,"mouseup",b4);br.registerListener(bQ,"mouseup",b4);bH.trigger(bU.canvas,"mousedown",bZ)};br.registerListener(bQ,"mousedown",bM);J[bK]=bM};bD=aL(bD);var bG=bD.length&&bD.constructor!=String?bD:[bD];for(var bF=0;bF<bG.length;bF++){bz(C(bG[bF]))}return br};this.unmakeSource=function(bA,bB){bA=n.CurrentLibrary.getElementObject(bA);var bC=H(bA),bz=J[bC];if(bz){br.unregisterListener(_el,"mousedown",bz)}if(!bB){delete aB[bC];delete bc[bC];delete ae[bC];delete J[bC];delete N[bC]}return br};this.unmakeEverySource=function(){for(var bz in ae){br.unmakeSource(bz,true)}aB={};bc={};ae={};J={}};this.unmakeEveryTarget=function(){for(var bz in am){br.unmakeTarget(bz,true)}a1={};bl={};at={};am={};return br};this.makeSources=function(bB,bC,bz){for(var bA=0;bA<bB.length;bA++){
 br.makeSource(bB[bA],bC,bz)}return br};var aP=function(bD,bC,bE,bz){var bA=bD=="source"?ae:am;if(z(bC)){bA[bC]=bz?!bA[bC]:bE}else{if(bC.length){bC=aL(bC);for(var bB=0;bB<bC.length;bB++){var bF=_el=n.CurrentLibrary.getElementObject(bC[bB]),bF=H(_el);bA[bF]=bz?!bA[bF]:bE}}}return br};this.setSourceEnabled=function(bz,bA){return aP("source",bz,bA)};this.toggleSourceEnabled=function(bz){aP("source",bz,null,true);return br.isSourceEnabled(bz)};this.isSource=function(bz){bz=n.CurrentLibrary.getElementObject(bz);return ae[H(bz)]!=null};this.isSourceEnabled=function(bz){bz=n.CurrentLibrary.getElementObject(bz);return ae[H(bz)]===true};this.setTargetEnabled=function(bz,bA){return aP("target",bz,bA)};this.toggleTargetEnabled=function(bz){return aP("target",bz,null,true);return br.isTargetEnabled(bz)};this.isTarget=function(bz){bz=n.CurrentLibrary.getElementObject(bz);return am[H(bz)]!=null};this.isTargetEnabled=function(bz){bz=n.CurrentLibrary.getElementObject(bz);return am[H(bz)]===true};thi
 s.ready=function(bz){br.bind("ready",bz)};this.repaint=function(bA,bC,bB){if(typeof bA=="object"){for(var bz=0;bz<bA.length;bz++){be(C(bA[bz]),bC,bB)}}else{be(C(bA),bC,bB)}return br};this.repaintEverything=function(){for(var bz in aV){be(C(bz))}return br};this.removeAllEndpoints=function(bB){var bz=c(bB,"id"),bC=aV[bz];if(bC){for(var bA=0;bA<bC.length;bA++){br.deleteEndpoint(bC[bA])}}aV[bz]=[];return br};var ai={},bi=function(){for(var bA in ai){for(var bz=0;bz<ai[bA].length;bz++){var bB=ai[bA][bz];n.CurrentLibrary.unbind(bB.el,bB.event,bB.listener)}}ai={}};this.registerListener=function(bA,bz,bB){n.CurrentLibrary.bind(bA,bz,bB);W(ai,bz,{el:bA,event:bz,listener:bB})};this.unregisterListener=function(bA,bz,bB){n.CurrentLibrary.unbind(bA,bz,bB);A(ai,function(bC){return bC.type==bz&&bC.listener==bB})};this.reset=function(){br.deleteEveryEndpoint();br.unbind();a1={};aK={};bl={};at={};aB={};a6={};bc={};N={};bi();br.anchorManager.reset();if(!jsPlumbAdapter.headless){br.dragManager.reset()
 }};this.setDefaultScope=function(bz){R=bz;return br};this.setDraggable=bo;this.setId=function(bD,bz,bF){var bG=bD.constructor==String?bD:br.getId(bD),bC=br.getConnections({source:bG,scope:"*"},true),bB=br.getConnections({target:bG,scope:"*"},true);bz=""+bz;if(!bF){bD=n.CurrentLibrary.getElementObject(bG);n.CurrentLibrary.setAttribute(bD,"id",bz)}bD=n.CurrentLibrary.getElementObject(bz);aV[bz]=aV[bG]||[];for(var bA=0;bA<aV[bz].length;bA++){aV[bz][bA].elementId=bz;aV[bz][bA].element=bD;aV[bz][bA].anchor.elementId=bz}delete aV[bG];br.anchorManager.changeId(bG,bz);var bE=function(bK,bH,bJ){for(var bI=0;bI<bK.length;bI++){bK[bI].endpoints[bH].elementId=bz;bK[bI].endpoints[bH].element=bD;bK[bI][bJ+"Id"]=bz;bK[bI][bJ]=bD}};bE(bC,0,"source");bE(bB,1,"target")};this.setIdChanged=function(bA,bz){br.setId(bA,bz,true)};this.setDebugLog=function(bz){O=bz};var a9=false,bd=null;this.setSuspendDrawing=function(bA,bz){a9=bA;if(bA){bd=new Date().getTime()}else{bd=null}if(bz){br.repaintEverything()}};
 this.isSuspendDrawing=function(){return a9};this.CANVAS="canvas";this.SVG="svg";this.VML="vml";this.setRenderMode=function(bz){Y=jsPlumbAdapter.setRenderMode(bz);return Y};this.getRenderMode=function(){return Y};this.show=function(bz,bA){a7(bz,"block",bA);return br};this.sizeCanvas=function(bB,bz,bD,bA,bC){if(bB){bB.style.height=bC+"px";bB.height=bC;bB.style.width=bA+"px";bB.width=bA;bB.style.left=bz+"px";bB.style.top=bD+"px"}return br};this.getTestHarness=function(){return{endpointsByElement:aV,endpointCount:function(bz){var bA=aV[bz];return bA?bA.length:0},connectionCount:function(bz){bz=bz||R;var bA=a0[bz];return bA?bA.length:0},getId:H,makeAnchor:self.makeAnchor,makeDynamicAnchor:self.makeDynamicAnchor}};this.toggle=aS;this.toggleVisible=aS;this.toggleDraggable=bm;this.wrap=ap;this.addListener=this.bind;var bs=function(bE,bB){var bC=null,bz=bE;if(bB.tagName.toLowerCase()==="svg"&&bB.parentNode){bC=bB.parentNode}else{if(bB.offsetParent){bC=bB.offsetParent}}if(bC!=null){var bA=bC.
 tagName.toLowerCase()==="body"?{left:0,top:0}:s(bC,br),bD=bC.tagName.toLowerCase()==="body"?{left:0,top:0}:{left:bC.scrollLeft,top:bC.scrollTop};bz[0]=bE[0]-bA.left+bD.left;bz[1]=bE[1]-bA.top+bD.top}return bz};var ab=function(bD){var bB=this;this.x=bD.x||0;this.y=bD.y||0;this.elementId=bD.elementId;var bA=bD.orientation||[0,0];var bC=null,bz=null;this.offsets=bD.offsets||[0,0];bB.timestamp=null;this.compute=function(bI){var bH=bI.xy,bE=bI.wh,bF=bI.element,bG=bI.timestamp;if(bG&&bG===bB.timestamp){return bz}bz=[bH[0]+(bB.x*bE[0])+bB.offsets[0],bH[1]+(bB.y*bE[1])+bB.offsets[1]];bz=bs(bz,bF.canvas);bB.timestamp=bG;return bz};this.getOrientation=function(bE){return bA};this.equals=function(bE){if(!bE){return false}var bF=bE.getOrientation();var bG=this.getOrientation();return this.x==bE.x&&this.y==bE.y&&this.offsets[0]==bE.offsets[0]&&this.offsets[1]==bE.offsets[1]&&bG[0]==bF[0]&&bG[1]==bF[1]};this.getCurrentLocation=function(){return bz}};var a3=function(bF){var bD=bF.reference,bE=bF.r
 eferenceCanvas,bB=a(C(bE)),bA=0,bG=0,bz=null,bC=null;this.x=0;this.y=0;this.isFloating=true;this.compute=function(bK){var bJ=bK.xy,bI=bK.element,bH=[bJ[0]+(bB[0]/2),bJ[1]+(bB[1]/2)];bH=bs(bH,bI.canvas);bC=bH;return bH};this.getOrientation=function(bI){if(bz){return bz}else{var bH=bD.getOrientation(bI);return[Math.abs(bH[0])*bA*-1,Math.abs(bH[1])*bG*-1]}};this.over=function(bH){bz=bH.getOrientation()};this.out=function(){bz=null};this.getCurrentLocation=function(){return bC}};var au=function(bB,bA,bG){this.isSelective=true;this.isDynamic=true;var bJ=[],bI=this,bH=function(bK){return bK.constructor==ab?bK:br.makeAnchor(bK,bG,br)};for(var bF=0;bF<bB.length;bF++){bJ[bF]=bH(bB[bF])}this.addAnchor=function(bK){bJ.push(bH(bK))};this.getAnchors=function(){return bJ};this.locked=false;var bC=bJ.length>0?bJ[0]:null,bE=bJ.length>0?0:-1,bI=this,bD=function(bQ,bN,bM,bR,bL){var bK=bR[0]+(bQ.x*bL[0]),bS=bR[1]+(bQ.y*bL[1]),bP=bR[0]+(bL[0]/2),bO=bR[1]+(bL[1]/2);return(Math.sqrt(Math.pow(bN-bK,2)+Mat
 h.pow(bM-bS,2))+Math.sqrt(Math.pow(bP-bK,2)+Math.pow(bO-bS,2)))},bz=bA||function(bU,bL,bM,bN,bK){var bP=bM[0]+(bN[0]/2),bO=bM[1]+(bN[1]/2);var bR=-1,bT=Infinity;for(var bQ=0;bQ<bK.length;bQ++){var bS=bD(bK[bQ],bP,bO,bU,bL);if(bS<bT){bR=bQ+0;bT=bS}}return bK[bR]};this.compute=function(bO){var bN=bO.xy,bK=bO.wh,bM=bO.timestamp,bL=bO.txy,bP=bO.twh;if(bI.locked||bL==null||bP==null){return bC.compute(bO)}else{bO.timestamp=null}bC=bz(bN,bK,bL,bP,bJ);bI.x=bC.x;bI.y=bC.y;return bC.compute(bO)};this.getCurrentLocation=function(){return bC!=null?bC.getCurrentLocation():null};this.getOrientation=function(bK){return bC!=null?bC.getOrientation(bK):[0,0]};this.over=function(bK){if(bC!=null){bC.over(bK)}};this.out=function(){if(bC!=null){bC.out()}}};var bn={},ak={},aM={},U={HORIZONTAL:"horizontal",VERTICAL:"vertical",DIAGONAL:"diagonal",IDENTITY:"identity"},bp=function(bI,bJ,bF,bC){if(bI===bJ){return{orientation:U.IDENTITY,a:["top","top"]}}var bA=Math.atan2((bC.centery-bF.centery),(bC.centerx-bF.c
 enterx)),bD=Math.atan2((bF.centery-bC.centery),(bF.centerx-bC.centerx)),bE=((bF.left<=bC.left&&bF.right>=bC.left)||(bF.left<=bC.right&&bF.right>=bC.right)||(bF.left<=bC.left&&bF.right>=bC.right)||(bC.left<=bF.left&&bC.right>=bF.right)),bK=((bF.top<=bC.top&&bF.bottom>=bC.top)||(bF.top<=bC.bottom&&bF.bottom>=bC.bottom)||(bF.top<=bC.top&&bF.bottom>=bC.bottom)||(bC.top<=bF.top&&bC.bottom>=bF.bottom));if(!(bE||bK)){var bH=null,bB=false,bz=false,bG=null;if(bC.left>bF.left&&bC.top>bF.top){bH=["right","top"]}else{if(bC.left>bF.left&&bF.top>bC.top){bH=["top","left"]}else{if(bC.left<bF.left&&bC.top<bF.top){bH=["top","right"]}else{if(bC.left<bF.left&&bC.top>bF.top){bH=["left","top"]}}}}return{orientation:U.DIAGONAL,a:bH,theta:bA,theta2:bD}}else{if(bE){return{orientation:U.HORIZONTAL,a:bF.top<bC.top?["bottom","top"]:["top","bottom"],theta:bA,theta2:bD}}else{return{orientation:U.VERTICAL,a:bF.left<bC.left?["right","left"]:["left","right"],theta:bA,theta2:bD}}}},a5=function(bN,bJ,bH,bI,bO,bK,bB){
 var bP=[],bA=bJ[bO?0:1]/(bI.length+1);for(var bL=0;bL<bI.length;bL++){var bQ=(bL+1)*bA,bz=bK*bJ[bO?1:0];if(bB){bQ=bJ[bO?0:1]-bQ}var bG=(bO?bQ:bz),bD=bH[0]+bG,bF=bG/bJ[0],bE=(bO?bz:bQ),bC=bH[1]+bE,bM=bE/bJ[1];bP.push([bD,bC,bF,bM,bI[bL][1],bI[bL][2]])}return bP},bb=function(bA,bz){return bA[0]>bz[0]?1:-1},aa=function(bz){return function(bB,bA){var bC=true;if(bz){if(bB[0][0]<bA[0][0]){bC=true}else{bC=bB[0][1]>bA[0][1]}}else{if(bB[0][0]>bA[0][0]){bC=true}else{bC=bB[0][1]>bA[0][1]}}return bC===false?-1:1}},M=function(bA,bz){var bC=bA[0][0]<0?-Math.PI-bA[0][0]:Math.PI-bA[0][0],bB=bz[0][0]<0?-Math.PI-bz[0][0]:Math.PI-bz[0][0];if(bC>bB){return 1}else{return bA[0][1]>bz[0][1]?1:-1}},a2={top:bb,right:aa(true),bottom:aa(true),left:M},ar=function(bz,bA){return bz.sort(bA)},ao=function(bA,bz){var bC=ag[bA],bD=aj[bA],bB=function(bK,bR,bG,bJ,bP,bO,bF){if(bJ.length>0){var bN=ar(bJ,a2[bK]),bL=bK==="right"||bK==="top",bE=a5(bK,bR,bG,bN,bP,bO,bL);var bS=function(bV,bU){var bT=bs([bU[0],bU[1]],bV.canv
 as);ak[bV.id]=[bT[0],bT[1],bU[2],bU[3]];aM[bV.id]=bF};for(var bH=0;bH<bE.length;bH++){var bM=bE[bH][4],bQ=bM.endpoints[0].elementId===bA,bI=bM.endpoints[1].elementId===bA;if(bQ){bS(bM.endpoints[0],bE[bH])}else{if(bI){bS(bM.endpoints[1],bE[bH])}}}}};bB("bottom",bC,[bD.left,bD.top],bz.bottom,true,1,[0,1]);bB("top",bC,[bD.left,bD.top],bz.top,true,0,[0,-1]);bB("left",bC,[bD.left,bD.top],bz.left,false,0,[-1,0]);bB("right",bC,[bD.left,bD.top],bz.right,false,1,[1,0])},aG=function(){var bz={},bD={},bA=this,bC={};this.reset=function(){bz={};bD={};bC={}};this.newConnection=function(bH){var bJ=bH.sourceId,bG=bH.targetId,bE=bH.endpoints,bI=true,bF=function(bK,bL,bN,bM,bO){if((bJ==bG)&&bN.isContinuous){n.CurrentLibrary.removeElement(bE[1].canvas);bI=false}W(bD,bM,[bO,bL,bN.constructor==au])};bF(0,bE[0],bE[0].anchor,bG,bH);if(bI){bF(1,bE[1],bE[1].anchor,bJ,bH)}};this.connectionDetached=function(bE){var bF=bE.connection||bE;var bK=bF.sourceId,bL=bF.targetId,bO=bF.endpoints,bJ=function(bP,bQ,bS,bR,
 bT){if(bS.constructor==a3){}else{A(bD[bR],function(bU){return bU[0].id==bT.id})}};bJ(1,bO[1],bO[1].anchor,bK,bF);bJ(0,bO[0],bO[0].anchor,bL,bF);var bG=bF.sourceId,bH=bF.targetId,bN=bF.endpoints[0].id,bI=bF.endpoints[1].id,bM=function(bR,bP){if(bR){var bQ=function(bS){return bS[4]==bP};A(bR.top,bQ);A(bR.left,bQ);A(bR.bottom,bQ);A(bR.right,bQ)}};bM(bC[bG],bN);bM(bC[bH],bI);bA.redraw(bG);bA.redraw(bH)};this.add=function(bF,bE){W(bz,bE,bF)};this.changeId=function(bF,bE){bD[bE]=bD[bF];bz[bE]=bz[bF];delete bD[bF];delete bz[bF]};this.getConnectionsFor=function(bE){return bD[bE]||[]};this.getEndpointsFor=function(bE){return bz[bE]||[]};this.deleteEndpoint=function(bE){A(bz[bE.elementId],function(bF){return bF.id==bE.id})};this.clearFor=function(bE){delete bz[bE];bz[bE]=[]};var bB=function(bY,bL,bT,bI,bO,bP,bR,bN,b0,bQ,bH,bX){var bV=-1,bG=-1,bJ=bI.endpoints[bR],bS=bJ.id,bM=[1,0][bR],bE=[[bL,bT],bI,bO,bP,bS],bF=bY[b0],bZ=bJ._continuousAnchorEdge?bY[bJ._continuousAnchorEdge]:null;if(bZ){var bW
 =g(bZ,function(b1){return b1[4]==bS});if(bW!=-1){bZ.splice(bW,1);for(var bU=0;bU<bZ.length;bU++){r(bH,bZ[bU][1],function(b1){return b1.id==bZ[bU][1].id});r(bX,bZ[bU][1].endpoints[bR],function(b1){return b1.id==bZ[bU][1].endpoints[bR].id})}}}for(var bU=0;bU<bF.length;bU++){if(bR==1&&bF[bU][3]===bP&&bG==-1){bG=bU}r(bH,bF[bU][1],function(b1){return b1.id==bF[bU][1].id});r(bX,bF[bU][1].endpoints[bR],function(b1){return b1.id==bF[bU][1].endpoints[bR].id})}if(bV!=-1){bF[bV]=bE}else{var bK=bN?bG!=-1?bG:0:bF.length;bF.splice(bK,0,bE)}bJ._continuousAnchorEdge=b0};this.redraw=function(bT,bV,bG,bJ){if(!a9){var b4=bz[bT]||[],b3=bD[bT]||[],bF=[],b2=[],bH=[];bG=bG||p();bJ=bJ||{left:0,top:0};if(bV){bV={left:bV.left+bJ.left,top:bV.top+bJ.top}}X({elId:bT,offset:bV,recalc:false,timestamp:bG});var bO=aj[bT],bK=ag[bT],bQ={};for(var b0=0;b0<b3.length;b0++){var bL=b3[b0][0],bN=bL.sourceId,bI=bL.targetId,bM=bL.endpoints[0].anchor.isContinuous,bS=bL.endpoints[1].anchor.isContinuous;if(bM||bS){var b1=bN+"_"
 +bI,bY=bI+"_"+bN,bX=bQ[b1],bR=bL.sourceId==bT?1:0;if(bM&&!bC[bN]){bC[bN]={top:[],right:[],bottom:[],left:[]}}if(bS&&!bC[bI]){bC[bI]={top:[],right:[],bottom:[],left:[]}}if(bT!=bI){X({elId:bI,timestamp:bG})}if(bT!=bN){X({elId:bN,timestamp:bG})}var bP=aQ(bI),bE=aQ(bN);if(bI==bN&&(bM||bS)){bB(bC[bN],-Math.PI/2,0,bL,false,bI,0,false,"top",bN,bF,b2)}else{if(!bX){bX=bp(bN,bI,bE.o,bP.o);bQ[b1]=bX}if(bM){bB(bC[bN],bX.theta,0,bL,false,bI,0,false,bX.a[0],bN,bF,b2)}if(bS){bB(bC[bI],bX.theta2,-1,bL,true,bN,1,true,bX.a[1],bI,bF,b2)}}if(bM){r(bH,bN,function(b5){return b5===bN})}if(bS){r(bH,bI,function(b5){return b5===bI})}r(bF,bL,function(b5){return b5.id==bL.id});if((bM&&bR==0)||(bS&&bR==1)){r(b2,bL.endpoints[bR],function(b5){return b5.id==bL.endpoints[bR].id})}}}for(var b0=0;b0<b4.length;b0++){if(b4[b0].connections.length==0&&b4[b0].anchor.isContinuous){if(!bC[bT]){bC[bT]={top:[],right:[],bottom:[],left:[]}}bB(bC[bT],-Math.PI/2,0,{endpoints:[b4[b0],b4[b0]],paint:function(){}},false,bT,0,false,"t
 op",bT,bF,b2);r(bH,bT,function(b5){return b5===bT})}}for(var b0=0;b0<bH.length;b0++){ao(bH[b0],bC[bH[b0]])}for(var b0=0;b0<b4.length;b0++){b4[b0].paint({timestamp:bG,offset:bO,dimensions:bK})}for(var b0=0;b0<b2.length;b0++){b2[b0].paint({timestamp:bG,offset:bO,dimensions:bK})}for(var b0=0;b0<b3.length;b0++){var bU=b3[b0][1];if(bU.anchor.constructor==au){bU.paint({elementWithPrecedence:bT});r(bF,b3[b0][0],function(b5){return b5.id==b3[b0][0].id});for(var bZ=0;bZ<bU.connections.length;bZ++){if(bU.connections[bZ]!==b3[b0][0]){r(bF,bU.connections[bZ],function(b5){return b5.id==bU.connections[bZ].id})}}}else{if(bU.anchor.constructor==ab){r(bF,b3[b0][0],function(b5){return b5.id==b3[b0][0].id})}}}var bW=bk[bT];if(bW){bW.paint({timestamp:bG,recalc:false,elId:bT})}for(var b0=0;b0<bF.length;b0++){bF[b0].paint({elId:bT,timestamp:bG,recalc:false})}}};this.rehomeEndpoint=function(bE,bI){var bF=bz[bE]||[],bG=br.getId(bI);if(bG!==bE){for(var bH=0;bH<bF.length;bH++){bA.add(bF[bH],bG)}bF.splice(0,b
 F.length)}}};br.anchorManager=new aG();br.continuousAnchorFactory={get:function(bA){var bz=bn[bA.elementId];if(!bz){bz={type:"Continuous",compute:function(bB){return ak[bB.element.id]||[0,0]},getCurrentLocation:function(bB){return ak[bB.id]||[0,0]},getOrientation:function(bB){return aM[bB.id]||[0,0]},isDynamic:true,isContinuous:true};bn[bA.elementId]=bz}return bz}};if(!jsPlumbAdapter.headless){br.dragManager=jsPlumbAdapter.getDragManager(br)}br.recalculateOffsets=br.dragManager.updateOffsets;var az=function(bU){var bK=this,bB=true,bP,bQ;bK.idPrefix="_jsplumb_c_";bK.defaultLabelLocation=0.5;bK.defaultOverlayKeys=["Overlays","ConnectionOverlays"];this.parent=bU.parent;x.apply(this,arguments);this.isVisible=function(){return bB};this.setVisible=function(bW){bB=bW;bK[bW?"showOverlays":"hideOverlays"]();if(bK.connector&&bK.connector.canvas){bK.connector.canvas.style.display=bW?"block":"none"}bK.repaint()};this.getTypeDescriptor=function(){return"connection"};this.getDefaultType=function(
 ){return{parameters:{},scope:null,detachable:bK._jsPlumb.Defaults.ConnectionsDetachable,rettach:bK._jsPlumb.Defaults.ReattachConnections,paintStyle:bK._jsPlumb.Defaults.PaintStyle||n.Defaults.PaintStyle,connector:bK._jsPlumb.Defaults.Connector||n.Defaults.Connector,hoverPaintStyle:bK._jsPlumb.Defaults.HoverPaintStyle||n.Defaults.HoverPaintStyle,overlays:bK._jsPlumb.Defaults.ConnectorOverlays||n.Defaults.ConnectorOverlays}};var bS=this.applyType;this.applyType=function(bW){bS(bW);if(bW.detachable!=null){bK.setDetachable(bW.detachable)}if(bW.reattach!=null){bK.setReattach(bW.reattach)}if(bW.scope){bK.scope=bW.scope}bK.setConnector(bW.connector)};bQ=bK.setHover;bK.setHover=function(bX){var bW=br.ConnectorZIndex||n.Defaults.ConnectorZIndex;if(bW){bK.connector.setZIndex(bW+(bX?1:0))}bK.connector.setHover.apply(bK.connector,arguments);bQ.apply(bK,arguments)};bP=function(bW){if(v==null){bK.setHover(bW,false)}};this.setConnector=function(bW,bY){if(bK.connector!=null){aZ(bK.connector.getDisp
 layElements(),bK.parent)}var bZ={_jsPlumb:bK._jsPlumb,parent:bU.parent,cssClass:bU.cssClass,container:bU.container,tooltip:bK.tooltip};if(z(bW)){this.connector=new n.Connectors[Y][bW](bZ)}else{if(l(bW)){if(bW.length==1){this.connector=new n.Connectors[Y][bW[0]](bZ)}else{this.connector=new n.Connectors[Y][bW[0]](n.extend(bW[1],bZ))}}}bK.canvas=bK.connector.canvas;D(bK.connector,bK,bP);var bX=br.ConnectorZIndex||n.Defaults.ConnectorZIndex;if(bX){bK.connector.setZIndex(bX)}if(!bY){bK.repaint()}};this.source=C(bU.source);this.target=C(bU.target);if(bU.sourceEndpoint){this.source=bU.sourceEndpoint.endpointWillMoveTo||bU.sourceEndpoint.getElement()}if(bU.targetEndpoint){this.target=bU.targetEndpoint.getElement()}bK.previousConnection=bU.previousConnection;this.sourceId=c(this.source,"id");this.targetId=c(this.target,"id");this.scope=bU.scope;this.endpoints=[];this.endpointStyles=[];var bT=function(bX,bW){return(bX)?br.makeAnchor(bX,bW,br):null},bO=function(bW,b2,bX,bZ,b0,bY,b1){var b3;if(
 bW){bK.endpoints[b2]=bW;bW.addConnection(bK)}else{if(!bX.endpoints){bX.endpoints=[null,null]}var b8=bX.endpoints[b2]||bX.endpoint||br.Defaults.Endpoints[b2]||n.Defaults.Endpoints[b2]||br.Defaults.Endpoint||n.Defaults.Endpoint;if(!bX.endpointStyles){bX.endpointStyles=[null,null]}if(!bX.endpointHoverStyles){bX.endpointHoverStyles=[null,null]}var b6=bX.endpointStyles[b2]||bX.endpointStyle||br.Defaults.EndpointStyles[b2]||n.Defaults.EndpointStyles[b2]||br.Defaults.EndpointStyle||n.Defaults.EndpointStyle;if(b6.fillStyle==null&&bY!=null){b6.fillStyle=bY.strokeStyle}if(b6.outlineColor==null&&bY!=null){b6.outlineColor=bY.outlineColor}if(b6.outlineWidth==null&&bY!=null){b6.outlineWidth=bY.outlineWidth}var b5=bX.endpointHoverStyles[b2]||bX.endpointHoverStyle||br.Defaults.EndpointHoverStyles[b2]||n.Defaults.EndpointHoverStyles[b2]||br.Defaults.EndpointHoverStyle||n.Defaults.EndpointHoverStyle;if(b1!=null){if(b5==null){b5={}}if(b5.fillStyle==null){b5.fillStyle=b1.strokeStyle}}var b4=bX.anchors?
 bX.anchors[b2]:bX.anchor?bX.anchor:bT(br.Defaults.Anchors[b2],b0)||bT(n.Defaults.Anchors[b2],b0)||bT(br.Defaults.Anchor,b0)||bT(n.Defaults.Anchor,b0),b7=bX.uuids?bX.uuids[b2]:null;b3=aH({paintStyle:b6,hoverPaintStyle:b5,endpoint:b8,connections:[bK],uuid:b7,anchor:b4,source:bZ,scope:bX.scope,container:bX.container,reattach:bX.reattach||br.Defaults.ReattachConnections,detachable:bX.detachable||br.Defaults.ConnectionsDetachable});bK.endpoints[b2]=b3;if(bX.drawEndpoints===false){b3.setVisible(false,true,true)}}return b3};var bM=bO(bU.sourceEndpoint,0,bU,bK.source,bK.sourceId,bU.paintStyle,bU.hoverPaintStyle);if(bM){W(aV,this.sourceId,bM)}var bL=bO(bU.targetEndpoint,1,bU,bK.target,bK.targetId,bU.paintStyle,bU.hoverPaintStyle);if(bL){W(aV,this.targetId,bL)}if(!this.scope){this.scope=this.endpoints[0].scope}bK.endpointsToDeleteOnDetach=[null,null];if(bU.deleteEndpointsOnDetach){if(bU.sourceIsNew){bK.endpointsToDeleteOnDetach[0]=bK.endpoints[0]}if(bU.targetIsNew){bK.endpointsToDeleteOnDetac
 h[1]=bK.endpoints[1]}}bK.setConnector(this.endpoints[0].connector||this.endpoints[1].connector||bU.connector||br.Defaults.Connector||n.Defaults.Connector,true);this.setPaintStyle(this.endpoints[0].connectorStyle||this.endpoints[1].connectorStyle||bU.paintStyle||br.Defaults.PaintStyle||n.Defaults.PaintStyle,true);this.setHoverPaintStyle(this.endpoints[0].connectorHoverStyle||this.endpoints[1].connectorHoverStyle||bU.hoverPaintStyle||br.Defaults.HoverPaintStyle||n.Defaults.HoverPaintStyle,true);this.paintStyleInUse=this.getPaintStyle();X({elId:this.sourceId,timestamp:bd});X({elId:this.targetId,timestamp:bd});var bD=aj[this.sourceId],bC=ag[this.sourceId],bz=aj[this.targetId],bH=ag[this.targetId],bI=bd||p(),bN=this.endpoints[0].anchor.compute({xy:[bD.left,bD.top],wh:bC,element:this.endpoints[0],elementId:this.endpoints[0].elementId,txy:[bz.left,bz.top],twh:bH,tElement:this.endpoints[1],timestamp:bI});this.endpoints[0].paint({anchorLoc:bN,timestamp:bI});bN=this.endpoints[1].anchor.comput
 e({xy:[bz.left,bz.top],wh:bH,element:this.endpoints[1],elementId:this.endpoints[1].elementId,txy:[bD.left,bD.top],twh:bC,tElement:this.endpoints[0],timestamp:bI});this.endpoints[1].paint({anchorLoc:bN,timestamp:bI});var bA=br.Defaults.ConnectionsDetachable;if(bU.detachable===false){bA=false}if(bK.endpoints[0].connectionsDetachable===false){bA=false}if(bK.endpoints[1].connectionsDetachable===false){bA=false}this.isDetachable=function(){return bA===true};this.setDetachable=function(bW){bA=bW===true};var bG=bU.reattach||bK.endpoints[0].reattachConnections||bK.endpoints[1].reattachConnections||br.Defaults.ReattachConnections;this.isReattach=function(){return bG===true};this.setReattach=function(bW){bG=bW===true};var bF=bU.cost||bK.endpoints[0].getConnectionCost();bK.getCost=function(){return bF};bK.setCost=function(bW){bF=bW};var bE=!(bU.bidirectional===false);if(bU.bidirectional==null){bE=bK.endpoints[0].areConnectionsBidirectional()}bK.isBidirectional=function(){return bE};var bV=n.ex
 tend({},this.endpoints[0].getParameters());n.extend(bV,this.endpoints[1].getParameters());n.extend(bV,bK.getParameters());bK.setParameters(bV);this.getAttachedElements=function(){return bK.endpoints};this.moveParent=function(bZ){var bY=n.CurrentLibrary,bX=bY.getParent(bK.connector.canvas);if(bK.connector.bgCanvas){bY.removeElement(bK.connector.bgCanvas,bX);bY.appendElement(bK.connector.bgCanvas,bZ)}bY.removeElement(bK.connector.canvas,bX);bY.appendElement(bK.connector.canvas,bZ);for(var bW=0;bW<bK.overlays.length;bW++){if(bK.overlays[bW].isAppendedAtTopLevel){bY.removeElement(bK.overlays[bW].canvas,bX);bY.appendElement(bK.overlays[bW].canvas,bZ);if(bK.overlays[bW].reattachListeners){bK.overlays[bW].reattachListeners(bK.connector)}}}if(bK.connector.reattachListeners){bK.connector.reattachListeners()}};var bJ=null;this.paint=function(cd){if(bB){cd=cd||{};var b4=cd.elId,b5=cd.ui,b2=cd.recalc,bX=cd.timestamp,b6=false,cc=b6?this.sourceId:this.targetId,b1=b6?this.targetId:this.sourceId,bY
 =b6?0:1,ce=b6?1:0;if(bX==null||bX!=bJ){var cf=X({elId:b4,offset:b5,recalc:b2,timestamp:bX}),b3=X({elId:cc,timestamp:bX});var b8=this.endpoints[ce],bW=this.endpoints[bY],b0=b8.anchor.getCurrentLocation(b8),cb=bW.anchor.getCurrentLocation(bW);var bZ=0;for(var ca=0;ca<bK.overlays.length;ca++){var b7=bK.overlays[ca];if(b7.isVisible()){bZ=Math.max(bZ,b7.computeMaxSize())}}var b9=this.connector.compute(b0,cb,this.endpoints[ce],this.endpoints[bY],this.endpoints[ce].anchor,this.endpoints[bY].anchor,bK.paintStyleInUse.lineWidth,bZ,cf,b3);bK.connector.paint(b9,bK.paintStyleInUse);for(var ca=0;ca<bK.overlays.length;ca++){var b7=bK.overlays[ca];if(b7.isVisible){bK.overlayPlacements[ca]=b7.draw(bK.connector,bK.paintStyleInUse,b9)}}}bJ=bX}};this.repaint=function(bX){bX=bX||{};var bW=!(bX.recalc===false);this.paint({elId:this.sourceId,recalc:bW,timestamp:bX.timestamp})};var bR=bU.type||bK.endpoints[0].connectionType||bK.endpoints[1].connectionType;if(bR){bK.addType(bR)}};var bh=function(bA){var bz
 =false;return{drag:function(){if(bz){bz=false;return true}var bB=n.CurrentLibrary.getUIPosition(arguments,br.getZoom()),bC=bA.element;if(bC){n.CurrentLibrary.setOffset(bC,bB);be(C(bC),bB)}},stopDrag:function(){bz=true}}};var av=function(bD,bC,bE,bB,bz){var bA=new a3({reference:bC,referenceCanvas:bB});return aH({paintStyle:bD,endpoint:bE,anchor:bA,source:bz,scope:"__floating"})};var S=function(bB,bz){var bD=document.createElement("div");bD.style.position="absolute";var bA=C(bD);aX(bD,bz);var bC=H(bA);X({elId:bC});bB.id=bC;bB.element=bA};var ah=function(b5){var bS=this;bS.idPrefix="_jsplumb_e_";bS.defaultLabelLocation=[0.5,0.5];bS.defaultOverlayKeys=["Overlays","EndpointOverlays"];this.parent=b5.parent;x.apply(this,arguments);b5=b5||{};this.getTypeDescriptor=function(){return"endpoint"};this.getDefaultType=function(){return{parameters:{},scope:null,maxConnections:bS._jsPlumb.Defaults.MaxConnections,paintStyle:bS._jsPlumb.Defaults.EndpointStyle||n.Defaults.EndpointStyle,endpoint:bS._js
 Plumb.Defaults.Endpoint||n.Defaults.Endpoint,hoverPaintStyle:bS._jsPlumb.Defaults.EndpointHoverStyle||n.Defaults.EndpointHoverStyle,overlays:bS._jsPlumb.Defaults.EndpointOverlays||n.Defaults.EndpointOverlays,connectorStyle:b5.connectorStyle,connectorHoverStyle:b5.connectorHoverStyle,connectorClass:b5.connectorClass,connectorHoverClass:b5.connectorHoverClass,connectorOverlays:b5.connectorOverlays,connector:b5.connector,connectorTooltip:b5.connectorTooltip}};var b3=this.applyType;this.applyType=function(b7){b3(b7);if(b7.maxConnections!=null){bY=b7.maxConnections}if(b7.scope){bS.scope=b7.scope}bS.connectorStyle=b7.connectorStyle;bS.connectorHoverStyle=b7.connectorHoverStyle;bS.connectorOverlays=b7.connectorOverlays;bS.connector=b7.connector;bS.connectorTooltip=b7.connectorTooltip;bS.connectionType=b7.connectionType;bS.connectorClass=b7.connectorClass;bS.connectorHoverClass=b7.connectorHoverClass};var bE=true,bC=!(b5.enabled===false);this.isVisible=function(){return bE};this.setVisible=
 function(b8,cb,b7){bE=b8;if(bS.canvas){bS.canvas.style.display=b8?"block":"none"}bS[b8?"showOverlays":"hideOverlays"]();if(!cb){for(var ca=0;ca<bS.connections.length;ca++){bS.connections[ca].setVisible(b8);if(!b7){var b9=bS===bS.connections[ca].endpoints[0]?1:0;if(bS.connections[ca].endpoints[b9].connections.length==1){bS.connections[ca].endpoints[b9].setVisible(b8,true,true)}}}}};this.isEnabled=function(){return bC};this.setEnabled=function(b7){bC=b7};var bR=b5.source,bL=b5.uuid,b2=null,bG=null;if(bL){aW[bL]=bS}var bJ=c(bR,"id");this.elementId=bJ;this.element=bR;var bB=b5.connectionCost;this.getConnectionCost=function(){return bB};this.setConnectionCost=function(b7){bB=b7};var b1=b5.connectionsBidirectional===false?false:true;this.areConnectionsBidirectional=function(){return b1};this.setConnectionsBidirectional=function(b7){b1=b7};bS.anchor=b5.anchor?br.makeAnchor(b5.anchor,bJ,br):b5.anchors?br.makeAnchor(b5.anchors,bJ,br):br.makeAnchor(br.Defaults.Anchor||"TopCenter",bJ,br);if(!b
 5._transient){br.anchorManager.add(bS,bJ)}var bP=null,bU=null;this.setEndpoint=function(b7){var b8={_jsPlumb:bS._jsPlumb,parent:b5.parent,container:b5.container,tooltip:b5.tooltip,connectorTooltip:b5.connectorTooltip,endpoint:bS};if(z(b7)){bP=new n.Endpoints[Y][b7](b8)}else{if(l(b7)){b8=n.extend(b7[1],b8);bP=new n.Endpoints[Y][b7[0]](b8)}else{bP=b7.clone()}}var b9=n.extend({},b8);bP.clone=function(){var ca=new Object();bP.constructor.apply(ca,[b9]);return ca};bS.endpoint=bP;bS.type=bS.endpoint.type};this.setEndpoint(b5.endpoint||br.Defaults.Endpoint||n.Defaults.Endpoint||"Dot");bU=bP;var bQ=bS.setHover;bS.setHover=function(){bS.endpoint.setHover.apply(bS.endpoint,arguments);bQ.apply(bS,arguments)};var b6=function(b7){if(bS.connections.length>0){bS.connections[0].setHover(b7,false)}else{bS.setHover(b7)}};D(bS.endpoint,bS,b6);this.setPaintStyle(b5.paintStyle||b5.style||br.Defaults.EndpointStyle||n.Defaults.EndpointStyle,true);this.setHoverPaintStyle(b5.hoverPaintStyle||br.Defaults.End
 pointHoverStyle||n.Defaults.EndpointHoverStyle,true);this.paintStyleInUse=this.getPaintStyle();var bN=this.getPaintStyle();this.connectorStyle=b5.connectorStyle;this.connectorHoverStyle=b5.connectorHoverStyle;this.connectorOverlays=b5.connectorOverlays;this.connector=b5.connector;this.connectorTooltip=b5.connectorTooltip;this.connectorClass=b5.connectorClass;this.connectorHoverClass=b5.connectorHoverClass;this.isSource=b5.isSource||false;this.isTarget=b5.isTarget||false;var bY=b5.maxConnections||br.Defaults.MaxConnections;this.getAttachedElements=function(){return bS.connections};this.canvas=this.endpoint.canvas;this.connections=b5.connections||[];this.scope=b5.scope||R;this.connectionType=b5.connectionType;this.timestamp=null;bS.reattachConnections=b5.reattach||br.Defaults.ReattachConnections;bS.connectionsDetachable=br.Defaults.ConnectionsDetachable;if(b5.connectionsDetachable===false||b5.detachable===false){bS.connectionsDetachable=false}var bM=b5.dragAllowedWhenFull||true;if(b5.
 onMaxConnections){bS.bind("maxConnections",b5.onMaxConnections)}this.computeAnchor=function(b7){return bS.anchor.compute(b7)};this.addConnection=function(b7){bS.connections.push(b7)};this.detach=function(b8,cd,b9,cg,b7){var cf=g(bS.connections,function(ci){return ci.id==b8.id}),ce=false;cg=(cg!==false);if(cf>=0){if(b9||b8._forceDetach||b8.isDetachable()||b8.isDetachAllowed(b8)){var ch=b8.endpoints[0]==bS?b8.endpoints[1]:b8.endpoints[0];if(b9||b8._forceDetach||(bS.isDetachAllowed(b8))){bS.connections.splice(cf,1);if(!cd){ch.detach(b8,true,b9);if(b8.endpointsToDeleteOnDetach){for(var cc=0;cc<b8.endpointsToDeleteOnDetach.length;cc++){var ca=b8.endpointsToDeleteOnDetach[cc];if(ca&&ca.connections.length==0){br.deleteEndpoint(ca)}}}}aZ(b8.connector.getDisplayElements(),b8.parent);A(a0[b8.scope],function(ci){return ci.id==b8.id});ce=true;var cb=(!cd&&cg);ba(b8,cb,b7)}}}return ce};this.detachAll=function(b8,b7){while(bS.connections.length>0){bS.detach(bS.connections[0],false,true,b8,b7)}};t
 his.detachFrom=function(ca,b9,b7){var cb=[];for(var b8=0;b8<bS.connections.length;b8++){if(bS.connections[b8].endpoints[1]==ca||bS.connections[b8].endpoints[0]==ca){cb.push(bS.connections[b8])}}for(var b8=0;b8<cb.length;b8++){if(bS.detach(cb[b8],false,true,b9,b7)){cb[b8].setHover(false,false)}}};this.detachFromConnection=function(b8){var b7=g(bS.connections,function(b9){return b9.id==b8.id});if(b7>=0){bS.connections.splice(b7,1)}};this.getElement=function(){return bR};this.setElement=function(ca,b7){var cc=H(ca);A(aV[bS.elementId],function(cd){return cd.id==bS.id});bR=C(ca);bJ=H(bR);bS.elementId=bJ;var cb=ay({source:cc,container:b7}),b9=bD.getParent(bS.canvas);bD.removeElement(bS.canvas,b9);bD.appendElement(bS.canvas,cb);for(var b8=0;b8<bS.connections.length;b8++){bS.connections[b8].moveParent(cb);bS.connections[b8].sourceId=bJ;bS.connections[b8].source=bR}W(aV,cc,bS)};this.getUuid=function(){return bL};this.makeInPlaceCopy=function(){var b9=bS.anchor.getCurrentLocation(bS),b8=bS.an
 chor.getOrientation(bS),b7={compute:function(){return[b9[0],b9[1]]},getCurrentLocation:function(){return[b9[0],b9[1]]},getOrientation:function(){return b8}};return aH({anchor:b7,source:bR,paintStyle:this.getPaintStyle(),endpoint:bP,_transient:true,scope:bS.scope})};this.isConnectedTo=function(b9){var b8=false;if(b9){for(var b7=0;b7<bS.connections.length;b7++){if(bS.connections[b7].endpoints[1]==b9){b8=true;break}}}return b8};this.isFloating=function(){return b2!=null};this.connectorSelector=function(){var b7=bS.connections[0];if(bS.isTarget&&b7){return b7}else{return(bS.connections.length<bY)||bY==-1?null:b7}};this.isFull=function(){return !(bS.isFloating()||bY<1||bS.connections.length<bY)};this.setDragAllowedWhenFull=function(b7){bM=b7};this.setStyle=bS.setPaintStyle;this.equals=function(b7){return this.anchor.equals(b7.anchor)};var bO=function(b8){var b7=0;if(b8!=null){for(var b9=0;b9<bS.connections.length;b9++){if(bS.connections[b9].sourceId==b8||bS.connections[b9].targetId==b8){
 b7=b9;break}}}return bS.connections[b7]};this.paint=function(ca){ca=ca||{};var cg=ca.timestamp,cf=!(ca.recalc===false);if(!cg||bS.timestamp!==cg){X({elId:bJ,timestamp:cg,recalc:cf});var cm=ca.offset||aj[bJ];if(cm){var cd=ca.anchorPoint,cb=ca.connectorPaintStyle;if(cd==null){var b7=ca.dimensions||ag[bJ];if(cm==null||b7==null){X({elId:bJ,timestamp:cg});cm=aj[bJ];b7=ag[bJ]}var b9={xy:[cm.left,cm.top],wh:b7,element:bS,timestamp:cg};if(cf&&bS.anchor.isDynamic&&bS.connections.length>0){var cj=bO(ca.elementWithPrecedence),cl=cj.endpoints[0]==bS?1:0,cc=cl==0?cj.sourceId:cj.targetId,ci=aj[cc],ck=ag[cc];b9.txy=[ci.left,ci.top];b9.twh=ck;b9.tElement=cj.endpoints[cl]}cd=bS.anchor.compute(b9)}var ch=bP.compute(cd,bS.anchor.getOrientation(bS),bS.paintStyleInUse,cb||bS.paintStyleInUse);bP.paint(ch,bS.paintStyleInUse,bS.anchor);bS.timestamp=cg;for(var ce=0;ce<bS.overlays.length;ce++){var b8=bS.overlays[ce];if(b8.isVisible){bS.overlayPlacements[ce]=b8.draw(bS.endpoint,bS.paintStyleInUse,ch)}}}}};thi
 s.repaint=this.paint;this.removeConnection=this.detach;if(n.CurrentLibrary.isDragSupported(bR)){var bX={id:null,element:null},bW=null,bA=false,bF=null,bz=bh(bX);var bH=function(){bW=bS.connectorSelector();var b7=true;if(!bS.isEnabled()){b7=false}if(bW==null&&!b5.isSource){b7=false}if(b5.isSource&&bS.isFull()&&!bM){b7=false}if(bW!=null&&!bW.isDetachable()){b7=false}if(b7===false){if(n.CurrentLibrary.stopDrag){n.CurrentLibrary.stopDrag()}bz.stopDrag();return false}if(bW&&!bS.isFull()&&b5.isSource){bW=null}X({elId:bJ});bG=bS.makeInPlaceCopy();bG.referenceEndpoint=bS;bG.paint();S(bX,bS.parent);var cd=C(bG.canvas),cb=s(cd,br),b8=bs([cb.left,cb.top],bG.canvas);n.CurrentLibrary.setOffset(bX.element,{left:b8[0],top:b8[1]});if(bS.parentAnchor){bS.anchor=br.makeAnchor(bS.parentAnchor,bS.elementId,br)}d(C(bS.canvas),"dragId",bX.id);d(C(bS.canvas),"elId",bJ);if(b5.proxy){bS.setPaintStyle(b5.proxy.paintStyle)}b2=av(bS.getPaintStyle(),bS.anchor,bP,bS.canvas,bX.element);if(bW==null){bS.anchor.lock
 ed=true;bS.setHover(false,false);bW=af({sourceEndpoint:bS,targetEndpoint:b2,source:bS.endpointWillMoveTo||C(bR),target:bX.element,anchors:[bS.anchor,b2.anchor],paintStyle:b5.connectorStyle,hoverPaintStyle:b5.connectorHoverStyle,connector:b5.connector,overlays:b5.connectorOverlays,type:bS.connectionType,cssClass:bS.connectorClass,hoverClass:bS.connectorHoverClass})}else{bA=true;bW.setHover(false);bI(C(bG.canvas),false,true);var ca=bW.endpoints[0].id==bS.id?0:1;bW.floatingAnchorIndex=ca;bS.detachFromConnection(bW);var ce=C(bS.canvas),cc=n.CurrentLibrary.getDragScope(ce);d(ce,"originalScope",cc);var b9=n.CurrentLibrary.getDropScope(ce);n.CurrentLibrary.setDragScope(ce,b9);if(ca==0){bF=[bW.source,bW.sourceId,b0,cc];bW.source=bX.element;bW.sourceId=bX.id}else{bF=[bW.target,bW.targetId,b0,cc];bW.target=bX.element;bW.targetId=bX.id}bW.endpoints[ca==0?1:0].anchor.locked=true;bW.suspendedEndpoint=bW.endpoints[ca];bW.suspendedEndpoint.setHover(false);b2.referenceEndpoint=bW.suspendedEndpoint;
 bW.endpoints[ca]=b2;a8(bW)}bk[bX.id]=bW;b2.addConnection(bW);W(aV,bX.id,b2);br.currentlyDragging=true};var bD=n.CurrentLibrary,bZ=b5.dragOptions||{},bT=n.extend({},bD.defaultDragOptions),bV=bD.dragEvents.start,b4=bD.dragEvents.stop,bK=bD.dragEvents.drag;bZ=n.extend(bT,bZ);bZ.scope=bZ.scope||bS.scope;bZ[bV]=ap(bZ[bV],bH);bZ[bK]=ap(bZ[bK],bz.drag);bZ[b4]=ap(bZ[b4],function(){var b8=bD.getDropEvent(arguments);br.currentlyDragging=false;A(aV[bX.id],function(b9){return b9.id==b2.id});aZ([bX.element[0],b2.canvas],bR);aw(bG.canvas,bR);br.anchorManager.clearFor(bX.id);var b7=bW.floatingAnchorIndex==null?1:bW.floatingAnchorIndex;bW.endpoints[b7==0?1:0].anchor.locked=false;bS.setPaintStyle(bN);if(bW.endpoints[b7]==b2){if(bA&&bW.suspendedEndpoint){if(b7==0){bW.source=bF[0];bW.sourceId=bF[1]}else{bW.target=bF[0];bW.targetId=bF[1]}n.CurrentLibrary.setDragScope(bF[2],bF[3]);bW.endpoints[b7]=bW.suspendedEndpoint;if(bW.isReattach()||bW._forceReattach||bW._forceDetach||!bW.endpoints[b7==0?1:0].detac
 h(bW,false,false,true,b8)){bW.setHover(false);bW.floatingAnchorIndex=null;bW.suspendedEndpoint.addConnection(bW);br.repaint(bF[1])}bW._forceDetach=null;bW._forceReattach=null}else{aZ(bW.connector.getDisplayElements(),bS.parent);bS.detachFromConnection(bW)}}bS.anchor.locked=false;bS.paint({recalc:false});aT(bW);bW=null;bG=null;delete aV[b2.elementId];b2.anchor=null;b2=null;br.currentlyDragging=false});var b0=C(bS.canvas);n.CurrentLibrary.initDraggable(b0,bZ,true)}var bI=function(b9,ce,cc,cf){if((b5.isTarget||ce)&&n.CurrentLibrary.isDropSupported(bR)){var ca=b5.dropOptions||br.Defaults.DropOptions||n.Defaults.DropOptions;ca=n.extend({},ca);ca.scope=ca.scope||bS.scope;var b8=n.CurrentLibrary.dragEvents.drop,cd=n.CurrentLibrary.dragEvents.over,b7=n.CurrentLibrary.dragEvents.out,cb=function(){var cg=n.CurrentLibrary.getDropEvent(arguments),ct=C(n.CurrentLibrary.getDragObject(arguments)),ci=c(ct,"dragId"),ck=c(ct,"elId"),cs=c(ct,"originalScope"),cn=bk[ci];var cl=cn.suspendedEndpoint&&(cn.
 suspendedEndpoint.id==bS.id||bS.referenceEndpoint&&cn.suspendedEndpoint.id==bS.referenceEndpoint.id);if(cl){cn._forceReattach=true;return}if(cn!=null){var cp=cn.floatingAnchorIndex==null?1:cn.floatingAnchorIndex,cq=cp==0?1:0;if(cs){n.CurrentLibrary.setDragScope(ct,cs)}var cr=cf!=null?cf.isEnabled():true;if(bS.isFull()){bS.fire("maxConnections",{endpoint:bS,connection:cn,maxConnections:bY},cg)}if(!bS.isFull()&&!(cp==0&&!bS.isSource)&&!(cp==1&&!bS.isTarget)&&cr){var cm=true;if(cn.suspendedEndpoint&&cn.suspendedEndpoint.id!=bS.id){if(cp==0){cn.source=cn.suspendedEndpoint.element;cn.sourceId=cn.suspendedEndpoint.elementId}else{cn.target=cn.suspendedEndpoint.element;cn.targetId=cn.suspendedEndpoint.elementId}if(!cn.isDetachAllowed(cn)||!cn.endpoints[cp].isDetachAllowed(cn)||!cn.suspendedEndpoint.isDetachAllowed(cn)||!br.checkCondition("beforeDetach",cn)){cm=false}}if(cp==0){cn.source=bS.element;cn.sourceId=bS.elementId}else{cn.target=bS.element;cn.targetId=bS.elementId}var co=function(){
 cn.floatingAnchorIndex=null};var ch=function(){cn.endpoints[cp].detachFromConnection(cn);if(cn.suspendedEndpoint){cn.suspendedEndpoint.detachFromConnection(cn)}cn.endpoints[cp]=bS;bS.addConnection(cn);var cx=bS.getParameters();for(var cv in cx){cn.setParameter(cv,cx[cv])}if(!cn.suspendedEndpoint){if(cx.draggable){n.CurrentLibrary.initDraggable(bS.element,bZ,true)}}else{var cw=cn.suspendedEndpoint.getElement(),cu=cn.suspendedEndpoint.elementId;ba({source:cp==0?cw:cn.source,target:cp==1?cw:cn.target,sourceId:cp==0?cu:cn.sourceId,targetId:cp==1?cu:cn.targetId,sourceEndpoint:cp==0?cn.suspendedEndpoint:cn.endpoints[0],targetEndpoint:cp==1?cn.suspendedEndpoint:cn.endpoints[1],connection:cn},true,cg)}bx(cn,null,cg);co()};var cj=function(){if(cn.suspendedEndpoint){cn.endpoints[cp]=cn.suspendedEndpoint;cn.setHover(false);cn._forceDetach=true;if(cp==0){cn.source=cn.suspendedEndpoint.element;cn.sourceId=cn.suspendedEndpoint.elementId}else{cn.target=cn.suspendedEndpoint.element;cn.targetId=cn.s
 uspendedEndpoint.elementId}cn.suspendedEndpoint.addConnection(cn);cn.endpoints[0].repaint();cn.repaint();br.repaint(cn.source.elementId);cn._forceDetach=false}co()};cm=cm&&bS.isDropAllowed(cn.sourceId,cn.targetId,cn.scope,cn,bS);if(cm){ch()}else{cj()}}br.currentlyDragging=false;delete bk[ci];cn.suspendedEndpoint=null}};ca[b8]=ap(ca[b8],cb);ca[cd]=ap(ca[cd],function(){var ch=n.CurrentLibrary.getDragObject(arguments),ck=c(C(ch),"dragId"),cj=bk[ck];if(cj!=null){var cg=cj.floatingAnchorIndex==null?1:cj.floatingAnchorIndex;var ci=(bS.isTarget&&cj.floatingAnchorIndex!=0)||(cj.suspendedEndpoint&&bS.referenceEndpoint&&bS.referenceEndpoint.id==cj.suspendedEndpoint.id);if(ci){cj.endpoints[cg].anchor.over(bS.anchor)}}});ca[b7]=ap(ca[b7],function(){var ch=n.CurrentLibrary.getDragObject(arguments),ck=c(C(ch),"dragId"),cj=bk[ck];if(cj!=null){var cg=cj.floatingAnchorIndex==null?1:cj.floatingAnchorIndex;var ci=(bS.isTarget&&cj.floatingAnchorIndex!=0)||(cj.suspendedEndpoint&&bS.referenceEndpoint&&bS
 .referenceEndpoint.id==cj.suspendedEndpoint.id);if(ci){cj.endpoints[cg].anchor.out()}}});n.CurrentLibrary.initDroppable(b9,ca,true,cc)}};bI(C(bS.canvas),true,!(b5._transient||bS.anchor.isFloating),bS);if(b5.type){bS.addType(b5.type)}return bS}};var n=new w();if(typeof window!="undefined"){window.jsPlumb=n}n.getInstance=function(I){var H=new w(I);H.init();return H};if(typeof define==="function"&&define.amd&&define.amd.jsPlumb){define("jsplumb",[],function(){return n})}var q=function(H,M,J,I,L,K){return function(O){O=O||{};var N=O.jsPlumbInstance.makeAnchor([H,M,J,I,0,0],O.elementId,O.jsPlumbInstance);N.type=L;if(K){K(N,O)}return N}};n.Anchors.TopCenter=q(0.5,0,0,-1,"TopCenter");n.Anchors.BottomCenter=q(0.5,1,0,1,"BottomCenter");n.Anchors.LeftMiddle=q(0,0.5,-1,0,"LeftMiddle");n.Anchors.RightMiddle=q(1,0.5,1,0,"RightMiddle");n.Anchors.Center=q(0.5,0.5,0,0,"Center");n.Anchors.TopRight=q(1,0,0,-1,"TopRight");n.Anchors.BottomRight=q(1,1,0,1,"BottomRight");n.Anchors.TopLeft=q(0,0,0,-1,"Top
 Left");n.Anchors.BottomLeft=q(0,1,0,1,"BottomLeft");n.Defaults.DynamicAnchors=function(H){return H.jsPlumbInstance.makeAnchors(["TopCenter","RightMiddle","BottomCenter","LeftMiddle"],H.elementId,H.jsPlumbInstance)};n.Anchors.AutoDefault=function(I){var H=I.jsPlumbInstance.makeDynamicAnchor(n.Defaults.DynamicAnchors(I));H.type="AutoDefault";return H};n.Anchors.Assign=q(0,0,0,0,"Assign",function(I,J){var H=J.position||"Fixed";I.positionFinder=H.constructor==String?J.jsPlumbInstance.AnchorPositionFinders[H]:H;I.constructorParams=J});n.Anchors.Continuous=function(H){return H.jsPlumbInstance.continuousAnchorFactory.get(H)};n.AnchorPositionFinders={Fixed:function(K,H,J,I){return[(K.left-H.left)/J[0],(K.top-H.top)/J[1]]},Grid:function(H,Q,L,I){var P=H.left-Q.left,O=H.top-Q.top,N=L[0]/(I.grid[0]),M=L[1]/(I.grid[1]),K=Math.floor(P/N),J=Math.floor(O/M);return[((K*N)+(N/2))/L[0],((J*M)+(M/2))/L[1]]}};n.Anchors.Perimeter=function(H){H=H||{};var I=H.anchorCount||60,L=H.shape;if(!L){throw new Err
 or("no shape supplied to Perimeter Anchor type")}var J=function(){var W=0.5,V=Math.PI*2/I,X=0,T=[];for(var U=0;U<I;U++){var S=W+(W*Math.sin(X)),Y=W+(W*Math.cos(X));T.push([S,Y,0,0]);X+=V}return T},M=function(U){var W=I/U.length,S=[],T=function(aa,ad,Z,ac,ae){W=I*ae;var Y=(Z-aa)/W,X=(ac-ad)/W;for(var ab=0;ab<W;ab++){S.push([aa+(Y*ab),ad+(X*ab),0,0])}};for(var V=0;V<U.length;V++){T.apply(null,U[V])}return S},P=function(S){var U=[];for(var T=0;T<S.length;T++){U.push([S[T][0],S[T][1],S[T][2],S[T][3],1/S.length])}return M(U)},N=function(){return P([[0,0,1,0],[1,0,1,1],[1,1,0,1],[0,1,0,0]])};var K={circle:J,ellipse:J,diamond:function(){return P([[0.5,0,1,0.5],[1,0.5,0.5,1],[0.5,1,0,0.5],[0,0.5,0.5,0]])},rectangle:N,square:N,triangle:function(){return P([[0.5,0,1,1],[1,1,0,1],[0,1,0.5,0]])},path:function(X){var V=X.points;var W=[],T=0;for(var U=0;U<V.length-1;U++){var S=Math.sqrt(Math.pow(V[U][2]-V[U][0])+Math.pow(V[U][3]-V[U][1]));T+=S;W.push([V[U][0],V[U][1],V[U+1][0],V[U+1][1],S])}for(v
 ar U=0;U<W.length;U++){W[U][4]=W[U][4]/T}return M(W)}},Q=function(X,W){var Y=[],V=W/180*Math.PI;for(var U=0;U<X.length;U++){var T=X[U][0]-0.5,S=X[U][1]-0.5;Y.push([0.5+((T*Math.cos(V))-(S*Math.sin(V))),0.5+((T*Math.sin(V))+(S*Math.cos(V))),X[U][2],X[U][3]])}return Y};if(!K[L]){throw new Error("Shape ["+L+"] is unknown by Perimeter Anchor type")}var R=K[L](H);if(H.rotation){R=Q(R,H.rotation)}var O=H.jsPlumbInstance.makeDynamicAnchor(R);O.type="Perimeter";return O}})();(function(){jsPlumb.DOMElementComponent=function(d){jsPlumb.jsPlumbUIComponent.apply(this,arguments);this.mousemove=this.dblclick=this.click=this.mousedown=this.mouseup=function(f){}};jsPlumb.Connectors.Straight=function(){this.type="Straight";var s=this,k=null,f,l,q,o,m,g,r,i,h,e,d,p,n;this.compute=function(B,K,t,A,G,u,E,z){var J=Math.abs(B[0]-K[0]),D=Math.abs(B[1]-K[1]),C=0.45*J,v=0.45*D;J*=1.9;D*=1.9;var H=Math.min(B[0],K[0])-C;var F=Math.min(B[1],K[1])-v;var I=Math.max(2*E,z);if(J<I){J=I;H=B[0]+((K[0]-B[0])/2)-(I/2)
 ;C=(J-Math.abs(B[0]-K[0]))/2}if(D<I){D=I;F=B[1]+((K[1]-B[1])/2)-(I/2);v=(D-Math.abs(B[1]-K[1]))/2}i=B[0]<K[0]?C:J-C;h=B[1]<K[1]?v:D-v;e=B[0]<K[0]?J-C:C;d=B[1]<K[1]?D-v:v;k=[H,F,J,D,i,h,e,d];o=e-i,m=d-h;f=jsPlumbUtil.gradient({x:i,y:h},{x:e,y:d}),l=-1/f;q=-1*((f*i)-h);g=Math.atan(f);r=Math.atan(l);n=Math.sqrt((o*o)+(m*m));return k};this.pointOnPath=function(u,v){if(u==0&&!v){return{x:i,y:h}}else{if(u==1&&!v){return{x:e,y:d}}else{var t=v?u>0?u:n+u:u*n;return jsPlumbUtil.pointOnLine({x:i,y:h},{x:e,y:d},t)}}};this.gradientAtPoint=function(t){return f};this.pointAlongPathFrom=function(t,x,w){var v=s.pointOnPath(t,w),u=t==1?{x:i+((e-i)*10),y:h+((h-d)*10)}:x<=0?{x:i,y:h}:{x:e,y:d};if(x<=0&&Math.abs(x)>1){x*=-1}return jsPlumbUtil.pointOnLine(v,u,x)}};jsPlumb.Connectors.Bezier=function(w){var q=this;w=w||{};this.majorAnchor=w.curviness||150;this.minorAnchor=10;var u=null;this.type="Bezier";this._findControlPoint=function(I,x,D,y,B,G,z){var F=G.getOrientation(y),H=z.getOrientation(B),C=F[0]!=
 H[0]||F[1]==H[1],A=[],J=q.majorAnchor,E=q.minorAnchor;if(!C){if(F[0]==0){A.push(x[0]<D[0]?I[0]+E:I[0]-E)}else{A.push(I[0]-(J*F[0]))}if(F[1]==0){A.push(x[1]<D[1]?I[1]+E:I[1]-E)}else{A.push(I[1]+(J*H[1]))}}else{if(H[0]==0){A.push(D[0]<x[0]?I[0]+E:I[0]-E)}else{A.push(I[0]+(J*H[0]))}if(H[1]==0){A.push(D[1]<x[1]?I[1]+E:I[1]-E)}else{A.push(I[1]+(J*F[1]))}}return A};var r,m,g,p,o,g,f,t,s,v,e,i,h,l,k;this.compute=function(T,A,N,B,R,y,x,M){x=Math.max(M,(x||0));v=Math.abs(T[0]-A[0])+x;e=Math.abs(T[1]-A[1])+x;t=Math.min(T[0],A[0])-(x/2);s=Math.min(T[1],A[1])-(x/2);g=T[0]<A[0]?v-(x/2):(x/2);f=T[1]<A[1]?e-(x/2):(x/2);p=T[0]<A[0]?(x/2):v-(x/2);o=T[1]<A[1]?(x/2):e-(x/2);r=q._findControlPoint([g,f],T,A,N,B,R,y);m=q._findControlPoint([p,o],A,T,B,N,y,R);var L=Math.min(g,p),K=Math.min(r[0],m[0]),G=Math.min(L,K),S=Math.max(g,p),P=Math.max(r[0],m[0]),D=Math.max(S,P);if(D>v){v=D}if(G<0){t+=G;var I=Math.abs(G);v+=I;r[0]+=I;g+=I;p+=I;m[0]+=I}var Q=Math.min(f,o),O=Math.min(r[1],m[1]),C=Math.min(Q,O),H=Math.
 max(f,o),F=Math.max(r[1],m[1]),z=Math.max(H,F);if(z>e){e=z}if(C<0){s+=C;var E=Math.abs(C);e+=E;r[1]+=E;f+=E;o+=E;m[1]+=E}if(M&&v<M){var J=(M-v)/2;v=M;t-=J;g=g+J;p=p+J;r[0]=r[0]+J;m[0]=m[0]+J}if(M&&e<M){var J=(M-e)/2;e=M;s-=J;f=f+J;o=o+J;r[1]=r[1]+J;m[1]=m[1]+J}u=[t,s,v,e,g,f,p,o,r[0],r[1],m[0],m[1]];return u};var d=function(){return[{x:g,y:f},{x:r[0],y:r[1]},{x:m[0],y:m[1]},{x:p,y:o}]};var n=function(y,x,z){if(z){x=jsBezier.locationAlongCurveFrom(y,x>0?0:1,x)}return x};this.pointOnPath=function(x,z){var y=d();x=n(y,x,z);return jsBezier.pointOnCurve(y,x)};this.gradientAtPoint=function(x,z){var y=d();x=n(y,x,z);return jsBezier.gradientAtPoint(y,x)};this.pointAlongPathFrom=function(x,A,z){var y=d();x=n(y,x,z);return jsBezier.pointAlongCurveFrom(y,x,A)}};jsPlumb.Connectors.Flowchart=function(A){this.type="Flowchart";A=A||{};var s=this,e=A.stub||A.minStubLength||30,k=jsPlumbUtil.isArray(e)?e[0]:e,p=jsPlumbUtil.isArray(e)?e[1]:e,u=A.gap||0,f=A.midpoint||0.5,v=[],n=0,h=[],r=[],w=[],t,q,z=-
 Infinity,x=-Infinity,B=Infinity,y=Infinity,d=A.grid,g=function(H,D){var G=H%D,E=Math.floor(H/D),F=G>(D/2)?1:0;return(E+F)*D},o=function(D,G,F,E){return[F||d==null?D:g(D,d[0]),E||d==null?G:g(G,d[1])]},C=function(E,D,I,H){var G=0;for(var F=0;F<v.length;F++){r[F]=v[F][5]/n;h[F]=[G,(G+=(v[F][5]/n))]}},m=function(){w.push(v.length);for(var D=0;D<v.length;D++){w.push(v[D][0]);w.push(v[D][1])}},l=function(M,J,L,K,I,H){var E=v.length==0?L:v[v.length-1][0],D=v.length==0?K:v[v.length-1][1],F=M==E?Infinity:0;var G=Math.abs(M==E?J-D:M-E);v.push([M,J,E,D,F,G]);n+=G;z=Math.max(z,M);x=Math.max(x,J);B=Math.min(B,M);y=Math.min(y,J)},i=function(F,H){if(H){F=F>0?F/n:(n+F)/n}var D=h.length-1,E=1;for(var G=0;G<h.length;G++){if(h[G][1]>=F){D=G;E=(F-h[G][0])/r[G];break}}return{segment:v[D],proportion:E,index:D}};this.compute=function(aa,ao,D,U,az,O,Y,T,au,aq){v=[];h=[];n=0;r=[];z=x=-Infinity;B=y=Infinity;s.lineWidth=Y;t=ao[0]<aa[0];q=ao[1]<aa[1];var ae=Y||1,I=(ae/2)+(k+p),F=(ae/2)+(p+k),H=(ae/2)+(k+p),E=(
 ae/2)+(p+k),R=az.orientation||az.getOrientation(D),aA=O.orientation||O.getOrientation(U),an=t?ao[0]:aa[0],am=q?ao[1]:aa[1],ap=Math.abs(ao[0]-aa[0])+I+F,ay=Math.abs(ao[1]-aa[1])+H+E;if(R[0]==0&&R[1]==0||aA[0]==0&&aA[1]==0){var ag=ap>ay?0:1,ai=[1,0][ag];R=[];aA=[];R[ag]=aa[ag]>ao[ag]?-1:1;aA[ag]=aa[ag]>ao[ag]?1:-1;R[ai]=0;aA[ai]=0}var M=t?(ap-F)+(u*R[0]):I+(u*R[0]),L=q?(ay-E)+(u*R[1]):H+(u*R[1]),aw=t?I+(u*aA[0]):(ap-F)+(u*aA[0]),av=q?H+(u*aA[1]):(ay-E)+(u*aA[1]),ad=M+(R[0]*k),ac=L+(R[1]*k),P=aw+(aA[0]*p),N=av+(aA[1]*p),Z=Math.abs(M-aw)>(k+p),ab=Math.abs(L-av)>(k+p),al=ad+((P-ad)*f),aj=ac+((N-ac)*f),S=((R[0]*aA[0])+(R[1]*aA[1])),af=S==-1,ah=S==0,G=S==1;an-=I;am-=H;w=[an,am,ap,ay,M,L,aw,av];var at=[];var W=R[0]==0?"y":"x",Q=af?"opposite":G?"orthogonal":"perpendicular",J=jsPlumbUtil.segment([M,L],[aw,av]),ak=R[W=="x"?0:1]==-1,V={x:[null,4,3,2,1],y:[null,2,1,4,3]};if(ak){J=V[W][J]}l(ad,ac,M,L,aw,av);var X=function(aE,aD,aB,aC){return aE+(aD*((1-aB)*aC)+Math.max(k,p))},K={oppositex:functio
 n(){if(D.elementId==U.elementId){var aB=ac+((1-az.y)*au.height)+Math.max(k,p);return[[ad,aB],[P,aB]]}else{if(Z&&(J==1||J==2)){return[[al,L],[al,av]]}else{return[[ad,aj],[P,aj]]}}},orthogonalx:function(){if(J==1||

<TRUNCATED>

[27/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java
new file mode 100644
index 0000000..82fd6fc
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java
@@ -0,0 +1,421 @@
+/*
+ * 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.master;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.master.DefaultFragmentScheduleAlgorithm.FragmentsPerDisk;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+/**
+ * GreedyFragmentScheduleAlgorithm selects a fragment considering the number of fragments that are not scheduled yet.
+ * Disks of hosts have the priorities which are represented by the remaining number of fragments.
+ * This algorithm selects a fragment with trying minimizing the maximum priority.
+ */
+public class GreedyFragmentScheduleAlgorithm implements FragmentScheduleAlgorithm {
+  private final static Log LOG = LogFactory.getLog(GreedyFragmentScheduleAlgorithm.class);
+  private final HostPriorityComparator hostComparator = new HostPriorityComparator();
+  private Map<String, Map<Integer, FragmentsPerDisk>> fragmentHostMapping =
+      new HashMap<String, Map<Integer, FragmentsPerDisk>>();
+  private Map<HostAndDisk, PrioritizedHost> totalHostPriority = new HashMap<HostAndDisk, PrioritizedHost>();
+  private Map<String, Set<PrioritizedHost>> hostPriorityPerRack = new HashMap<String, Set<PrioritizedHost>>();
+  private TopologyCache topologyCache = new TopologyCache();
+  private int totalFragmentNum = 0;
+
+  private FragmentsPerDisk getHostFragmentSet(String host, Integer diskId) {
+    Map<Integer, FragmentsPerDisk> fragmentsPerDiskMap;
+    FragmentsPerDisk fragmentsPerDisk;
+    if (fragmentHostMapping.containsKey(host)) {
+      fragmentsPerDiskMap = fragmentHostMapping.get(host);
+    } else {
+      fragmentsPerDiskMap = new HashMap<Integer, FragmentsPerDisk>();
+      fragmentHostMapping.put(host, fragmentsPerDiskMap);
+    }
+    if (fragmentsPerDiskMap.containsKey(diskId)) {
+      fragmentsPerDisk = fragmentsPerDiskMap.get(diskId);
+    } else {
+      fragmentsPerDisk = new FragmentsPerDisk(diskId);
+      fragmentsPerDiskMap.put(diskId, fragmentsPerDisk);
+    }
+    return fragmentsPerDisk;
+  }
+
+  private void updateHostPriority(HostAndDisk hostAndDisk, int priority) {
+    if (priority > 0) {
+      // update the priority among the total hosts
+      PrioritizedHost prioritizedHost;
+      if (totalHostPriority.containsKey(hostAndDisk)) {
+        prioritizedHost = totalHostPriority.get(hostAndDisk);
+        prioritizedHost.priority = priority;
+      } else {
+        prioritizedHost = new PrioritizedHost(hostAndDisk, priority);
+        totalHostPriority.put(hostAndDisk, prioritizedHost);
+      }
+
+      // update the priority among the hosts in a rack
+      String rack = topologyCache.resolve(hostAndDisk.host);
+      Set<PrioritizedHost> hostsOfRack;
+      if (!hostPriorityPerRack.containsKey(rack)) {
+        hostsOfRack = new HashSet<PrioritizedHost>();
+        hostsOfRack.add(prioritizedHost);
+        hostPriorityPerRack.put(rack, hostsOfRack);
+      }
+    } else {
+      if (totalHostPriority.containsKey(hostAndDisk)) {
+        PrioritizedHost prioritizedHost = totalHostPriority.remove(hostAndDisk);
+
+        String rack = topologyCache.resolve(hostAndDisk.host);
+        if (hostPriorityPerRack.containsKey(rack)) {
+          Set<PrioritizedHost> hostsOfRack = hostPriorityPerRack.get(rack);
+          hostsOfRack.remove(prioritizedHost);
+          if (hostsOfRack.size() == 0){
+            hostPriorityPerRack.remove(rack);
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void addFragment(FragmentPair fragmentPair) {
+    String[] hosts = fragmentPair.getLeftFragment().getHosts();
+    int[] diskIds = fragmentPair.getLeftFragment().getDiskIds();
+    for (int i = 0; i < hosts.length; i++) {
+      addFragment(hosts[i], diskIds[i], fragmentPair);
+    }
+    totalFragmentNum++;
+  }
+
+  private void addFragment(String host, Integer diskId, FragmentPair fragmentPair) {
+    host = topologyCache.normalize(host);
+    FragmentsPerDisk fragmentsPerDisk = getHostFragmentSet(host, diskId);
+    fragmentsPerDisk.addFragmentPair(fragmentPair);
+
+    int priority;
+    HostAndDisk hostAndDisk = new HostAndDisk(host, diskId);
+    if (totalHostPriority.containsKey(hostAndDisk)) {
+      priority = totalHostPriority.get(hostAndDisk).priority;
+    } else {
+      priority = 0;
+    }
+    updateHostPriority(hostAndDisk, priority+1);
+  }
+
+  public int size() {
+    return totalFragmentNum;
+  }
+
+  /**
+   * Selects a fragment that is stored in the given host, and replicated at the disk of the maximum
+   * priority.
+   * @param host
+   * @return If there are fragments stored in the host, returns a fragment. Otherwise, return null.
+   */
+  @Override
+  public FragmentPair getHostLocalFragment(String host) {
+    String normalizedHost = topologyCache.normalize(host);
+    if (!fragmentHostMapping.containsKey(normalizedHost)) {
+      return null;
+    }
+
+    Map<Integer, FragmentsPerDisk> fragmentsPerDiskMap = fragmentHostMapping.get(normalizedHost);
+    List<Integer> disks = Lists.newArrayList(fragmentsPerDiskMap.keySet());
+    Collections.shuffle(disks);
+    FragmentsPerDisk fragmentsPerDisk = null;
+    FragmentPair fragmentPair = null;
+
+    for (Integer diskId : disks) {
+      fragmentsPerDisk = fragmentsPerDiskMap.get(diskId);
+      if (fragmentsPerDisk != null && !fragmentsPerDisk.isEmpty()) {
+        fragmentPair = getBestFragment(fragmentsPerDisk);
+      }
+      if (fragmentPair != null) {
+        return fragmentPair;
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * Selects a fragment that is stored at the given disk of the given host, and replicated at the disk of the maximum
+   * priority.
+   * @param host
+   * @param diskId
+   * @return If there are fragments stored at the disk of the host, returns a fragment. Otherwise, return null.
+   */
+  @Override
+  public FragmentPair getHostLocalFragment(String host, Integer diskId) {
+    String normalizedHost = NetUtils.normalizeHost(host);
+    if (fragmentHostMapping.containsKey(normalizedHost)) {
+      Map<Integer, FragmentsPerDisk> fragmentsPerDiskMap = fragmentHostMapping.get(normalizedHost);
+      if (fragmentsPerDiskMap.containsKey(diskId)) {
+        FragmentsPerDisk fragmentsPerDisk = fragmentsPerDiskMap.get(diskId);
+        if (!fragmentsPerDisk.isEmpty()) {
+          return getBestFragment(fragmentsPerDisk);
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * In the descending order of priority, find a fragment that is shared by the given fragment set and the fragment set
+   * of the maximal priority.
+   * @param fragmentsPerDisk a fragment set
+   * @return a fragment that is shared by the given fragment set and the fragment set of the maximal priority
+   */
+  private FragmentPair getBestFragment(FragmentsPerDisk fragmentsPerDisk) {
+    // Select a fragment that is shared by host and another hostAndDisk that has the most fragments
+    Collection<PrioritizedHost> prioritizedHosts = totalHostPriority.values();
+    PrioritizedHost[] sortedHosts = prioritizedHosts.toArray(new PrioritizedHost[prioritizedHosts.size()]);
+    Arrays.sort(sortedHosts, hostComparator);
+
+    for (PrioritizedHost nextHost : sortedHosts) {
+      if (fragmentHostMapping.containsKey(nextHost.hostAndDisk.host)) {
+        Map<Integer, FragmentsPerDisk> diskFragmentsMap = fragmentHostMapping.get(nextHost.hostAndDisk.host);
+        if (diskFragmentsMap.containsKey(nextHost.hostAndDisk.diskId)) {
+          Set<FragmentPair> largeFragmentPairSet = diskFragmentsMap.get(nextHost.hostAndDisk.diskId).getFragmentPairSet();
+          Iterator<FragmentPair> smallFragmentSetIterator = fragmentsPerDisk.getFragmentPairIterator();
+          while (smallFragmentSetIterator.hasNext()) {
+            FragmentPair eachFragmentOfSmallSet = smallFragmentSetIterator.next();
+            if (largeFragmentPairSet.contains(eachFragmentOfSmallSet)) {
+              return eachFragmentOfSmallSet;
+            }
+          }
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Selects a fragment that is stored at the same rack of the given host, and replicated at the disk of the maximum
+   * priority.
+   * @param host
+   * @return If there are fragments stored at the same rack of the given host, returns a fragment. Otherwise, return null.
+   */
+  public FragmentPair getRackLocalFragment(String host) {
+    host = topologyCache.normalize(host);
+    // Select a fragment from a host that has the most fragments in the rack
+    String rack = topologyCache.resolve(host);
+    Set<PrioritizedHost> hostsOfRack = hostPriorityPerRack.get(rack);
+    if (hostsOfRack != null && hostsOfRack.size() > 0) {
+      PrioritizedHost[] sortedHosts = hostsOfRack.toArray(new PrioritizedHost[hostsOfRack.size()]);
+      Arrays.sort(sortedHosts, hostComparator);
+      for (PrioritizedHost nextHost : sortedHosts) {
+        if (fragmentHostMapping.containsKey(nextHost.hostAndDisk.host)) {
+          List<FragmentsPerDisk> disks = Lists.newArrayList(fragmentHostMapping.get(nextHost.hostAndDisk.host).values());
+          Collections.shuffle(disks);
+
+          for (FragmentsPerDisk fragmentsPerDisk : disks) {
+            if (!fragmentsPerDisk.isEmpty()) {
+              return fragmentsPerDisk.getFragmentPairIterator().next();
+            }
+          }
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Selects a fragment from the disk of the maximum priority.
+   * @return If there are remaining fragments, it returns a fragment. Otherwise, it returns null.
+   */
+  public FragmentPair getRandomFragment() {
+    // Select a fragment from a host that has the most fragments
+    Collection<PrioritizedHost> prioritizedHosts = totalHostPriority.values();
+    PrioritizedHost[] sortedHosts = prioritizedHosts.toArray(new PrioritizedHost[prioritizedHosts.size()]);
+    Arrays.sort(sortedHosts, hostComparator);
+    PrioritizedHost randomHost = sortedHosts[0];
+    if (fragmentHostMapping.containsKey(randomHost.hostAndDisk.host)) {
+      Iterator<FragmentsPerDisk> fragmentsPerDiskIterator = fragmentHostMapping.get(randomHost.hostAndDisk.host).values().iterator();
+      if (fragmentsPerDiskIterator.hasNext()) {
+        Iterator<FragmentPair> fragmentPairIterator = fragmentsPerDiskIterator.next().getFragmentPairIterator();
+        if (fragmentPairIterator.hasNext()) {
+          return fragmentPairIterator.next();
+        }
+      }
+    }
+    return null;
+  }
+
+  public FragmentPair[] getAllFragments() {
+    List<FragmentPair> fragmentPairs = new ArrayList<FragmentPair>();
+    for (Map<Integer, FragmentsPerDisk> eachValue : fragmentHostMapping.values()) {
+      for (FragmentsPerDisk fragmentsPerDisk : eachValue.values()) {
+        Set<FragmentPair> pairSet = fragmentsPerDisk.getFragmentPairSet();
+        fragmentPairs.addAll(pairSet);
+      }
+    }
+    return fragmentPairs.toArray(new FragmentPair[fragmentPairs.size()]);
+  }
+
+  public void removeFragment(FragmentPair fragmentPair) {
+    String [] hosts = fragmentPair.getLeftFragment().getHosts();
+    int[] diskIds = fragmentPair.getLeftFragment().getDiskIds();
+    for (int i = 0; i < hosts.length; i++) {
+      String normalizedHost = NetUtils.normalizeHost(hosts[i]);
+      Map<Integer, FragmentsPerDisk> diskFragmentMap = fragmentHostMapping.get(normalizedHost);
+
+      if (diskFragmentMap != null) {
+        FragmentsPerDisk fragmentsPerDisk = diskFragmentMap.get(diskIds[i]);
+        if (fragmentsPerDisk != null) {
+          boolean isRemoved = fragmentsPerDisk.removeFragmentPair(fragmentPair);
+          if (isRemoved) {
+            if (fragmentsPerDisk.size() == 0) {
+              diskFragmentMap.remove(diskIds[i]);
+              if (diskFragmentMap.size() == 0) {
+                fragmentHostMapping.remove(normalizedHost);
+              }
+            }
+            HostAndDisk hostAndDisk = new HostAndDisk(normalizedHost, diskIds[i]);
+            if (totalHostPriority.containsKey(hostAndDisk)) {
+              PrioritizedHost prioritizedHost = totalHostPriority.get(hostAndDisk);
+              updateHostPriority(prioritizedHost.hostAndDisk, prioritizedHost.priority-1);
+            }
+          }
+        }
+      }
+    }
+
+    totalFragmentNum--;
+  }
+
+  private static class HostAndDisk {
+    private String host;
+    private Integer diskId;
+
+    public HostAndDisk(String host, Integer diskId) {
+      this.host = host;
+      this.diskId = diskId;
+    }
+
+    public String getHost() {
+      return host;
+    }
+
+    public int getDiskId() {
+      return diskId;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(host, diskId);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof HostAndDisk) {
+        HostAndDisk other = (HostAndDisk) o;
+        return this.host.equals(other.host) &&
+            TUtil.checkEquals(this.diskId, other.diskId);
+      }
+      return false;
+    }
+  }
+
+  public static class PrioritizedHost {
+    private HostAndDisk hostAndDisk;
+    private int priority;
+
+    public PrioritizedHost(HostAndDisk hostAndDisk, int priority) {
+      this.hostAndDisk = hostAndDisk;
+      this.priority = priority;
+    }
+
+    public PrioritizedHost(String host, Integer diskId, int priority) {
+      this.hostAndDisk = new HostAndDisk(host, diskId);
+      this.priority = priority;
+    }
+
+    public String getHost() {
+      return hostAndDisk.host;
+    }
+
+    public Integer getDiskId() {
+      return hostAndDisk.diskId;
+    }
+
+    public Integer getPriority() {
+      return priority;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof PrioritizedHost) {
+        PrioritizedHost other = (PrioritizedHost) o;
+        return this.hostAndDisk.equals(other.hostAndDisk);
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return hostAndDisk.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return "host: " + hostAndDisk.host + " disk: " + hostAndDisk.diskId + " priority: " + priority;
+    }
+  }
+
+
+  public static class HostPriorityComparator implements Comparator<PrioritizedHost> {
+
+    @Override
+    public int compare(PrioritizedHost prioritizedHost, PrioritizedHost prioritizedHost2) {
+      return prioritizedHost2.priority - prioritizedHost.priority;
+    }
+  }
+
+
+  public static class TopologyCache {
+    private Map<String, String> hostRackMap = new HashMap<String, String>();
+    private Map<String, String> normalizedHostMap = new HashMap<String, String>();
+
+    public String normalize(String host) {
+      if (normalizedHostMap.containsKey(host)) {
+        return normalizedHostMap.get(host);
+      } else {
+        String normalized = NetUtils.normalizeHost(host);
+        normalizedHostMap.put(host, normalized);
+        return normalized;
+      }
+    }
+
+    public String resolve(String host) {
+      if (hostRackMap.containsKey(host)) {
+        return hostRackMap.get(host);
+      } else {
+        String rack = RackResolver.resolve(host).getNetworkLocation();
+        hostRackMap.put(host, rack);
+        return rack;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
new file mode 100644
index 0000000..434ea22
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
@@ -0,0 +1,522 @@
+/**
+ * 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.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.query.QueryUnitRequest;
+import org.apache.tajo.engine.query.QueryUnitRequestImpl;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext;
+import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryUnit;
+import org.apache.tajo.master.querymaster.QueryUnitAttempt;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.NetUtils;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public class LazyTaskScheduler extends AbstractTaskScheduler {
+  private static final Log LOG = LogFactory.getLog(LazyTaskScheduler.class);
+
+  private final TaskSchedulerContext context;
+  private final SubQuery subQuery;
+
+  private Thread schedulingThread;
+  private volatile boolean stopEventHandling;
+
+  BlockingQueue<TaskSchedulerEvent> eventQueue
+      = new LinkedBlockingQueue<TaskSchedulerEvent>();
+
+  private TaskRequests taskRequests;
+  private FragmentScheduleAlgorithm scheduledFragments;
+  private ScheduledFetches scheduledFetches;
+
+  private int diskLocalAssigned = 0;
+  private int hostLocalAssigned = 0;
+  private int rackLocalAssigned = 0;
+  private int totalAssigned = 0;
+
+  private int nextTaskId = 0;
+  private int containerNum;
+
+  public LazyTaskScheduler(TaskSchedulerContext context, SubQuery subQuery) {
+    super(LazyTaskScheduler.class.getName());
+    this.context = context;
+    this.subQuery = subQuery;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    taskRequests  = new TaskRequests();
+    try {
+      scheduledFragments = FragmentScheduleAlgorithmFactory.get(conf);
+      LOG.info(scheduledFragments.getClass().getSimpleName() + " is selected for the scheduling algorithm.");
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    if (!context.isLeafQuery()) {
+      scheduledFetches = new ScheduledFetches();
+    }
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    containerNum = subQuery.getContext().getResourceAllocator().calculateNumRequestContainers(
+        subQuery.getContext().getQueryMasterContext().getWorkerContext(),
+        context.getEstimatedTaskNum(), 512);
+
+    LOG.info("Start TaskScheduler");
+    this.schedulingThread = new Thread() {
+      public void run() {
+
+        while(!stopEventHandling && !Thread.currentThread().isInterrupted()) {
+          try {
+            Thread.sleep(100);
+          } catch (InterruptedException e) {
+            break;
+          }
+
+          schedule();
+        }
+        LOG.info("TaskScheduler schedulingThread stopped");
+      }
+    };
+
+    this.schedulingThread.start();
+    super.start();
+  }
+
+  private static final QueryUnitAttemptId NULL_ATTEMPT_ID;
+  public static final TajoWorkerProtocol.QueryUnitRequestProto stopTaskRunnerReq;
+  static {
+    ExecutionBlockId nullSubQuery = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
+    NULL_ATTEMPT_ID = QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(nullSubQuery, 0), 0);
+
+    TajoWorkerProtocol.QueryUnitRequestProto.Builder builder =
+        TajoWorkerProtocol.QueryUnitRequestProto.newBuilder();
+    builder.setId(NULL_ATTEMPT_ID.getProto());
+    builder.setShouldDie(true);
+    builder.setOutputTable("");
+    builder.setSerializedData("");
+    builder.setClusteredOutput(false);
+    stopTaskRunnerReq = builder.build();
+  }
+
+  @Override
+  public void stop() {
+    stopEventHandling = true;
+    schedulingThread.interrupt();
+
+    // Return all of request callbacks instantly.
+    for (TaskRequestEvent req : taskRequests.taskRequestQueue) {
+      req.getCallback().run(stopTaskRunnerReq);
+    }
+
+    LOG.info("Task Scheduler stopped");
+    super.stop();
+  }
+
+  List<TaskRequestEvent> taskRequestEvents = new ArrayList<TaskRequestEvent>();
+  public void schedule() {
+    if (taskRequests.size() > 0) {
+      if (context.isLeafQuery()) {
+        LOG.debug("Try to schedule tasks with taskRequestEvents: " +
+            taskRequests.size() + ", Fragment Schedule Request: " +
+            scheduledFragments.size());
+        taskRequests.getTaskRequests(taskRequestEvents,
+            scheduledFragments.size());
+        LOG.debug("Get " + taskRequestEvents.size() + " taskRequestEvents ");
+        if (taskRequestEvents.size() > 0) {
+          assignLeafTasks(taskRequestEvents);
+        }
+        taskRequestEvents.clear();
+      } else {
+        LOG.debug("Try to schedule tasks with taskRequestEvents: " +
+            taskRequests.size() + ", Fetch Schedule Request: " +
+            scheduledFetches.size());
+        taskRequests.getTaskRequests(taskRequestEvents,
+            scheduledFetches.size());
+        LOG.debug("Get " + taskRequestEvents.size() + " taskRequestEvents ");
+        if (taskRequestEvents.size() > 0) {
+          assignNonLeafTasks(taskRequestEvents);
+        }
+        taskRequestEvents.clear();
+      }
+    }
+  }
+
+  @Override
+  public void handle(TaskSchedulerEvent event) {
+    int qSize = eventQueue.size();
+    if (qSize != 0 && qSize % 1000 == 0) {
+      LOG.info("Size of event-queue in YarnRMContainerAllocator is " + qSize);
+    }
+    int remCapacity = eventQueue.remainingCapacity();
+    if (remCapacity < 1000) {
+      LOG.warn("Very low remaining capacity in the event-queue "
+          + "of YarnRMContainerAllocator: " + remCapacity);
+    }
+
+    if (event.getType() == EventType.T_SCHEDULE) {
+      if (event instanceof FragmentScheduleEvent) {
+        FragmentScheduleEvent castEvent = (FragmentScheduleEvent) event;
+        Collection<FileFragment> rightFragments = castEvent.getRightFragments();
+        if (rightFragments == null || rightFragments.isEmpty()) {
+          scheduledFragments.addFragment(new FragmentPair(castEvent.getLeftFragment(), null));
+        } else {
+          for (FileFragment eachFragment: rightFragments) {
+            scheduledFragments.addFragment(new FragmentPair(castEvent.getLeftFragment(), eachFragment));
+          }
+        }
+        initDiskBalancer(castEvent.getLeftFragment().getHosts(), castEvent.getLeftFragment().getDiskIds());
+      } else if (event instanceof FetchScheduleEvent) {
+        FetchScheduleEvent castEvent = (FetchScheduleEvent) event;
+        scheduledFetches.addFetch(castEvent.getFetches());
+      } else if (event instanceof QueryUnitAttemptScheduleEvent) {
+        QueryUnitAttemptScheduleEvent castEvent = (QueryUnitAttemptScheduleEvent) event;
+        assignTask(castEvent.getContext(), castEvent.getQueryUnitAttempt());
+      }
+    }
+  }
+
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+    taskRequests.handle(event);
+  }
+
+  @Override
+  public int remainingScheduledObjectNum() {
+    if (context.isLeafQuery()) {
+      return scheduledFragments.size();
+    } else {
+      return scheduledFetches.size();
+    }
+  }
+
+  private Map<String, DiskBalancer> hostDiskBalancerMap = new HashMap<String, DiskBalancer>();
+
+  private void initDiskBalancer(String[] hosts, int[] diskIds) {
+    for (int i = 0; i < hosts.length; i++) {
+      DiskBalancer diskBalancer;
+      String normalized = NetUtils.normalizeHost(hosts[i]);
+      if (hostDiskBalancerMap.containsKey(normalized)) {
+        diskBalancer = hostDiskBalancerMap.get(normalized);
+      } else {
+        diskBalancer = new DiskBalancer(normalized);
+        hostDiskBalancerMap.put(normalized, diskBalancer);
+      }
+      diskBalancer.addDiskId(diskIds[i]);
+    }
+  }
+
+  private static class DiskBalancer {
+    private HashMap<ContainerId, Integer> containerDiskMap = new HashMap<ContainerId, Integer>();
+    private HashMap<Integer, Integer> diskReferMap = new HashMap<Integer, Integer>();
+    private String host;
+
+    public DiskBalancer(String host){
+      this.host = host;
+    }
+
+    public void addDiskId(Integer diskId) {
+      if (!diskReferMap.containsKey(diskId)) {
+        diskReferMap.put(diskId, 0);
+      }
+    }
+
+    public Integer getDiskId(ContainerId containerId) {
+      if (!containerDiskMap.containsKey(containerId)) {
+        assignVolumeId(containerId);
+      }
+
+      return containerDiskMap.get(containerId);
+    }
+
+    public void assignVolumeId(ContainerId containerId){
+      Map.Entry<Integer, Integer> volumeEntry = null;
+
+      for (Map.Entry<Integer, Integer> entry : diskReferMap.entrySet()) {
+        if(volumeEntry == null) volumeEntry = entry;
+
+        if (volumeEntry.getValue() >= entry.getValue()) {
+          volumeEntry = entry;
+        }
+      }
+
+      if(volumeEntry != null){
+        diskReferMap.put(volumeEntry.getKey(), volumeEntry.getValue() + 1);
+        LOG.info("Assigned host : " + host + " Volume : " + volumeEntry.getKey() + ", Concurrency : "
+            + diskReferMap.get(volumeEntry.getKey()));
+        containerDiskMap.put(containerId, volumeEntry.getKey());
+      }
+    }
+
+    public String getHost() {
+      return host;
+    }
+  }
+
+  private class TaskRequests implements EventHandler<TaskRequestEvent> {
+    private final LinkedBlockingQueue<TaskRequestEvent> taskRequestQueue =
+        new LinkedBlockingQueue<TaskRequestEvent>();
+
+    @Override
+    public void handle(TaskRequestEvent event) {
+      LOG.info("TaskRequest: " + event.getContainerId() + "," + event.getExecutionBlockId());
+      if(stopEventHandling) {
+        event.getCallback().run(stopTaskRunnerReq);
+        return;
+      }
+      int qSize = taskRequestQueue.size();
+      if (qSize != 0 && qSize % 1000 == 0) {
+        LOG.info("Size of event-queue in YarnRMContainerAllocator is " + qSize);
+      }
+      int remCapacity = taskRequestQueue.remainingCapacity();
+      if (remCapacity < 1000) {
+        LOG.warn("Very low remaining capacity in the event-queue "
+            + "of YarnRMContainerAllocator: " + remCapacity);
+      }
+
+      taskRequestQueue.add(event);
+    }
+
+    public void getTaskRequests(final Collection<TaskRequestEvent> taskRequests,
+                                int num) {
+      taskRequestQueue.drainTo(taskRequests, num);
+    }
+
+    public int size() {
+      return taskRequestQueue.size();
+    }
+  }
+
+  private long adjustTaskSize() {
+    long originTaskSize = context.getMasterContext().getConf().getIntVar(ConfVars.TASK_DEFAULT_SIZE) * 1024 * 1024;
+    long fragNumPerTask = context.getTaskSize() / originTaskSize;
+    if (fragNumPerTask * containerNum > remainingScheduledObjectNum()) {
+      return context.getTaskSize();
+    } else {
+      fragNumPerTask = (long) Math.ceil((double)remainingScheduledObjectNum() / (double)containerNum);
+      return originTaskSize * fragNumPerTask;
+    }
+  }
+
+  private void assignLeafTasks(List<TaskRequestEvent> taskRequests) {
+    Collections.shuffle(taskRequests);
+    Iterator<TaskRequestEvent> it = taskRequests.iterator();
+
+    TaskRequestEvent taskRequest;
+    while (it.hasNext() && scheduledFragments.size() > 0) {
+      taskRequest = it.next();
+      LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," +
+          "containerId=" + taskRequest.getContainerId());
+      ContainerProxy container = context.getMasterContext().getResourceAllocator().
+          getContainer(taskRequest.getContainerId());
+
+      if(container == null) {
+        continue;
+      }
+
+      String host = container.getTaskHostName();
+      QueryUnitAttemptScheduleContext queryUnitContext = new QueryUnitAttemptScheduleContext(container.containerID,
+          host, taskRequest.getCallback());
+      QueryUnit task = SubQuery.newEmptyQueryUnit(context, queryUnitContext, subQuery, nextTaskId++);
+
+      FragmentPair fragmentPair;
+      List<FragmentPair> fragmentPairs = new ArrayList<FragmentPair>();
+      boolean diskLocal = false;
+      long assignedFragmentSize = 0;
+      long taskSize = adjustTaskSize();
+      LOG.info("Adjusted task size: " + taskSize);
+
+      // host local, disk local
+      String normalized = NetUtils.normalizeHost(host);
+      Integer diskId = hostDiskBalancerMap.get(normalized).getDiskId(container.containerID);
+      if (diskId != null && diskId != -1) {
+        do {
+          fragmentPair = scheduledFragments.getHostLocalFragment(host, diskId);
+          if (fragmentPair == null || fragmentPair.getLeftFragment() == null) {
+            break;
+          }
+
+          if (assignedFragmentSize + fragmentPair.getLeftFragment().getEndKey() > taskSize) {
+            break;
+          } else {
+            fragmentPairs.add(fragmentPair);
+            assignedFragmentSize += fragmentPair.getLeftFragment().getEndKey();
+            if (fragmentPair.getRightFragment() != null) {
+              assignedFragmentSize += fragmentPair.getRightFragment().getEndKey();
+            }
+          }
+          scheduledFragments.removeFragment(fragmentPair);
+          diskLocal = true;
+        } while (scheduledFragments.size() > 0 && assignedFragmentSize < taskSize);
+      }
+
+      if (assignedFragmentSize < taskSize) {
+        // host local
+        do {
+          fragmentPair = scheduledFragments.getHostLocalFragment(host);
+          if (fragmentPair == null || fragmentPair.getLeftFragment() == null) {
+            break;
+          }
+
+          if (assignedFragmentSize + fragmentPair.getLeftFragment().getEndKey() > taskSize) {
+            break;
+          } else {
+            fragmentPairs.add(fragmentPair);
+            assignedFragmentSize += fragmentPair.getLeftFragment().getEndKey();
+            if (fragmentPair.getRightFragment() != null) {
+              assignedFragmentSize += fragmentPair.getRightFragment().getEndKey();
+            }
+          }
+          scheduledFragments.removeFragment(fragmentPair);
+          diskLocal = false;
+        } while (scheduledFragments.size() > 0 && assignedFragmentSize < taskSize);
+      }
+
+      // rack local
+      if (fragmentPairs.size() == 0) {
+        fragmentPair = scheduledFragments.getRackLocalFragment(host);
+
+        // random
+        if (fragmentPair == null) {
+          fragmentPair = scheduledFragments.getRandomFragment();
+        } else {
+          rackLocalAssigned++;
+        }
+
+        if (fragmentPair != null) {
+          fragmentPairs.add(fragmentPair);
+          scheduledFragments.removeFragment(fragmentPair);
+        }
+      } else {
+        if (diskLocal) {
+          diskLocalAssigned++;
+        } else {
+          hostLocalAssigned++;
+        }
+      }
+
+      if (fragmentPairs.size() == 0) {
+        throw new RuntimeException("Illegal State!!!!!!!!!!!!!!!!!!!!!");
+      }
+
+      LOG.info("host: " + host + " disk id: " + diskId + " fragment num: " + fragmentPairs.size());
+
+      task.setFragment(fragmentPairs.toArray(new FragmentPair[fragmentPairs.size()]));
+      subQuery.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+    }
+  }
+
+  private void assignNonLeafTasks(List<TaskRequestEvent> taskRequests) {
+    Iterator<TaskRequestEvent> it = taskRequests.iterator();
+
+    TaskRequestEvent taskRequest;
+    while (it.hasNext()) {
+      taskRequest = it.next();
+      LOG.debug("assignToNonLeafTasks: " + taskRequest.getExecutionBlockId());
+
+      // random allocation
+      if (scheduledFetches.size() > 0) {
+        LOG.debug("Assigned based on * match");
+        ContainerProxy container = context.getMasterContext().getResourceAllocator().getContainer(
+            taskRequest.getContainerId());
+        QueryUnitAttemptScheduleContext queryUnitContext = new QueryUnitAttemptScheduleContext(container.containerID,
+            container.getTaskHostName(), taskRequest.getCallback());
+        QueryUnit task = SubQuery.newEmptyQueryUnit(context, queryUnitContext, subQuery, nextTaskId++);
+        task.setFragment(scheduledFragments.getAllFragments());
+        subQuery.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+      }
+    }
+  }
+
+  private void assignTask(QueryUnitAttemptScheduleContext attemptContext, QueryUnitAttempt taskAttempt) {
+    QueryUnitAttemptId attemptId = taskAttempt.getId();
+    ContainerProxy containerProxy = context.getMasterContext().getResourceAllocator().
+        getContainer(attemptContext.getContainerId());
+    QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
+        attemptId,
+        new ArrayList<FragmentProto>(taskAttempt.getQueryUnit().getAllFragments()),
+        "",
+        false,
+        taskAttempt.getQueryUnit().getLogicalPlan().toJson(),
+        context.getMasterContext().getQueryContext(),
+        subQuery.getDataChannel(), subQuery.getBlock().getEnforcer());
+    if (checkIfInterQuery(subQuery.getMasterPlan(), subQuery.getBlock())) {
+      taskAssign.setInterQuery();
+    }
+
+    if (!context.isLeafQuery()) {
+      Map<String, List<URI>> fetch = scheduledFetches.getNextFetch();
+      scheduledFetches.popNextFetch();
+
+      for (Entry<String, List<URI>> fetchEntry : fetch.entrySet()) {
+        for (URI eachValue : fetchEntry.getValue()) {
+          taskAssign.addFetch(fetchEntry.getKey(), eachValue);
+        }
+      }
+    }
+
+    context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
+        attemptContext.getContainerId(), attemptContext.getHost(), containerProxy.getTaskPort()));
+
+    totalAssigned++;
+    attemptContext.getCallback().run(taskAssign.getProto());
+
+    if (context.isLeafQuery()) {
+      LOG.debug("DiskLocalAssigned / Total: " + diskLocalAssigned + " / " + totalAssigned);
+      LOG.debug("HostLocalAssigned / Total: " + hostLocalAssigned + " / " + totalAssigned);
+      LOG.debug("RackLocalAssigned: " + rackLocalAssigned + " / " + totalAssigned);
+    }
+  }
+
+  private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) {
+    if (masterPlan.isRoot(block)) {
+      return false;
+    }
+
+    ExecutionBlock parent = masterPlan.getParent(block);
+    if (masterPlan.isRoot(parent) && parent.hasUnion()) {
+      return false;
+    }
+
+    return true;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java b/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java
new file mode 100644
index 0000000..9b7dc22
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java
@@ -0,0 +1,48 @@
+/*
+ * 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.master;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class ScheduledFetches {
+  private List<Map<String, List<URI>>> fetches = new ArrayList<Map<String, List<URI>>>();
+
+  public void addFetch(Map<String, List<URI>> fetch) {
+    this.fetches.add(fetch);
+  }
+
+  public boolean hasNextFetch() {
+    return fetches.size() > 0;
+  }
+
+  public Map<String, List<URI>> getNextFetch() {
+    return hasNextFetch() ? fetches.get(0) : null;
+  }
+
+  public Map<String, List<URI>> popNextFetch() {
+    return hasNextFetch() ? fetches.remove(0) : null;
+  }
+
+  public int size() {
+    return fetches.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
new file mode 100644
index 0000000..751b21b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
@@ -0,0 +1,232 @@
+/**
+ * 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.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class TajoAsyncDispatcher extends AbstractService  implements Dispatcher {
+
+  private static final Log LOG = LogFactory.getLog(TajoAsyncDispatcher.class);
+
+  private final BlockingQueue<Event> eventQueue;
+  private volatile boolean stopped = false;
+
+  private Thread eventHandlingThread;
+  protected final Map<Class<? extends Enum>, EventHandler> eventDispatchers;
+  private boolean exitOnDispatchException;
+
+  private String id;
+
+  public TajoAsyncDispatcher(String id) {
+    this(id, new LinkedBlockingQueue<Event>());
+  }
+
+  public TajoAsyncDispatcher(String id, BlockingQueue<Event> eventQueue) {
+    super(TajoAsyncDispatcher.class.getName());
+    this.id = id;
+    this.eventQueue = eventQueue;
+    this.eventDispatchers = new HashMap<Class<? extends Enum>, EventHandler>();
+  }
+
+  Runnable createThread() {
+    return new Runnable() {
+      @Override
+      public void run() {
+        while (!stopped && !Thread.currentThread().isInterrupted()) {
+          Event event;
+          try {
+            event = eventQueue.take();
+            if(LOG.isDebugEnabled()) {
+              LOG.debug(id + ",event take:" + event.getType() + "," + event);
+            }
+          } catch(InterruptedException ie) {
+            if (!stopped) {
+              LOG.warn("AsyncDispatcher thread interrupted");
+            }
+            return;
+          }
+          dispatch(event);
+        }
+      }
+    };
+  }
+
+  @Override
+  public synchronized void init(Configuration conf) {
+    this.exitOnDispatchException =
+        conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
+            Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    //start all the components
+    super.start();
+    eventHandlingThread = new Thread(createThread());
+    eventHandlingThread.setName("AsyncDispatcher event handler");
+    eventHandlingThread.start();
+
+    LOG.info("AsyncDispatcher started:" + id);
+  }
+
+  @Override
+  public synchronized void stop() {
+    if(stopped) {
+      return;
+    }
+    stopped = true;
+    if (eventHandlingThread != null) {
+      eventHandlingThread.interrupt();
+      try {
+        eventHandlingThread.join();
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted Exception while stopping");
+      }
+    }
+
+    // stop all the components
+    super.stop();
+
+    LOG.info("AsyncDispatcher stopped:" + id);
+  }
+
+  @SuppressWarnings("unchecked")
+  protected void dispatch(Event event) {
+    //all events go thru this loop
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Dispatching the event " + event.getClass().getName() + "."
+          + event.toString());
+    }
+    Class<? extends Enum> type = event.getType().getDeclaringClass();
+
+    try{
+      EventHandler handler = eventDispatchers.get(type);
+      if(handler != null) {
+        handler.handle(event);
+      } else {
+        throw new Exception("No handler for registered for " + type);
+      }
+    } catch (Throwable t) {
+      //TODO Maybe log the state of the queue
+      LOG.fatal("Error in dispatcher thread:" + event.getType(), t);
+      if (exitOnDispatchException && (ShutdownHookManager.get().isShutdownInProgress()) == false) {
+        LOG.info("Exiting, bye..");
+        System.exit(-1);
+      }
+    } finally {
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void register(Class<? extends Enum> eventType,
+                       EventHandler handler) {
+    /* check to see if we have a listener registered */
+    EventHandler<Event> registeredHandler = (EventHandler<Event>)
+        eventDispatchers.get(eventType);
+    LOG.debug("Registering " + eventType + " for " + handler.getClass());
+    if (registeredHandler == null) {
+      eventDispatchers.put(eventType, handler);
+    } else if (!(registeredHandler instanceof MultiListenerHandler)){
+      /* for multiple listeners of an event add the multiple listener handler */
+      MultiListenerHandler multiHandler = new MultiListenerHandler();
+      multiHandler.addHandler(registeredHandler);
+      multiHandler.addHandler(handler);
+      eventDispatchers.put(eventType, multiHandler);
+    } else {
+      /* already a multilistener, just add to it */
+      MultiListenerHandler multiHandler
+          = (MultiListenerHandler) registeredHandler;
+      multiHandler.addHandler(handler);
+    }
+  }
+
+  @Override
+  public EventHandler getEventHandler() {
+    return new GenericEventHandler();
+  }
+
+  class GenericEventHandler implements EventHandler<Event> {
+    public void handle(Event event) {
+      /* all this method does is enqueue all the events onto the queue */
+      int qSize = eventQueue.size();
+      if (qSize !=0 && qSize %1000 == 0) {
+        LOG.info("Size of event-queue is " + qSize);
+      }
+      int remCapacity = eventQueue.remainingCapacity();
+      if (remCapacity < 1000) {
+        LOG.warn("Very low remaining capacity in the event-queue: "
+            + remCapacity);
+      }
+      try {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug(id + ",add event:" +
+              event.getType() + "," + event + "," +
+              (eventHandlingThread == null ? "null" : eventHandlingThread.isAlive()));
+        }
+        eventQueue.put(event);
+      } catch (InterruptedException e) {
+        if (!stopped) {
+          LOG.warn("AsyncDispatcher thread interrupted", e);
+        }
+        throw new YarnRuntimeException(e);
+      }
+    }
+  }
+
+  /**
+   * Multiplexing an event. Sending it to different handlers that
+   * are interested in the event.
+   */
+  static class MultiListenerHandler implements EventHandler<Event> {
+    List<EventHandler<Event>> listofHandlers;
+
+    public MultiListenerHandler() {
+      listofHandlers = new ArrayList<EventHandler<Event>>();
+    }
+
+    @Override
+    public void handle(Event event) {
+      for (EventHandler<Event> handler: listofHandlers) {
+        handler.handle(event);
+      }
+    }
+
+    void addHandler(EventHandler<Event> handler) {
+      listofHandlers.add(handler);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
new file mode 100644
index 0000000..7f1eac6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
@@ -0,0 +1,178 @@
+/**
+ * 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.master;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.rm.TajoWorkerContainer;
+import org.apache.tajo.master.rm.TajoWorkerContainerId;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.RpcConnectionPool;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TajoContainerProxy extends ContainerProxy {
+  public TajoContainerProxy(QueryMasterTask.QueryMasterTaskContext context,
+                            Configuration conf, Container container,
+                            ExecutionBlockId executionBlockId) {
+    super(context, conf, executionBlockId, container);
+  }
+
+  @Override
+  public void launch(ContainerLaunchContext containerLaunchContext) {
+    context.getResourceAllocator().addContainer(containerID, this);
+
+    this.hostName = container.getNodeId().getHost();
+    this.port = ((TajoWorkerContainer)container).getWorkerResource().getPullServerPort();
+    this.state = ContainerState.RUNNING;
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Launch Container:" + executionBlockId + "," + containerID.getId() + "," +
+          container.getId() + "," + container.getNodeId() + ", pullServer=" + port);
+    }
+
+    assignExecutionBlock(executionBlockId, container);
+  }
+
+  /**
+   * It sends a kill RPC request to a corresponding worker.
+   *
+   * @param taskAttemptId The TaskAttemptId to be killed.
+   */
+  public void killTaskAttempt(QueryUnitAttemptId taskAttemptId) {
+    NettyClientBase tajoWorkerRpc = null;
+    try {
+      InetSocketAddress addr = new InetSocketAddress(container.getNodeId().getHost(), container.getNodeId().getPort());
+      tajoWorkerRpc = RpcConnectionPool.getPool(context.getConf()).getConnection(addr, TajoWorkerProtocol.class, true);
+      TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub();
+      tajoWorkerRpcClient.killTaskAttempt(null, taskAttemptId.getProto(), NullCallback.get());
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    } finally {
+      RpcConnectionPool.getPool(context.getConf()).releaseConnection(tajoWorkerRpc);
+    }
+  }
+
+  private void assignExecutionBlock(ExecutionBlockId executionBlockId, Container container) {
+    NettyClientBase tajoWorkerRpc = null;
+    try {
+      InetSocketAddress myAddr= context.getQueryMasterContext().getWorkerContext()
+          .getQueryMasterManagerService().getBindAddr();
+
+      InetSocketAddress addr = new InetSocketAddress(container.getNodeId().getHost(), container.getNodeId().getPort());
+      tajoWorkerRpc = RpcConnectionPool.getPool(context.getConf()).getConnection(addr, TajoWorkerProtocol.class, true);
+      TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub();
+
+      TajoWorkerProtocol.RunExecutionBlockRequestProto request =
+          TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder()
+              .setExecutionBlockId(executionBlockId.toString())
+              .setQueryMasterHost(myAddr.getHostName())
+              .setQueryMasterPort(myAddr.getPort())
+              .setNodeId(container.getNodeId().toString())
+              .setContainerId(container.getId().toString())
+              .setQueryOutputPath(context.getStagingDir().toString())
+              .build();
+
+      tajoWorkerRpcClient.executeExecutionBlock(null, request, NullCallback.get());
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    } finally {
+      RpcConnectionPool.getPool(context.getConf()).releaseConnection(tajoWorkerRpc);
+    }
+  }
+
+  @Override
+  public synchronized void stopContainer() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Release TajoWorker Resource: " + executionBlockId + "," + containerID + ", state:" + this.state);
+    }
+    if(isCompletelyDone()) {
+      LOG.info("Container already stopped:" + containerID);
+      return;
+    }
+    if(this.state == ContainerState.PREP) {
+      this.state = ContainerState.KILLED_BEFORE_LAUNCH;
+    } else {
+      try {
+        TajoWorkerContainer tajoWorkerContainer = ((TajoWorkerContainer)container);
+        releaseWorkerResource(context, executionBlockId, tajoWorkerContainer.getId());
+        context.getResourceAllocator().removeContainer(containerID);
+        this.state = ContainerState.DONE;
+      } catch (Throwable t) {
+        // ignore the cleanup failure
+        String message = "cleanup failed for container "
+            + this.containerID + " : "
+            + StringUtils.stringifyException(t);
+        LOG.warn(message);
+        this.state = ContainerState.DONE;
+        return;
+      }
+    }
+  }
+
+  public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext context,
+                                           ExecutionBlockId executionBlockId,
+                                           ContainerId containerId) throws Exception {
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
+
+    releaseWorkerResource(context, executionBlockId, containerIds);
+  }
+
+  public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext context,
+                                           ExecutionBlockId executionBlockId,
+                                           List<ContainerId> containerIds) throws Exception {
+    List<YarnProtos.ContainerIdProto> containerIdProtos =
+        new ArrayList<YarnProtos.ContainerIdProto>();
+
+    for(ContainerId eachContainerId: containerIds) {
+      containerIdProtos.add(TajoWorkerContainerId.getContainerIdProto(eachContainerId));
+    }
+
+    RpcConnectionPool connPool = RpcConnectionPool.getPool(context.getConf());
+    NettyClientBase tmClient = null;
+    try {
+        tmClient = connPool.getConnection(context.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
+            TajoMasterProtocol.class, true);
+        TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+        masterClientService.releaseWorkerResource(null,
+          TajoMasterProtocol.WorkerResourceReleaseRequest.newBuilder()
+              .setExecutionBlockId(executionBlockId.getProto())
+              .addAllContainerIds(containerIdProtos)
+              .build(),
+          NullCallback.get());
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    } finally {
+      connPool.releaseConnection(tmClient);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
new file mode 100644
index 0000000..9d54bb5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -0,0 +1,579 @@
+/**
+ * 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.master;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.function.Function;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamOptionTypes;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.master.metrics.CatalogMetricsGaugeSet;
+import org.apache.tajo.master.metrics.WorkerResourceMetricsGaugeSet;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.master.rm.WorkerResourceManager;
+import org.apache.tajo.master.session.SessionManager;
+import org.apache.tajo.rpc.RpcChannelFactory;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.StorageManagerFactory;
+import org.apache.tajo.util.ClassUtil;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.metrics.TajoSystemMetrics;
+import org.apache.tajo.webapp.QueryExecutorServlet;
+import org.apache.tajo.webapp.StaticHttpServer;
+
+import java.io.*;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Modifier;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+
+public class TajoMaster extends CompositeService {
+  private static final String METRICS_GROUP_NAME = "tajomaster";
+
+  /** Class Logger */
+  private static final Log LOG = LogFactory.getLog(TajoMaster.class);
+
+  public static final int SHUTDOWN_HOOK_PRIORITY = 30;
+
+  /** rw-r--r-- */
+  @SuppressWarnings("OctalInteger")
+  final public static FsPermission TAJO_ROOT_DIR_PERMISSION = FsPermission.createImmutable((short) 0755);
+  /** rw-r--r-- */
+  @SuppressWarnings("OctalInteger")
+  final public static FsPermission SYSTEM_DIR_PERMISSION = FsPermission.createImmutable((short) 0755);
+  /** rw-r--r-- */
+  final public static FsPermission SYSTEM_RESOURCE_DIR_PERMISSION = FsPermission.createImmutable((short) 0755);
+  /** rw-r--r-- */
+  @SuppressWarnings("OctalInteger")
+  final public static FsPermission WAREHOUSE_DIR_PERMISSION = FsPermission.createImmutable((short) 0755);
+  /** rw-r--r-- */
+  @SuppressWarnings("OctalInteger")
+  final public static FsPermission STAGING_ROOTDIR_PERMISSION = FsPermission.createImmutable((short) 0755);
+  /** rw-r--r-- */
+  @SuppressWarnings("OctalInteger")
+  final public static FsPermission SYSTEM_CONF_FILE_PERMISSION = FsPermission.createImmutable((short) 0755);
+
+
+  private MasterContext context;
+  private TajoConf systemConf;
+  private FileSystem defaultFS;
+  private Clock clock;
+
+  private Path tajoRootPath;
+  private Path wareHousePath;
+
+  private CatalogServer catalogServer;
+  private CatalogService catalog;
+  private AbstractStorageManager storeManager;
+  private GlobalEngine globalEngine;
+  private AsyncDispatcher dispatcher;
+  private TajoMasterClientService tajoMasterClientService;
+  private TajoMasterService tajoMasterService;
+  private SessionManager sessionManager;
+
+  private WorkerResourceManager resourceManager;
+  //Web Server
+  private StaticHttpServer webServer;
+
+  private QueryJobManager queryJobManager;
+
+  private ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+
+  private TajoSystemMetrics systemMetrics;
+
+  public TajoMaster() throws Exception {
+    super(TajoMaster.class.getName());
+  }
+
+  public String getMasterName() {
+    return NetUtils.normalizeInetSocketAddress(tajoMasterService.getBindAddress());
+  }
+
+  public String getVersion() {
+    return TajoConstants.TAJO_VERSION;
+  }
+
+  public TajoMasterClientService getTajoMasterClientService() {
+    return  tajoMasterClientService;
+  }
+
+  @Override
+  public void serviceInit(Configuration _conf) throws Exception {
+    this.systemConf = (TajoConf) _conf;
+
+    context = new MasterContext(systemConf);
+    clock = new SystemClock();
+
+    try {
+      RackResolver.init(systemConf);
+
+      initResourceManager();
+      initWebServer();
+
+      this.dispatcher = new AsyncDispatcher();
+      addIfService(dispatcher);
+
+      // check the system directory and create if they are not created.
+      checkAndInitializeSystemDirectories();
+      this.storeManager = StorageManagerFactory.getStorageManager(systemConf);
+
+      catalogServer = new CatalogServer(initBuiltinFunctions());
+      addIfService(catalogServer);
+      catalog = new LocalCatalogWrapper(catalogServer, systemConf);
+
+      sessionManager = new SessionManager(dispatcher);
+      addIfService(sessionManager);
+
+      globalEngine = new GlobalEngine(context);
+      addIfService(globalEngine);
+
+      queryJobManager = new QueryJobManager(context);
+      addIfService(queryJobManager);
+
+      tajoMasterClientService = new TajoMasterClientService(context);
+      addIfService(tajoMasterClientService);
+
+      tajoMasterService = new TajoMasterService(context);
+      addIfService(tajoMasterService);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw e;
+    }
+
+    super.serviceInit(systemConf);
+    LOG.info("Tajo Master is initialized.");
+  }
+
+  private void initSystemMetrics() {
+    systemMetrics = new TajoSystemMetrics(systemConf, METRICS_GROUP_NAME, getMasterName());
+    systemMetrics.start();
+
+    systemMetrics.register("resource", new WorkerResourceMetricsGaugeSet(context));
+    systemMetrics.register("catalog", new CatalogMetricsGaugeSet(context));
+  }
+
+  private void initResourceManager() throws Exception {
+    Class<WorkerResourceManager>  resourceManagerClass = (Class<WorkerResourceManager>)
+        systemConf.getClass(ConfVars.RESOURCE_MANAGER_CLASS.varname, TajoWorkerResourceManager.class);
+    Constructor<WorkerResourceManager> constructor = resourceManagerClass.getConstructor(MasterContext.class);
+    resourceManager = constructor.newInstance(context);
+    addIfService(resourceManager);
+  }
+
+  private void initWebServer() throws Exception {
+    if (!systemConf.get(CommonTestingUtil.TAJO_TEST, "FALSE").equalsIgnoreCase("TRUE")) {
+      InetSocketAddress address = systemConf.getSocketAddrVar(ConfVars.TAJO_MASTER_INFO_ADDRESS);
+      webServer = StaticHttpServer.getInstance(this ,"admin", address.getHostName(), address.getPort(),
+          true, null, context.getConf(), null);
+      webServer.addServlet("queryServlet", "/query_exec", QueryExecutorServlet.class);
+      webServer.start();
+    }
+  }
+
+  private void checkAndInitializeSystemDirectories() throws IOException {
+    // Get Tajo root dir
+    this.tajoRootPath = TajoConf.getTajoRootDir(systemConf);
+    LOG.info("Tajo Root Directory: " + tajoRootPath);
+
+    // Check and Create Tajo root dir
+    this.defaultFS = tajoRootPath.getFileSystem(systemConf);
+    systemConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultFS.getUri().toString());
+    LOG.info("FileSystem (" + this.defaultFS.getUri() + ") is initialized.");
+    if (!defaultFS.exists(tajoRootPath)) {
+      defaultFS.mkdirs(tajoRootPath, new FsPermission(TAJO_ROOT_DIR_PERMISSION));
+      LOG.info("Tajo Root Directory '" + tajoRootPath + "' is created.");
+    }
+
+    // Check and Create system and system resource dir
+    Path systemPath = TajoConf.getSystemDir(systemConf);
+    if (!defaultFS.exists(systemPath)) {
+      defaultFS.mkdirs(systemPath, new FsPermission(SYSTEM_DIR_PERMISSION));
+      LOG.info("System dir '" + systemPath + "' is created");
+    }
+    Path systemResourcePath = TajoConf.getSystemResourceDir(systemConf);
+    if (!defaultFS.exists(systemResourcePath)) {
+      defaultFS.mkdirs(systemResourcePath, new FsPermission(SYSTEM_RESOURCE_DIR_PERMISSION));
+      LOG.info("System resource dir '" + systemResourcePath + "' is created");
+    }
+
+    // Get Warehouse dir
+    this.wareHousePath = TajoConf.getWarehouseDir(systemConf);
+    LOG.info("Tajo Warehouse dir: " + wareHousePath);
+
+    // Check and Create Warehouse dir
+    if (!defaultFS.exists(wareHousePath)) {
+      defaultFS.mkdirs(wareHousePath, new FsPermission(WAREHOUSE_DIR_PERMISSION));
+      LOG.info("Warehouse dir '" + wareHousePath + "' is created");
+    }
+
+    Path stagingPath = TajoConf.getStagingDir(systemConf);
+    LOG.info("Staging dir: " + wareHousePath);
+    if (!defaultFS.exists(stagingPath)) {
+      defaultFS.mkdirs(stagingPath, new FsPermission(STAGING_ROOTDIR_PERMISSION));
+      LOG.info("Staging dir '" + stagingPath + "' is created");
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public static List<FunctionDesc> initBuiltinFunctions() throws ServiceException {
+    List<FunctionDesc> sqlFuncs = new ArrayList<FunctionDesc>();
+
+    Set<Class> functionClasses = ClassUtil.findClasses(org.apache.tajo.catalog.function.Function.class,
+          "org.apache.tajo.engine.function");
+
+    for (Class eachClass : functionClasses) {
+      if(eachClass.isInterface() || Modifier.isAbstract(eachClass.getModifiers())) {
+        continue;
+      }
+      Function function = null;
+      try {
+        function = (Function)eachClass.newInstance();
+      } catch (Exception e) {
+        LOG.warn(eachClass + " cannot instantiate Function class because of " + e.getMessage());
+        continue;
+      }
+      String functionName = function.getClass().getAnnotation(Description.class).functionName();
+      String[] synonyms = function.getClass().getAnnotation(Description.class).synonyms();
+      String description = function.getClass().getAnnotation(Description.class).description();
+      String detail = function.getClass().getAnnotation(Description.class).detail();
+      String example = function.getClass().getAnnotation(Description.class).example();
+      Type returnType = function.getClass().getAnnotation(Description.class).returnType();
+      ParamTypes[] paramArray = function.getClass().getAnnotation(Description.class).paramTypes();
+
+      String[] allFunctionNames = null;
+      if(synonyms != null && synonyms.length > 0) {
+        allFunctionNames = new String[1 + synonyms.length];
+        allFunctionNames[0] = functionName;
+        System.arraycopy(synonyms, 0, allFunctionNames, 1, synonyms.length);
+      } else {
+        allFunctionNames = new String[]{functionName};
+      }
+
+      for(String eachFunctionName: allFunctionNames) {
+        for (ParamTypes params : paramArray) {
+          ParamOptionTypes[] paramOptionArray;
+          if(params.paramOptionTypes() == null ||
+              params.paramOptionTypes().getClass().getAnnotation(ParamTypes.class) == null) {
+            paramOptionArray = new ParamOptionTypes[0];
+          } else {
+            paramOptionArray = params.paramOptionTypes().getClass().getAnnotation(ParamTypes.class).paramOptionTypes();
+          }
+
+          Type[] paramTypes = params.paramTypes();
+          if (paramOptionArray.length > 0)
+            paramTypes = params.paramTypes().clone();
+
+          for (int i=0; i < paramOptionArray.length + 1; i++) {
+            FunctionDesc functionDesc = new FunctionDesc(eachFunctionName,
+                function.getClass(), function.getFunctionType(),
+                CatalogUtil.newSimpleDataType(returnType),
+                paramTypes.length == 0 ? CatalogUtil.newSimpleDataTypeArray() : CatalogUtil.newSimpleDataTypeArray(paramTypes));
+
+            functionDesc.setDescription(description);
+            functionDesc.setExample(example);
+            functionDesc.setDetail(detail);
+            sqlFuncs.add(functionDesc);
+
+            if (i != paramOptionArray.length) {
+              paramTypes = new Type[paramTypes.length +
+                  paramOptionArray[i].paramOptionTypes().length];
+              System.arraycopy(params.paramTypes(), 0, paramTypes, 0, paramTypes.length);
+              System.arraycopy(paramOptionArray[i].paramOptionTypes(), 0, paramTypes, paramTypes.length,
+                  paramOptionArray[i].paramOptionTypes().length);
+            }
+          }
+        }
+      }
+    }
+
+    return sqlFuncs;
+  }
+
+  public MasterContext getContext() {
+    return this.context;
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    LOG.info("TajoMaster is starting up");
+
+    // check base tablespace and databases
+    checkBaseTBSpaceAndDatabase();
+
+    super.serviceStart();
+
+    // Setting the system global configs
+    systemConf.setSocketAddr(ConfVars.CATALOG_ADDRESS.varname,
+        NetUtils.getConnectAddress(catalogServer.getBindAddress()));
+
+    try {
+      writeSystemConf();
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    initSystemMetrics();
+  }
+
+  private void writeSystemConf() throws IOException {
+    // Storing the system configs
+    Path systemConfPath = TajoConf.getSystemConfPath(systemConf);
+
+    if (!defaultFS.exists(systemConfPath.getParent())) {
+      defaultFS.mkdirs(systemConfPath.getParent());
+    }
+
+    if (defaultFS.exists(systemConfPath)) {
+      defaultFS.delete(systemConfPath, false);
+    }
+
+    FSDataOutputStream out = FileSystem.create(defaultFS, systemConfPath,
+        new FsPermission(SYSTEM_CONF_FILE_PERMISSION));
+    try {
+      systemConf.writeXml(out);
+    } finally {
+      out.close();
+    }
+    defaultFS.setReplication(systemConfPath, (short) systemConf.getIntVar(ConfVars.SYSTEM_CONF_REPLICA_COUNT));
+  }
+
+  private void checkBaseTBSpaceAndDatabase() throws IOException {
+    if (!catalog.existTablespace(DEFAULT_TABLESPACE_NAME)) {
+      catalog.createTablespace(DEFAULT_TABLESPACE_NAME, context.getConf().getVar(ConfVars.WAREHOUSE_DIR));
+    } else {
+      LOG.info(String.format("Default tablespace (%s) is already prepared.", DEFAULT_TABLESPACE_NAME));
+    }
+
+    if (!catalog.existDatabase(DEFAULT_DATABASE_NAME)) {
+      globalEngine.createDatabase(null, DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME, false);
+    } else {
+      LOG.info(String.format("Default database (%s) is already prepared.", DEFAULT_DATABASE_NAME));
+    }
+  }
+
+  @Override
+  public void stop() {
+    if (webServer != null) {
+      try {
+        webServer.stop();
+      } catch (Exception e) {
+        LOG.error(e);
+      }
+    }
+
+    IOUtils.cleanup(LOG, catalogServer);
+
+    if(systemMetrics != null) {
+      systemMetrics.stop();
+    }
+
+    RpcChannelFactory.shutdown();
+
+    super.stop();
+    LOG.info("Tajo Master main thread exiting");
+  }
+
+  public EventHandler getEventHandler() {
+    return dispatcher.getEventHandler();
+  }
+
+  public boolean isMasterRunning() {
+    return getServiceState() == STATE.STARTED;
+  }
+
+  public CatalogService getCatalog() {
+    return this.catalog;
+  }
+
+  public CatalogServer getCatalogServer() {
+    return this.catalogServer;
+  }
+
+  public AbstractStorageManager getStorageManager() {
+    return this.storeManager;
+  }
+
+  public class MasterContext {
+    private final TajoConf conf;
+
+    public MasterContext(TajoConf conf) {
+      this.conf = conf;
+    }
+
+    public TajoConf getConf() {
+      return conf;
+    }
+
+    public Clock getClock() {
+      return clock;
+    }
+
+    public QueryJobManager getQueryJobManager() {
+      return queryJobManager;
+    }
+
+    public WorkerResourceManager getResourceManager() {
+      return resourceManager;
+    }
+
+    public EventHandler getEventHandler() {
+      return dispatcher.getEventHandler();
+    }
+
+    public CatalogService getCatalog() {
+      return catalog;
+    }
+
+    public SessionManager getSessionManager() {
+      return sessionManager;
+    }
+
+    public GlobalEngine getGlobalEngine() {
+      return globalEngine;
+    }
+
+    public AbstractStorageManager getStorageManager() {
+      return storeManager;
+    }
+
+    public TajoMasterService getTajoMasterService() {
+      return tajoMasterService;
+    }
+
+    public TajoSystemMetrics getSystemMetrics() {
+      return systemMetrics;
+    }
+  }
+
+  String getThreadTaskName(long id, String name) {
+    if (name == null) {
+      return Long.toString(id);
+    }
+    return id + " (" + name + ")";
+  }
+
+  public void dumpThread(Writer writer) {
+    PrintWriter stream = new PrintWriter(writer);
+    int STACK_DEPTH = 20;
+    boolean contention = threadBean.isThreadContentionMonitoringEnabled();
+    long[] threadIds = threadBean.getAllThreadIds();
+    stream.println("Process Thread Dump: Tajo Worker");
+    stream.println(threadIds.length + " active threads");
+    for (long tid : threadIds) {
+      ThreadInfo info = threadBean.getThreadInfo(tid, STACK_DEPTH);
+      if (info == null) {
+        stream.println("  Inactive");
+        continue;
+      }
+      stream.println("Thread " + getThreadTaskName(info.getThreadId(), info.getThreadName()) + ":");
+      Thread.State state = info.getThreadState();
+      stream.println("  State: " + state + ", Blocked count: " + info.getBlockedCount() +
+          ", Waited count: " + info.getWaitedCount());
+      if (contention) {
+        stream.println("  Blocked time: " + info.getBlockedTime() + ", Waited time: " + info.getWaitedTime());
+      }
+      if (state == Thread.State.WAITING) {
+        stream.println("  Waiting on " + info.getLockName());
+      } else if (state == Thread.State.BLOCKED) {
+        stream.println("  Blocked on " + info.getLockName() +
+            ", Blocked by " + getThreadTaskName(info.getLockOwnerId(), info.getLockOwnerName()));
+      }
+      stream.println("  Stack:");
+      for (StackTraceElement frame : info.getStackTrace()) {
+        stream.println("    " + frame.toString());
+      }
+      stream.println("");
+    }
+  }
+
+  public static List<File> getMountPath() throws Exception {
+    BufferedReader mountOutput = null;
+    try {
+      Process mountProcess = Runtime.getRuntime ().exec("mount");
+      mountOutput = new BufferedReader(new InputStreamReader(mountProcess.getInputStream()));
+      List<File> mountPaths = new ArrayList<File>();
+      while (true) {
+        String line = mountOutput.readLine();
+        if (line == null) {
+          break;
+        }
+
+        int indexStart = line.indexOf(" on /");
+        int indexEnd = line.indexOf(" ", indexStart + 4);
+
+        mountPaths.add(new File(line.substring (indexStart + 4, indexEnd)));
+      }
+      return mountPaths;
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    } finally {
+      if(mountOutput != null) {
+        mountOutput.close();
+      }
+    }
+  }
+  public static void main(String[] args) throws Exception {
+    StringUtils.startupShutdownMessage(TajoMaster.class, args, LOG);
+
+    try {
+      TajoMaster master = new TajoMaster();
+      ShutdownHookManager.get().addShutdownHook(new CompositeServiceShutdownHook(master), SHUTDOWN_HOOK_PRIORITY);
+      TajoConf conf = new TajoConf(new YarnConfiguration());
+      master.init(conf);
+      master.start();
+    } catch (Throwable t) {
+      LOG.fatal("Error starting TajoMaster", t);
+      System.exit(-1);
+    }
+  }
+}


[36/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java
new file mode 100644
index 0000000..1ee0878
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java
@@ -0,0 +1,274 @@
+/**
+ * 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.planner;
+
+import com.google.common.collect.ObjectArrays;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.util.TUtil;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.Stack;
+
+public class PreLogicalPlanVerifier extends BaseAlgebraVisitor <PreLogicalPlanVerifier.Context, Expr> {
+  private CatalogService catalog;
+
+  public PreLogicalPlanVerifier(CatalogService catalog) {
+    this.catalog = catalog;
+  }
+
+  public static class Context {
+    Session session;
+    VerificationState state;
+
+    public Context(Session session, VerificationState state) {
+      this.session = session;
+      this.state = state;
+    }
+  }
+
+  public VerificationState verify(Session session, VerificationState state, Expr expr) throws PlanningException {
+    Context context = new Context(session, state);
+    visit(context, new Stack<Expr>(), expr);
+    return context.state;
+  }
+
+  public Expr visitProjection(Context context, Stack<Expr> stack, Projection expr) throws PlanningException {
+    super.visitProjection(context, stack, expr);
+
+    Set<String> names = TUtil.newHashSet();
+    Expr [] distinctValues = null;
+
+    for (NamedExpr namedExpr : expr.getNamedExprs()) {
+
+      if (namedExpr.hasAlias()) {
+        if (names.contains(namedExpr.getAlias())) {
+          context.state.addVerification(String.format("column name \"%s\" specified more than once",
+              namedExpr.getAlias()));
+        } else {
+          names.add(namedExpr.getAlias());
+        }
+      }
+
+      // no two aggregations can have different DISTINCT columns.
+      //
+      // For example, the following query will work
+      // SELECT count(DISTINCT col1) and sum(DISTINCT col1) ..
+      //
+      // But, the following query will not work in this time
+      //
+      // SELECT count(DISTINCT col1) and SUM(DISTINCT col2) ..
+      Set<GeneralSetFunctionExpr> exprs = ExprFinder.finds(namedExpr.getExpr(), OpType.GeneralSetFunction);
+      if (exprs.size() > 0) {
+        for (GeneralSetFunctionExpr setFunction : exprs) {
+          if (distinctValues == null && setFunction.isDistinct()) {
+            distinctValues = setFunction.getParams();
+          } else if (distinctValues != null && setFunction.isDistinct()) {
+            if (!Arrays.equals(distinctValues, setFunction.getParams())) {
+              Expr [] differences = ObjectArrays.concat(distinctValues, setFunction.getParams(), Expr.class);
+              throw new PlanningException("different DISTINCT columns are not supported yet: "
+                  + TUtil.arrayToString(differences));
+            }
+          }
+        }
+      }
+
+      // Currently, avg functions with distinct aggregation are not supported.
+      // This code does not allow users to use avg functions with distinct aggregation.
+      if (distinctValues != null) {
+        for (GeneralSetFunctionExpr setFunction : exprs) {
+          if (setFunction.getSignature().equalsIgnoreCase("avg")) {
+            if (setFunction.isDistinct()) {
+              throw new PlanningException("avg(distinct) function is not supported yet.");
+            } else {
+              throw new PlanningException("avg() function with distinct aggregation functions is not supported yet.");
+            }
+          }
+        }
+      }
+    }
+    return expr;
+  }
+
+  @Override
+  public Expr visitGroupBy(Context context, Stack<Expr> stack, Aggregation expr) throws PlanningException {
+    super.visitGroupBy(context, stack, expr);
+
+    // Enforcer only ordinary grouping set.
+    for (Aggregation.GroupElement groupingElement : expr.getGroupSet()) {
+      if (groupingElement.getType() != Aggregation.GroupType.OrdinaryGroup) {
+        context.state.addVerification(groupingElement.getType() + " is not supported yet");
+      }
+    }
+
+    Projection projection = null;
+    for (Expr parent : stack) {
+      if (parent.getType() == OpType.Projection) {
+        projection = (Projection) parent;
+        break;
+      }
+    }
+
+    if (projection == null) {
+      throw new PlanningException("No Projection");
+    }
+
+    return expr;
+  }
+
+  @Override
+  public Expr visitRelation(Context context, Stack<Expr> stack, Relation expr) throws PlanningException {
+    assertRelationExistence(context, expr.getName());
+    return expr;
+  }
+
+  private boolean assertRelationExistence(Context context, String tableName) {
+    String qualifiedName;
+
+    if (CatalogUtil.isFQTableName(tableName)) {
+      qualifiedName = tableName;
+    } else {
+      qualifiedName = CatalogUtil.buildFQName(context.session.getCurrentDatabase(), tableName);
+    }
+
+    if (!catalog.existsTable(qualifiedName)) {
+      context.state.addVerification(String.format("relation \"%s\" does not exist", qualifiedName));
+      return false;
+    }
+    return true;
+  }
+
+  private boolean assertRelationNoExistence(Context context, String tableName) {
+    String qualifiedName;
+
+    if (CatalogUtil.isFQTableName(tableName)) {
+      qualifiedName = tableName;
+    } else {
+      qualifiedName = CatalogUtil.buildFQName(context.session.getCurrentDatabase(), tableName);
+    }
+    if (catalog.existsTable(qualifiedName)) {
+      context.state.addVerification(String.format("relation \"%s\" already exists", qualifiedName));
+      return false;
+    }
+    return true;
+  }
+
+  private boolean assertUnsupportedStoreType(VerificationState state, String name) {
+    if (name != null && name.equals(CatalogProtos.StoreType.RAW.name())) {
+      state.addVerification(String.format("Unsupported store type :%s", name));
+      return false;
+    }
+    return true;
+  }
+
+  private boolean assertDatabaseExistence(VerificationState state, String name) {
+    if (!catalog.existDatabase(name)) {
+      state.addVerification(String.format("database \"%s\" does not exist", name));
+      return false;
+    }
+    return true;
+  }
+
+  private boolean assertDatabaseNoExistence(VerificationState state, String name) {
+    if (catalog.existDatabase(name)) {
+      state.addVerification(String.format("database \"%s\" already exists", name));
+      return false;
+    }
+    return true;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Data Definition Language Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+
+  @Override
+  public Expr visitCreateDatabase(Context context, Stack<Expr> stack, CreateDatabase expr)
+      throws PlanningException {
+    super.visitCreateDatabase(context, stack, expr);
+    if (!expr.isIfNotExists()) {
+      assertDatabaseNoExistence(context.state, expr.getDatabaseName());
+    }
+    return expr;
+  }
+
+  @Override
+  public Expr visitDropDatabase(Context context, Stack<Expr> stack, DropDatabase expr) throws PlanningException {
+    super.visitDropDatabase(context, stack, expr);
+    if (!expr.isIfExists()) {
+      assertDatabaseExistence(context.state, expr.getDatabaseName());
+    }
+    return expr;
+  }
+
+  @Override
+  public Expr visitCreateTable(Context context, Stack<Expr> stack, CreateTable expr) throws PlanningException {
+    super.visitCreateTable(context, stack, expr);
+    if (!expr.isIfNotExists()) {
+      assertRelationNoExistence(context, expr.getTableName());
+    }
+    assertUnsupportedStoreType(context.state, expr.getStorageType());
+    return expr;
+  }
+
+  @Override
+  public Expr visitDropTable(Context context, Stack<Expr> stack, DropTable expr) throws PlanningException {
+    super.visitDropTable(context, stack, expr);
+    if (!expr.isIfExists()) {
+      assertRelationExistence(context, expr.getTableName());
+    }
+    return expr;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Insert or Update Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public Expr visitInsert(Context context, Stack<Expr> stack, Insert expr) throws PlanningException {
+    Expr child = super.visitInsert(context, stack, expr);
+
+    if (!expr.isOverwrite()) {
+      context.state.addVerification("INSERT INTO statement is not supported yet.");
+    }
+
+    if (expr.hasTableName()) {
+      assertRelationExistence(context, expr.getTableName());
+    }
+
+    if (child != null && child.getType() == OpType.Projection) {
+      if (expr.hasTargetColumns()) {
+        Projection projection = (Projection) child;
+        int projectColumnNum = projection.getNamedExprs().length;
+        int targetColumnNum = expr.getTargetColumns().length;
+
+        if (targetColumnNum > projectColumnNum)  {
+          context.state.addVerification("INSERT has more target columns than expressions");
+        } else if (targetColumnNum < projectColumnNum) {
+          context.state.addVerification("INSERT has more expressions than target columns");
+        }
+      }
+    }
+
+    return expr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java
new file mode 100644
index 0000000..161d39b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java
@@ -0,0 +1,51 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.storage.Tuple;
+
+public class Projector {
+  private final Schema inSchema;
+
+  // for projection
+  private final int targetNum;
+  private final EvalNode[] evals;
+
+  public Projector(Schema inSchema, Schema outSchema, Target [] targets) {
+    this.inSchema = inSchema;
+    if (targets == null) {
+      targets = PlannerUtil.schemaToTargets(outSchema);
+    }
+    this.targetNum = targets.length;
+    evals = new EvalNode[targetNum];
+    for (int i = 0; i < targetNum; i++) {
+      evals[i] = targets[i].getEvalTree();
+    }
+  }
+
+  public void eval(Tuple in, Tuple out) {
+    if (targetNum > 0) {
+      for (int i = 0; i < evals.length; i++) {
+        out.put(i, evals[i].eval(inSchema, in));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java
new file mode 100644
index 0000000..a3522c7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java
@@ -0,0 +1,187 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleRange;
+
+import java.math.BigDecimal;
+
+public abstract class RangePartitionAlgorithm {
+  protected SortSpec [] sortSpecs;
+  protected TupleRange range;
+  protected final BigDecimal totalCard;
+  /** true if the end of the range is inclusive. Otherwise, it should be false. */
+  protected final boolean inclusive;
+
+  /**
+   *
+   * @param sortSpecs The array of sort keys
+   * @param totalRange The total range to be partition
+   * @param inclusive true if the end of the range is inclusive. Otherwise, false.
+   */
+  public RangePartitionAlgorithm(SortSpec [] sortSpecs, TupleRange totalRange, boolean inclusive) {
+    this.sortSpecs = sortSpecs;
+    this.range = totalRange;
+    this.inclusive = inclusive;
+    this.totalCard = computeCardinalityForAllColumns(sortSpecs, totalRange, inclusive);
+  }
+
+  /**
+   * It computes the value cardinality of a tuple range.
+   *
+   * @param dataType
+   * @param start
+   * @param end
+   * @return
+   */
+  public static BigDecimal computeCardinality(DataType dataType, Datum start, Datum end,
+                                              boolean inclusive, boolean isAscending) {
+    BigDecimal columnCard;
+
+    switch (dataType.getType()) {
+      case BOOLEAN:
+        columnCard = new BigDecimal(2);
+        break;
+      case CHAR:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asChar() - start.asChar());
+        } else {
+          columnCard = new BigDecimal(start.asChar() - end.asChar());
+        }
+        break;
+      case BIT:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asByte() - start.asByte());
+        } else {
+          columnCard = new BigDecimal(start.asByte() - end.asByte());
+        }
+        break;
+      case INT2:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt2() - start.asInt2());
+        } else {
+          columnCard = new BigDecimal(start.asInt2() - end.asInt2());
+        }
+        break;
+      case INT4:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt4() - start.asInt4());
+        } else {
+          columnCard = new BigDecimal(start.asInt4() - end.asInt4());
+        }
+        break;
+      case INT8:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt8() - start.asInt8());
+        } else {
+          columnCard = new BigDecimal(start.asInt8() - end.asInt8());
+        }
+        break;
+      case FLOAT4:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt4() - start.asInt4());
+        } else {
+          columnCard = new BigDecimal(start.asInt4() - end.asInt4());
+        }
+        break;
+      case FLOAT8:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt8() - start.asInt8());
+        } else {
+          columnCard = new BigDecimal(start.asInt8() - end.asInt8());
+        }
+        break;
+      case TEXT:
+        final char textStart =  (start instanceof NullDatum || start.size() == 0) ? '0' : start.asChars().charAt(0);
+        final char textEnd = (end instanceof NullDatum || start.size() == 0) ? '0' : end.asChars().charAt(0);
+        if (isAscending) {
+          columnCard = new BigDecimal(textEnd - textStart);
+        } else {
+          columnCard = new BigDecimal(textStart - textEnd);
+        }
+        break;
+      case DATE:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt4() - start.asInt4());
+        } else {
+          columnCard = new BigDecimal(start.asInt4() - end.asInt4());
+        }
+        break;
+      case TIME:
+      case TIMESTAMP:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt8() - start.asInt8());
+        } else {
+          columnCard = new BigDecimal(start.asInt8() - end.asInt8());
+        }
+        break;
+      case INET4:
+        if (isAscending) {
+          columnCard = new BigDecimal(end.asInt4() - start.asInt4());
+        } else {
+          columnCard = new BigDecimal(start.asInt4() - end.asInt4());
+        }
+        break;
+      default:
+        throw new UnsupportedOperationException(dataType + " is not supported yet");
+    }
+
+    return inclusive ? columnCard.add(new BigDecimal(1)).abs() : columnCard.abs();
+  }
+
+  /**
+   * It computes the value cardinality of a tuple range.
+   * @return
+   */
+  public static BigDecimal computeCardinalityForAllColumns(SortSpec[] sortSpecs, TupleRange range, boolean inclusive) {
+    Tuple start = range.getStart();
+    Tuple end = range.getEnd();
+    Column col;
+
+    BigDecimal cardinality = new BigDecimal(1);
+    BigDecimal columnCard;
+    for (int i = 0; i < sortSpecs.length; i++) {
+      columnCard = computeCardinality(sortSpecs[i].getSortKey().getDataType(), start.get(i), end.get(i), inclusive,
+          sortSpecs[i].isAscending());
+
+      if (new BigDecimal(0).compareTo(columnCard) < 0) {
+        cardinality = cardinality.multiply(columnCard);
+      }
+    }
+
+    return cardinality;
+  }
+
+  public BigDecimal getTotalCardinality() {
+    return totalCard;
+  }
+
+  /**
+   *
+   * @param partNum the number of desired partitions, but it may return the less partitions.
+   * @return the end of intermediate ranges are exclusive, and the end of final range is inclusive.
+   */
+  public abstract TupleRange[] partition(int partNum);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java
new file mode 100644
index 0000000..bae6e4a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java
@@ -0,0 +1,215 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.algebra.*;
+
+import java.util.Stack;
+
+/**
+ * <code>SimpleAlgebraVisitor</code> provides a simple and fewer visit methods. It makes building concrete class easier.
+ */
+public abstract class SimpleAlgebraVisitor<CONTEXT, RESULT> extends BaseAlgebraVisitor<CONTEXT, RESULT> {
+
+  public RESULT visit(CONTEXT ctx, Stack<Expr> stack, Expr expr) throws PlanningException {
+    RESULT result = null;
+    if (expr instanceof UnaryOperator) {
+      preHook(ctx, stack, expr);
+      result = visitUnaryOperator(ctx, stack, (UnaryOperator) expr);
+      postHook(ctx, stack, expr, result);
+    } else if (expr instanceof BinaryOperator) {
+      preHook(ctx, stack, expr);
+      result = visitBinaryOperator(ctx, stack, (BinaryOperator) expr);
+      postHook(ctx, stack, expr, result);
+    } else {
+      result = super.visit(ctx, stack, expr);
+    }
+
+    return result;
+  }
+
+  public RESULT visitUnaryOperator(CONTEXT ctx, Stack<Expr> stack, UnaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    RESULT result = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return result;
+  }
+
+  public RESULT visitBinaryOperator(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    visit(ctx, stack, expr.getLeft());
+    RESULT result = visit(ctx, stack, expr.getRight());
+    stack.pop();
+    return result;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Relational Operator Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitProjection(CONTEXT ctx, Stack<Expr> stack, Projection expr) throws PlanningException {
+    return super.visitProjection(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitLimit(CONTEXT ctx, Stack<Expr> stack, Limit expr) throws PlanningException {
+    return super.visitLimit(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitSort(CONTEXT ctx, Stack<Expr> stack, Sort expr) throws PlanningException {
+    return super.visitSort(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitHaving(CONTEXT ctx, Stack<Expr> stack, Having expr) throws PlanningException {
+    return super.visitHaving(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitGroupBy(CONTEXT ctx, Stack<Expr> stack, Aggregation expr) throws PlanningException {
+    return super.visitGroupBy(ctx, stack, expr);
+  }
+
+  public RESULT visitFilter(CONTEXT ctx, Stack<Expr> stack, Selection expr) throws PlanningException {
+    return super.visitFilter(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitJoin(CONTEXT ctx, Stack<Expr> stack, Join expr) throws PlanningException {
+    return super.visitJoin(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitTableSubQuery(CONTEXT ctx, Stack<Expr> stack, TablePrimarySubQuery expr) throws PlanningException {
+    return super.visitTableSubQuery(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitRelationList(CONTEXT ctx, Stack<Expr> stack, RelationList expr) throws PlanningException {
+    return super.visitRelationList(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Data Definition Language Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitCreateTable(CONTEXT ctx, Stack<Expr> stack, CreateTable expr) throws PlanningException {
+    return super.visitCreateTable(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitDropTable(CONTEXT ctx, Stack<Expr> stack, DropTable expr) throws PlanningException {
+    return super.visitDropTable(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitAlterTable(CONTEXT ctx, Stack<Expr> stack, AlterTable expr) throws PlanningException {
+    return super.visitAlterTable(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Insert or Update Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  @Override
+  public RESULT visitInsert(CONTEXT ctx, Stack<Expr> stack, Insert expr) throws PlanningException {
+    return super.visitInsert(ctx, stack, expr);
+  }
+
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Other Predicates Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitBetween(CONTEXT ctx, Stack<Expr> stack, BetweenPredicate expr) throws PlanningException {
+    return super.visitBetween(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitCaseWhen(CONTEXT ctx, Stack<Expr> stack, CaseWhenPredicate expr) throws PlanningException {
+    return super.visitCaseWhen(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitValueListExpr(CONTEXT ctx, Stack<Expr> stack, ValueListExpr expr) throws PlanningException {
+    return super.visitValueListExpr(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Other Expressions
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitFunction(CONTEXT ctx, Stack<Expr> stack, FunctionExpr expr) throws PlanningException {
+    return super.visitFunction(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // General Set Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitCountRowsFunction(CONTEXT ctx, Stack<Expr> stack, CountRowsFunctionExpr expr)
+      throws PlanningException {
+    return super.visitCountRowsFunction(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitGeneralSetFunction(CONTEXT ctx, Stack<Expr> stack, GeneralSetFunctionExpr expr)
+      throws PlanningException {
+    return super.visitGeneralSetFunction(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Literal Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitDataType(CONTEXT ctx, Stack<Expr> stack, DataTypeExpr expr) throws PlanningException {
+    return super.visitDataType(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitCastExpr(CONTEXT ctx, Stack<Expr> stack, CastExpr expr) throws PlanningException {
+    return super.visitCastExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitLiteral(CONTEXT ctx, Stack<Expr> stack, LiteralValue expr) throws PlanningException {
+    return super.visitLiteral(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitNullLiteral(CONTEXT ctx, Stack<Expr> stack, NullLiteral expr) throws PlanningException {
+    return super.visitNullLiteral(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitTimestampLiteral(CONTEXT ctx, Stack<Expr> stack, TimestampLiteral expr) throws PlanningException {
+    return super.visitTimestampLiteral(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitTimeLiteral(CONTEXT ctx, Stack<Expr> stack, TimeLiteral expr) throws PlanningException {
+    return super.visitTimeLiteral(ctx, stack, expr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/Target.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/Target.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/Target.java
new file mode 100644
index 0000000..6a16d3c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/Target.java
@@ -0,0 +1,129 @@
+/**
+ * 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.planner;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.eval.FieldEval;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.util.TUtil;
+
+/**
+ * A Target contains how to evaluate an expression and its alias name.
+ */
+public class Target implements Cloneable, GsonObject {
+  @Expose private EvalNode expr;
+  @Expose private Column column;
+  @Expose private String alias = null;
+
+  public Target(FieldEval fieldEval) {
+    this.expr = fieldEval;
+    this.column = fieldEval.getColumnRef();
+  }
+
+  public Target(final EvalNode eval, final String alias) {
+    this.expr = eval;
+    // force lower case
+    String normalized = alias;
+
+    // If an expr is a column reference and its alias is equivalent to column name, ignore a given alias.
+    if (eval instanceof FieldEval && eval.getName().equals(normalized)) {
+      column = ((FieldEval) eval).getColumnRef();
+    } else {
+      column = new Column(normalized, eval.getValueType());
+      setAlias(alias);
+    }
+  }
+
+  public String getCanonicalName() {
+    return !hasAlias() ? column.getQualifiedName() : alias;
+  }
+
+  public final void setExpr(EvalNode expr) {
+    this.expr = expr;
+  }
+
+  public final void setAlias(String alias) {
+    this.alias = alias;
+    this.column = new Column(alias, expr.getValueType());
+  }
+
+  public final String getAlias() {
+    return alias;
+  }
+
+  public final boolean hasAlias() {
+    return alias != null;
+  }
+
+  public DataType getDataType() {
+    return column.getDataType();
+  }
+
+  public <T extends EvalNode> T getEvalTree() {
+    return (T) this.expr;
+  }
+
+  public Column getNamedColumn() {
+    return this.column;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder(expr.toString());
+    if(hasAlias()) {
+      sb.append(" as ").append(alias);
+    }
+    return sb.toString();
+  }
+
+  public boolean equals(Object obj) {
+    if(obj instanceof Target) {
+      Target other = (Target) obj;
+
+      boolean b1 = expr.equals(other.expr);
+      boolean b2 = column.equals(other.column);
+      boolean b3 = TUtil.checkEquals(alias, other.alias);
+
+      return b1 && b2 && b3;
+    } else {
+      return false;
+    }
+  }
+
+  public int hashCode() {
+    return this.expr.getName().hashCode();
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    Target target = (Target) super.clone();
+    target.expr = (EvalNode) expr.clone();
+    target.column = column;
+    target.alias = alias != null ? alias : null;
+
+    return target;
+  }
+
+  public String toJson() {
+    return CoreGsonHelper.toJson(this, Target.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java
new file mode 100644
index 0000000..f6922ed
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java
@@ -0,0 +1,451 @@
+/**
+ * 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.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.exception.RangeOverflowException;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.util.Bytes;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.List;
+
+
+public class UniformRangePartition extends RangePartitionAlgorithm {
+  private int variableId;
+  private BigDecimal[] cardForEachDigit;
+  private BigDecimal[] colCards;
+
+  /**
+   *
+   * @param totalRange
+   * @param sortSpecs The description of sort keys
+   * @param inclusive true if the end of the range is inclusive
+   */
+  public UniformRangePartition(TupleRange totalRange, SortSpec[] sortSpecs, boolean inclusive) {
+    super(sortSpecs, totalRange, inclusive);
+    colCards = new BigDecimal[sortSpecs.length];
+    for (int i = 0; i < sortSpecs.length; i++) {
+      colCards[i] = computeCardinality(sortSpecs[i].getSortKey().getDataType(), totalRange.getStart().get(i),
+          totalRange.getEnd().get(i), inclusive, sortSpecs[i].isAscending());
+    }
+
+    cardForEachDigit = new BigDecimal[colCards.length];
+    for (int i = 0; i < colCards.length ; i++) {
+      if (i == 0) {
+        cardForEachDigit[i] = colCards[i];
+      } else {
+        cardForEachDigit[i] = cardForEachDigit[i - 1].multiply(colCards[i]);
+      }
+    }
+  }
+
+  public UniformRangePartition(TupleRange range, SortSpec [] sortSpecs) {
+    this(range, sortSpecs, true);
+  }
+
+  @Override
+  public TupleRange[] partition(int partNum) {
+    Preconditions.checkArgument(partNum > 0,
+        "The number of partitions must be positive, but the given number: "
+            + partNum);
+    Preconditions.checkArgument(totalCard.compareTo(new BigDecimal(partNum)) >= 0,
+        "the number of partition cannot exceed total cardinality (" + totalCard + ")");
+
+    int varId;
+    for (varId = 0; varId < cardForEachDigit.length; varId++) {
+      if (cardForEachDigit[varId].compareTo(new BigDecimal(partNum)) >= 0)
+        break;
+    }
+    this.variableId = varId;
+
+    BigDecimal [] reverseCardsForDigit = new BigDecimal[variableId+1];
+    for (int i = variableId; i >= 0; i--) {
+      if (i == variableId) {
+        reverseCardsForDigit[i] = colCards[i];
+      } else {
+        reverseCardsForDigit[i] = reverseCardsForDigit[i+1].multiply(colCards[i]);
+      }
+    }
+
+    List<TupleRange> ranges = Lists.newArrayList();
+    BigDecimal term = reverseCardsForDigit[0].divide(
+        new BigDecimal(partNum), RoundingMode.CEILING);
+    BigDecimal reminder = reverseCardsForDigit[0];
+    Tuple last = range.getStart();
+    while(reminder.compareTo(new BigDecimal(0)) > 0) {
+      if (reminder.compareTo(term) <= 0) { // final one is inclusive
+        ranges.add(new TupleRange(sortSpecs, last, range.getEnd()));
+      } else {
+        Tuple next = increment(last, term.longValue(), variableId);
+        ranges.add(new TupleRange(sortSpecs, last, next));
+      }
+      last = ranges.get(ranges.size() - 1).getEnd();
+      reminder = reminder.subtract(term);
+    }
+
+    return ranges.toArray(new TupleRange[ranges.size()]);
+  }
+
+  /**
+  *  Check whether an overflow occurs or not.
+   *
+   * @param colId The column id to be checked
+   * @param last
+   * @param inc
+   * @param sortSpecs
+   * @return
+   */
+  public boolean isOverflow(int colId, Datum last, BigDecimal inc, SortSpec [] sortSpecs) {
+    Column column = sortSpecs[colId].getSortKey();
+    BigDecimal candidate;
+    boolean overflow = false;
+
+    switch (column.getDataType().getType()) {
+      case BIT: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal(last.asByte()));
+          return new BigDecimal(range.getEnd().get(colId).asByte()).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal(last.asByte()).subtract(inc);
+          return candidate.compareTo(new BigDecimal(range.getEnd().get(colId).asByte())) < 0;
+        }
+      }
+      case CHAR: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal((int)last.asChar()));
+          return new BigDecimal((int)range.getEnd().get(colId).asChar()).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal((int)last.asChar()).subtract(inc);
+          return candidate.compareTo(new BigDecimal((int)range.getEnd().get(colId).asChar())) < 0;
+        }
+      }
+      case INT2: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal(last.asInt2()));
+          return new BigDecimal(range.getEnd().get(colId).asInt2()).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal(last.asInt2()).subtract(inc);
+          return candidate.compareTo(new BigDecimal(range.getEnd().get(colId).asInt2())) < 0;
+        }
+      }
+      case DATE:
+      case INT4: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal(last.asInt4()));
+          return new BigDecimal(range.getEnd().get(colId).asInt4()).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal(last.asInt4()).subtract(inc);
+          return candidate.compareTo(new BigDecimal(range.getEnd().get(colId).asInt4())) < 0;
+        }
+      }
+      case TIME:
+      case TIMESTAMP:
+      case INT8: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal(last.asInt8()));
+          return new BigDecimal(range.getEnd().get(colId).asInt8()).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal(last.asInt8()).subtract(inc);
+          return candidate.compareTo(new BigDecimal(range.getEnd().get(colId).asInt8())) < 0;
+        }
+      }
+      case FLOAT4: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal(last.asFloat4()));
+          return new BigDecimal(range.getEnd().get(colId).asFloat4()).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal(last.asFloat4()).subtract(inc);
+          return candidate.compareTo(new BigDecimal(range.getEnd().get(colId).asFloat4())) < 0;
+        }
+      }
+      case FLOAT8: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal(last.asFloat8()));
+          return new BigDecimal(range.getEnd().get(colId).asFloat8()).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal(last.asFloat8()).subtract(inc);
+          return candidate.compareTo(new BigDecimal(range.getEnd().get(colId).asFloat8())) < 0;
+        }
+
+      }
+      case TEXT: {
+        if (sortSpecs[colId].isAscending()) {
+          candidate = inc.add(new BigDecimal((int)(last instanceof NullDatum ? '0' : last.asChars().charAt(0))));
+          return new BigDecimal(range.getEnd().get(colId).asChars().charAt(0)).compareTo(candidate) < 0;
+        } else {
+          candidate = new BigDecimal((int)(last.asChars().charAt(0))).subtract(inc);
+          return candidate.compareTo(new BigDecimal(range.getEnd().get(colId).asChars().charAt(0))) < 0;
+        }
+      }
+      case INET4: {
+        int candidateIntVal;
+        byte[] candidateBytesVal = new byte[4];
+        if (sortSpecs[colId].isAscending()) {
+          candidateIntVal = inc.intValue() + last.asInt4();
+          if (candidateIntVal - inc.intValue() != last.asInt4()) {
+            return true;
+          }
+          Bytes.putInt(candidateBytesVal, 0, candidateIntVal);
+          return Bytes.compareTo(range.getEnd().get(colId).asByteArray(), candidateBytesVal) < 0;
+        } else {
+          candidateIntVal = last.asInt4() - inc.intValue();
+          if (candidateIntVal + inc.intValue() != last.asInt4()) {
+            return true;
+          }
+          Bytes.putInt(candidateBytesVal, 0, candidateIntVal);
+          return Bytes.compareTo(candidateBytesVal, range.getEnd().get(colId).asByteArray()) < 0;
+        }
+      }
+    }
+    return overflow;
+  }
+
+  public long incrementAndGetReminder(int colId, Datum last, long inc) {
+    Column column = sortSpecs[colId].getSortKey();
+    long reminder = 0;
+    switch (column.getDataType().getType()) {
+      case BIT: {
+        long candidate = last.asByte() + inc;
+        byte end = range.getEnd().get(colId).asByte();
+        reminder = candidate - end;
+        break;
+      }
+      case CHAR: {
+        long candidate = last.asChar() + inc;
+        char end = range.getEnd().get(colId).asChar();
+        reminder = candidate - end;
+        break;
+      }
+      case DATE:
+      case INT4: {
+        int candidate = (int) (last.asInt4() + inc);
+        int end = range.getEnd().get(colId).asInt4();
+        reminder = candidate - end;
+        break;
+      }
+      case TIME:
+      case TIMESTAMP:
+      case INT8:
+      case INET4: {
+        long candidate = last.asInt8() + inc;
+        long end = range.getEnd().get(colId).asInt8();
+        reminder = candidate - end;
+        break;
+      }
+      case FLOAT4: {
+        float candidate = last.asFloat4() + inc;
+        float end = range.getEnd().get(colId).asFloat4();
+        reminder = (long) (candidate - end);
+        break;
+      }
+      case FLOAT8: {
+        double candidate = last.asFloat8() + inc;
+        double end = range.getEnd().get(colId).asFloat8();
+        reminder = (long) Math.ceil(candidate - end);
+        break;
+      }
+      case TEXT: {
+        char candidate = ((char)(last.asChars().charAt(0) + inc));
+        char end = range.getEnd().get(colId).asChars().charAt(0);
+        reminder = (char) (candidate - end);
+        break;
+      }
+    }
+
+    // including zero
+    return reminder - 1;
+  }
+
+  /**
+   *
+   * @param last
+   * @param inc
+   * @return
+   */
+  public Tuple increment(final Tuple last, final long inc, final int baseDigit) {
+    BigDecimal [] incs = new BigDecimal[last.size()];
+    boolean [] overflowFlag = new boolean[last.size()];
+    BigDecimal [] result;
+    BigDecimal value = new BigDecimal(inc);
+
+    BigDecimal [] reverseCardsForDigit = new BigDecimal[baseDigit + 1];
+    for (int i = baseDigit; i >= 0; i--) {
+      if (i == baseDigit) {
+        reverseCardsForDigit[i] = colCards[i];
+      } else {
+        reverseCardsForDigit[i] = reverseCardsForDigit[i+1].multiply(colCards[i]);
+      }
+    }
+
+    for (int i = 0; i < baseDigit; i++) {
+      result = value.divideAndRemainder(reverseCardsForDigit[i + 1]);
+      incs[i] = result[0];
+      value = result[1];
+    }
+    int finalId = baseDigit;
+    incs[finalId] = value;
+    for (int i = finalId; i >= 0; i--) {
+      if (isOverflow(i, last.get(i), incs[i], sortSpecs)) {
+        if (i == 0) {
+          throw new RangeOverflowException(range, last, incs[i].longValue());
+        }
+        long rem = incrementAndGetReminder(i, last.get(i), value.longValue());
+        incs[i] = new BigDecimal(rem);
+        incs[i - 1] = incs[i-1].add(new BigDecimal(1));
+        overflowFlag[i] = true;
+      } else {
+        if (i > 0) {
+          incs[i] = value;
+          break;
+        }
+      }
+    }
+
+    for (int i = 0; i < incs.length; i++) {
+      if (incs[i] == null) {
+        incs[i] = new BigDecimal(0);
+      }
+    }
+
+    Tuple end = new VTuple(sortSpecs.length);
+    Column column;
+    for (int i = 0; i < last.size(); i++) {
+      column = sortSpecs[i].getSortKey();
+      switch (column.getDataType().getType()) {
+        case CHAR:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createChar((char) (range.getStart().get(i).asChar() + incs[i].longValue())));
+          } else {
+            end.put(i, DatumFactory.createChar((char) (last.get(i).asChar() + incs[i].longValue())));
+          }
+          break;
+        case BIT:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createBit(
+                (byte) (range.getStart().get(i).asByte() + incs[i].longValue())));
+          } else {
+            end.put(i, DatumFactory.createBit((byte) (last.get(i).asByte() + incs[i].longValue())));
+          }
+          break;
+        case INT2:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createInt2(
+                (short) (range.getStart().get(i).asInt2() + incs[i].longValue())));
+          } else {
+            end.put(i, DatumFactory.createInt2((short) (last.get(i).asInt2() + incs[i].longValue())));
+          }
+          break;
+        case INT4:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createInt4(
+                (int) (range.getStart().get(i).asInt4() + incs[i].longValue())));
+          } else {
+            if (sortSpecs[i].isAscending()) {
+              end.put(i, DatumFactory.createInt4((int) (last.get(i).asInt4() + incs[i].longValue())));
+            } else {
+              end.put(i, DatumFactory.createInt4((int) (last.get(i).asInt4() - incs[i].longValue())));
+            }
+          }
+          break;
+        case INT8:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createInt8(
+                range.getStart().get(i).asInt8() + incs[i].longValue()));
+          } else {
+            end.put(i, DatumFactory.createInt8(last.get(i).asInt8() + incs[i].longValue()));
+          }
+          break;
+        case FLOAT4:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createFloat4(
+                range.getStart().get(i).asFloat4() + incs[i].longValue()));
+          } else {
+            end.put(i, DatumFactory.createFloat4(last.get(i).asFloat4() + incs[i].longValue()));
+          }
+          break;
+        case FLOAT8:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createFloat8(
+                range.getStart().get(i).asFloat8() + incs[i].longValue()));
+          } else {
+            end.put(i, DatumFactory.createFloat8(last.get(i).asFloat8() + incs[i].longValue()));
+          }
+          break;
+        case TEXT:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createText(((char) (range.getStart().get(i).asChars().charAt(0)
+                + incs[i].longValue())) + ""));
+          } else {
+            end.put(i, DatumFactory.createText(
+                ((char) ((last.get(i) instanceof NullDatum ? '0': last.get(i).asChars().charAt(0)) + incs[i].longValue())) + ""));
+          }
+          break;
+        case DATE:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createDate((int) (range.getStart().get(i).asInt4() + incs[i].longValue())));
+          } else {
+            end.put(i, DatumFactory.createDate((int) (last.get(i).asInt4() + incs[i].longValue())));
+          }
+          break;
+        case TIME:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createTime(range.getStart().get(i).asInt8() + incs[i].longValue()));
+          } else {
+            end.put(i, DatumFactory.createTime(last.get(i).asInt8() + incs[i].longValue()));
+          }
+          break;
+        case TIMESTAMP:
+          if (overflowFlag[i]) {
+            end.put(i, DatumFactory.createTimeStampFromMillis(
+                range.getStart().get(i).asInt8() + incs[i].longValue()));
+          } else {
+            end.put(i, DatumFactory.createTimeStampFromMillis(last.get(i).asInt8() + incs[i].longValue()));
+          }
+          break;
+        case INET4:
+          byte[] ipBytes;
+          if (overflowFlag[i]) {
+            ipBytes = range.getStart().get(i).asByteArray();
+            assert ipBytes.length == 4;
+            end.put(i, DatumFactory.createInet4(ipBytes));
+          } else {
+            int lastVal = last.get(i).asInt4() + incs[i].intValue();
+            ipBytes = new byte[4];
+            Bytes.putInt(ipBytes, 0, lastVal);
+            end.put(i, DatumFactory.createInet4(ipBytes));
+          }
+          break;
+        default:
+          throw new UnsupportedOperationException(column.getDataType() + " is not supported yet");
+      }
+    }
+
+    return end;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/VerificationState.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/VerificationState.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/VerificationState.java
new file mode 100644
index 0000000..18882b8
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/VerificationState.java
@@ -0,0 +1,44 @@
+/**
+ * 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.planner;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.util.TUtil;
+
+import java.util.List;
+
+public class VerificationState {
+  private static final Log LOG = LogFactory.getLog(VerificationState.class);
+  List<String> errorMessages = Lists.newArrayList();
+
+  public void addVerification(String error) {
+    LOG.warn(TUtil.getCurrentCodePoint(1) + " causes: " + error);
+    errorMessages.add(error);
+  }
+
+  public boolean verified() {
+    return errorMessages.size() == 0;
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java
new file mode 100644
index 0000000..91190f6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java
@@ -0,0 +1,262 @@
+/**
+ * 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.planner.enforce;
+
+
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.util.TUtil;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.*;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.GroupbyEnforce.GroupbyAlgorithm;
+
+public class Enforcer implements ProtoObject<EnforcerProto> {
+  Map<EnforceType, List<EnforceProperty>> properties;
+  private EnforcerProto proto;
+
+  @SuppressWarnings("unused")
+  public Enforcer() {
+    properties = TUtil.newHashMap();
+  }
+
+  public Enforcer(EnforcerProto proto) {
+    this.proto = proto;
+  }
+
+  private EnforceProperty.Builder newProperty() {
+    return EnforceProperty.newBuilder();
+  }
+
+  private void initProperties() {
+    if (properties == null) {
+      properties = TUtil.newHashMap();
+      for (EnforceProperty property : proto.getPropertiesList()) {
+        TUtil.putToNestedList(properties, property.getType(), property);
+      }
+    }
+  }
+
+  public boolean hasEnforceProperty(EnforceType type) {
+    initProperties();
+    return properties.containsKey(type);
+  }
+
+  public List<EnforceProperty> getEnforceProperties(EnforceType type) {
+    initProperties();
+    return properties.get(type);
+  }
+
+  public void addSortedInput(String tableName, SortSpec[] sortSpecs) {
+    EnforceProperty.Builder builder = newProperty();
+    SortedInputEnforce.Builder enforce = SortedInputEnforce.newBuilder();
+    enforce.setTableName(tableName);
+    for (SortSpec sortSpec : sortSpecs) {
+      enforce.addSortSpecs(sortSpec.getProto());
+    }
+
+    builder.setType(EnforceType.SORTED_INPUT);
+    builder.setSortedInput(enforce.build());
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public void addOutputDistinct() {
+    EnforceProperty.Builder builder = newProperty();
+    OutputDistinctEnforce.Builder enforce = OutputDistinctEnforce.newBuilder();
+
+    builder.setType(EnforceType.OUTPUT_DISTINCT);
+    builder.setOutputDistinct(enforce.build());
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public void enforceJoinAlgorithm(int pid, JoinEnforce.JoinAlgorithm algorithm) {
+    EnforceProperty.Builder builder = newProperty();
+    JoinEnforce.Builder enforce = JoinEnforce.newBuilder();
+    enforce.setPid(pid);
+    enforce.setAlgorithm(algorithm);
+
+    builder.setType(EnforceType.JOIN);
+    builder.setJoin(enforce.build());
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public void enforceSortAggregation(int pid, @Nullable SortSpec[] sortSpecs) {
+    EnforceProperty.Builder builder = newProperty();
+    GroupbyEnforce.Builder enforce = GroupbyEnforce.newBuilder();
+    enforce.setPid(pid);
+    enforce.setAlgorithm(GroupbyAlgorithm.SORT_AGGREGATION);
+    if (sortSpecs != null) {
+      for (SortSpec sortSpec : sortSpecs) {
+        enforce.addSortSpecs(sortSpec.getProto());
+      }
+    }
+
+    builder.setType(EnforceType.GROUP_BY);
+    builder.setGroupby(enforce.build());
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public void enforceHashAggregation(int pid) {
+    EnforceProperty.Builder builder = newProperty();
+    GroupbyEnforce.Builder enforce = GroupbyEnforce.newBuilder();
+    enforce.setPid(pid);
+    enforce.setAlgorithm(GroupbyAlgorithm.HASH_AGGREGATION);
+
+    builder.setType(EnforceType.GROUP_BY);
+    builder.setGroupby(enforce.build());
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public void enforceSortAlgorithm(int pid, SortEnforce.SortAlgorithm algorithm) {
+    EnforceProperty.Builder builder = newProperty();
+    SortEnforce.Builder enforce = SortEnforce.newBuilder();
+    enforce.setPid(pid);
+    enforce.setAlgorithm(algorithm);
+
+    builder.setType(EnforceType.SORT);
+    builder.setSort(enforce.build());
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public void addBroadcast(String tableName) {
+    EnforceProperty.Builder builder = newProperty();
+    BroadcastEnforce.Builder enforce = BroadcastEnforce.newBuilder();
+    enforce.setTableName(tableName);
+
+    builder.setType(EnforceType.BROADCAST);
+    builder.setBroadcast(enforce);
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public void enforceColumnPartitionAlgorithm(int pid, ColumnPartitionAlgorithm algorithm) {
+    EnforceProperty.Builder builder = newProperty();
+    ColumnPartitionEnforcer.Builder enforce = ColumnPartitionEnforcer.newBuilder();
+    enforce.setPid(pid);
+    enforce.setAlgorithm(algorithm);
+
+    builder.setType(EnforceType.COLUMN_PARTITION);
+    builder.setColumnPartition(enforce);
+    TUtil.putToNestedList(properties, builder.getType(), builder.build());
+  }
+
+  public Collection<EnforceProperty> getProperties() {
+    if (proto != null) {
+      return proto.getPropertiesList();
+    } else {
+      List<EnforceProperty> list = TUtil.newList();
+      for (List<EnforceProperty> propertyList : properties.values()) {
+        list.addAll(propertyList);
+      }
+      return list;
+    }
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Enforce ").append(properties.size()).append(" properties: ");
+    boolean first = true;
+    for (EnforceType enforceType : properties.keySet()) {
+      if (first) {
+        first = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append(enforceType);
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public EnforcerProto getProto() {
+    EnforcerProto.Builder builder = EnforcerProto.newBuilder();
+    builder.addAllProperties(getProperties());
+    return builder.build();
+  }
+
+  public static String toString(EnforceProperty property) {
+    StringBuilder sb = new StringBuilder();
+    switch (property.getType()) {
+    case GROUP_BY:
+      GroupbyEnforce groupby = property.getGroupby();
+      sb.append("type=GroupBy,alg=");
+      if (groupby.getAlgorithm() == GroupbyAlgorithm.HASH_AGGREGATION) {
+        sb.append("hash");
+      } else {
+        sb.append("sort");
+        sb.append(",keys=");
+        boolean first = true;
+        for (CatalogProtos.SortSpecProto sortSpec : groupby.getSortSpecsList()) {
+          if (first == true) {
+            first = false;
+          } else {
+            sb.append(", ");
+          }
+          sb.append(sortSpec.getColumn().getName());
+          sb.append(" (").append(sortSpec.getAscending() ? "asc":"desc").append(")");
+        }
+      }
+      break;
+    case BROADCAST:
+      BroadcastEnforce broadcast = property.getBroadcast();
+      sb.append("type=Broadcast, tables=").append(broadcast.getTableName());
+      break;
+    case COLUMN_PARTITION:
+      ColumnPartitionEnforcer columnPartition = property.getColumnPartition();
+      sb.append("type=ColumnPartition, alg=");
+      if (columnPartition.getAlgorithm() == ColumnPartitionAlgorithm.SORT_PARTITION) {
+        sb.append("sort");
+      } else {
+        sb.append("hash");
+      }
+      break;
+    case JOIN:
+      JoinEnforce join = property.getJoin();
+      sb.append("type=Join,alg=");
+      if (join.getAlgorithm() == JoinEnforce.JoinAlgorithm.MERGE_JOIN) {
+        sb.append("merge_join");
+      } else if (join.getAlgorithm() == JoinEnforce.JoinAlgorithm.NESTED_LOOP_JOIN) {
+        sb.append("nested_loop");
+      } else if (join.getAlgorithm() == JoinEnforce.JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN) {
+        sb.append("block_nested_loop");
+      } else if (join.getAlgorithm() == JoinEnforce.JoinAlgorithm.IN_MEMORY_HASH_JOIN) {
+        sb.append("in_memory_hash");
+      }
+      break;
+    case OUTPUT_DISTINCT:
+    case SORT:
+      SortEnforce sort = property.getSort();
+      sb.append("type=Sort,alg=");
+      if (sort.getAlgorithm() == SortEnforce.SortAlgorithm.IN_MEMORY_SORT) {
+        sb.append("in-memory");
+      } else {
+        sb.append("external");
+      }
+      break;
+    case SORTED_INPUT:
+    }
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java
new file mode 100644
index 0000000..b3b5bb0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java
@@ -0,0 +1,200 @@
+/**
+ * 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.planner.global;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.util.TUtil;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.*;
+
+public class DataChannel {
+  private ExecutionBlockId srcId;
+  private ExecutionBlockId targetId;
+  private TransmitType transmitType = TransmitType.PULL_TRANSMIT;
+  private ShuffleType shuffleType;
+  private Integer numOutputs = 1;
+  private Column[] shuffleKeys;
+
+  private Schema schema;
+
+  private StoreType storeType = StoreType.RAW;
+
+  public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId) {
+    this.srcId = srcId;
+    this.targetId = targetId;
+  }
+
+  public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId, ShuffleType shuffleType) {
+    this(srcId, targetId);
+    this.shuffleType = shuffleType;
+  }
+
+  public DataChannel(ExecutionBlock src, ExecutionBlock target, ShuffleType shuffleType, int numOutput) {
+    this(src.getId(), target.getId(), shuffleType, numOutput);
+    setSchema(src.getPlan().getOutSchema());
+  }
+
+  public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId, ShuffleType shuffleType, int numOutputs) {
+    this(srcId, targetId, shuffleType);
+    this.numOutputs = numOutputs;
+  }
+
+  public DataChannel(DataChannelProto proto) {
+    this.srcId = new ExecutionBlockId(proto.getSrcId());
+    this.targetId = new ExecutionBlockId(proto.getTargetId());
+    this.transmitType = proto.getTransmitType();
+    this.shuffleType = proto.getShuffleType();
+    if (proto.hasSchema()) {
+      this.setSchema(new Schema(proto.getSchema()));
+    }
+    if (proto.getShuffleKeysCount() > 0) {
+      shuffleKeys = new Column[proto.getShuffleKeysCount()];
+      for (int i = 0; i < proto.getShuffleKeysCount(); i++) {
+        shuffleKeys[i] = new Column(proto.getShuffleKeys(i));
+      }
+    } else {
+      shuffleKeys = new Column[] {};
+    }
+    if (proto.hasNumOutputs()) {
+      this.numOutputs = proto.getNumOutputs();
+    }
+
+    if (proto.hasStoreType()) {
+      this.storeType = proto.getStoreType();
+    }
+  }
+
+  public ExecutionBlockId getSrcId() {
+    return srcId;
+  }
+
+  public ExecutionBlockId getTargetId() {
+    return targetId;
+  }
+
+  public ShuffleType getShuffleType() {
+    return shuffleType;
+  }
+
+  public TransmitType getTransmitType() {
+    return this.transmitType;
+  }
+
+  public void setTransmitType(TransmitType transmitType) {
+    this.transmitType = transmitType;
+  }
+
+  public void setShuffle(ShuffleType shuffleType, Column[] keys, int numOutputs) {
+    Preconditions.checkArgument(keys.length >= 0, "At least one shuffle key must be specified.");
+    Preconditions.checkArgument(numOutputs > 0, "The number of outputs must be positive: %s", numOutputs);
+
+    this.shuffleType = shuffleType;
+    this.shuffleKeys = keys;
+    this.numOutputs = numOutputs;
+  }
+
+  public void setShuffleType(ShuffleType shuffleType) {
+    this.shuffleType = shuffleType;
+  }
+
+  public boolean hasShuffleKeys() {
+    return shuffleKeys != null;
+  }
+
+  public void setShuffleKeys(Column[] key) {
+    this.shuffleKeys = key;
+  }
+
+  public Column [] getShuffleKeys() {
+    return this.shuffleKeys;
+  }
+
+  public void setShuffleOutputNum(int partNum) {
+    this.numOutputs = partNum;
+  }
+
+  public int getShuffleOutputNum() {
+    return numOutputs;
+  }
+
+  public boolean hasStoreType() {
+    return this.storeType != null;
+  }
+
+  public void setStoreType(StoreType storeType) {
+    this.storeType = storeType;
+  }
+
+  public StoreType getStoreType() {
+    return storeType;
+  }
+
+  public DataChannelProto getProto() {
+    DataChannelProto.Builder builder = DataChannelProto.newBuilder();
+    builder.setSrcId(srcId.getProto());
+    builder.setTargetId(targetId.getProto());
+    if (transmitType != null) {
+      builder.setTransmitType(transmitType);
+    }
+    builder.setShuffleType(shuffleType);
+    if (schema != null) {
+      builder.setSchema(schema.getProto());
+    }
+    if (shuffleKeys != null) {
+      for (Column column : shuffleKeys) {
+        builder.addShuffleKeys(column.getProto());
+      }
+    }
+    if (numOutputs != null) {
+      builder.setNumOutputs(numOutputs);
+    }
+
+    if(storeType != null){
+      builder.setStoreType(storeType);
+    }
+    return builder.build();
+  }
+
+  public void setSchema(Schema schema) {
+    this.schema = SchemaUtil.clone(schema);
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("[").append(srcId.getQueryId()).append("] ");
+    sb.append(srcId.getId()).append(" => ").append(targetId.getId());
+    sb.append(" (type=").append(shuffleType);
+    if (hasShuffleKeys()) {
+      sb.append(", key=");
+      sb.append(TUtil.arrayToString(shuffleKeys));
+      sb.append(", num=").append(numOutputs);
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java
new file mode 100644
index 0000000..7df6b43
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed 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.planner.global;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.*;
+
+import java.util.*;
+
+/**
+ * A distributed execution plan (DEP) is a direct acyclic graph (DAG) of ExecutionBlocks.
+ * An ExecutionBlock is a basic execution unit that could be distributed across a number of nodes.
+ * An ExecutionBlock class contains input information (e.g., child execution blocks or input
+ * tables), and output information (e.g., partition type, partition key, and partition number).
+ * In addition, it includes a logical plan to be executed in each node.
+ */
+public class ExecutionBlock {
+  private ExecutionBlockId executionBlockId;
+  private LogicalNode plan = null;
+  private StoreTableNode store = null;
+  private List<ScanNode> scanlist = new ArrayList<ScanNode>();
+  private Enforcer enforcer = new Enforcer();
+
+  private boolean hasJoinPlan;
+  private boolean hasUnionPlan;
+
+  private Set<String> broadcasted = new HashSet<String>();
+
+  public ExecutionBlock(ExecutionBlockId executionBlockId) {
+    this.executionBlockId = executionBlockId;
+  }
+
+  public ExecutionBlockId getId() {
+    return executionBlockId;
+  }
+
+  public void setPlan(LogicalNode plan) {
+    hasJoinPlan = false;
+    hasUnionPlan = false;
+    this.scanlist.clear();
+    this.plan = plan;
+
+    if (plan == null) {
+      return;
+    }
+
+    LogicalNode node = plan;
+    ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
+    s.add(node);
+    while (!s.isEmpty()) {
+      node = s.remove(s.size()-1);
+      if (node instanceof UnaryNode) {
+        UnaryNode unary = (UnaryNode) node;
+        s.add(s.size(), unary.getChild());
+      } else if (node instanceof BinaryNode) {
+        BinaryNode binary = (BinaryNode) node;
+        if (binary.getType() == NodeType.JOIN) {
+          hasJoinPlan = true;
+        } else if (binary.getType() == NodeType.UNION) {
+          hasUnionPlan = true;
+        }
+        s.add(s.size(), binary.getLeftChild());
+        s.add(s.size(), binary.getRightChild());
+      } else if (node instanceof ScanNode) {
+        scanlist.add((ScanNode)node);
+      } else if (node instanceof TableSubQueryNode) {
+        TableSubQueryNode subQuery = (TableSubQueryNode) node;
+        s.add(s.size(), subQuery.getSubQuery());
+      }
+    }
+  }
+
+
+  public LogicalNode getPlan() {
+    return plan;
+  }
+
+  public Enforcer getEnforcer() {
+    return enforcer;
+  }
+
+  public StoreTableNode getStoreTableNode() {
+    return store;
+  }
+
+  public ScanNode [] getScanNodes() {
+    return this.scanlist.toArray(new ScanNode[scanlist.size()]);
+  }
+
+  public boolean hasJoin() {
+    return hasJoinPlan;
+  }
+
+  public boolean hasUnion() {
+    return hasUnionPlan;
+  }
+
+  public void addBroadcastTable(String tableName) {
+    broadcasted.add(tableName);
+    enforcer.addBroadcast(tableName);
+  }
+
+  public boolean isBroadcastTable(String tableName) {
+    return broadcasted.contains(tableName);
+  }
+
+  public Collection<String> getBroadcastTables() {
+    return broadcasted;
+  }
+
+  public String toString() {
+    return executionBlockId.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlockCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlockCursor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlockCursor.java
new file mode 100644
index 0000000..d4ab068
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlockCursor.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed 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.planner.global;
+
+import java.util.ArrayList;
+import java.util.Stack;
+
+/**
+ * A distributed execution plan (DEP) is a direct acyclic graph (DAG) of ExecutionBlocks.
+ * This class is a pointer to an ExecutionBlock that the query engine should execute.
+ * For each call of nextBlock(), it retrieves a next ExecutionBlock in a postfix order.
+ */
+public class ExecutionBlockCursor {
+  private MasterPlan masterPlan;
+  private ArrayList<ExecutionBlock> orderedBlocks = new ArrayList<ExecutionBlock>();
+  private int cursor = 0;
+
+  public ExecutionBlockCursor(MasterPlan plan) {
+    this.masterPlan = plan;
+    buildOrder(plan.getRoot());
+  }
+
+  public int size() {
+    return orderedBlocks.size();
+  }
+
+  // Add all execution blocks in a depth first and postfix order
+  private void buildOrder(ExecutionBlock current) {
+    Stack<ExecutionBlock> stack = new Stack<ExecutionBlock>();
+    if (!masterPlan.isLeaf(current.getId())) {
+      for (ExecutionBlock execBlock : masterPlan.getChilds(current)) {
+        if (!masterPlan.isLeaf(execBlock)) {
+          buildOrder(execBlock);
+        } else {
+          stack.push(execBlock);
+        }
+      }
+      for (ExecutionBlock execBlock : stack) {
+        buildOrder(execBlock);
+      }
+    }
+    orderedBlocks.add(current);
+  }
+
+  public boolean hasNext() {
+    return cursor < orderedBlocks.size();
+  }
+
+  public ExecutionBlock nextBlock() {
+    return orderedBlocks.get(cursor++);
+  }
+
+  public ExecutionBlock peek() {
+    return orderedBlocks.get(cursor);
+  }
+
+  public ExecutionBlock peek(int skip) {
+    return  orderedBlocks.get(cursor + skip);
+  }
+
+  public void reset() {
+    cursor = 0;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < orderedBlocks.size(); i++) {
+      if (i == (cursor == 0 ? 0 : cursor - 1)) {
+        sb.append("(").append(orderedBlocks.get(i).getId().getId()).append(")");
+      } else {
+        sb.append(orderedBlocks.get(i).getId().getId());
+      }
+
+      if (i < orderedBlocks.size() - 1) {
+        sb.append(",");
+      }
+    }
+
+    return sb.toString();
+  }
+}


[06/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
new file mode 100644
index 0000000..b586f56
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -0,0 +1,1136 @@
+/**
+ * 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.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.function.builtin.SumInt;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.TUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
+public class TestLogicalPlanner {
+  private static TajoTestingCluster util;
+  private static CatalogService catalog;
+  private static SQLAnalyzer sqlAnalyzer;
+  private static LogicalPlanner planner;
+  private static TPCH tpch;
+  private static Session session = LocalTajoTestingUtility.createDummySession();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.startCatalogCluster();
+    catalog = util.getMiniCatalogCluster().getCatalog();
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234");
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+
+    for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+      catalog.createFunction(funcDesc);
+    }
+
+    Schema schema = new Schema();
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("deptname", Type.TEXT);
+    schema2.addColumn("manager", Type.TEXT);
+
+    Schema schema3 = new Schema();
+    schema3.addColumn("deptname", Type.TEXT);
+    schema3.addColumn("score", Type.INT4);
+
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    TableDesc people = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta,
+        CommonTestingUtil.getTestDir());
+    catalog.createTable(people);
+
+    TableDesc student = new TableDesc(
+        CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dept"), schema2, StoreType.CSV, new Options(),
+        CommonTestingUtil.getTestDir());
+    catalog.createTable(student);
+
+    TableDesc score = new TableDesc(
+        CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), schema3, StoreType.CSV, new Options(),
+        CommonTestingUtil.getTestDir());
+    catalog.createTable(score);
+
+    FunctionDesc funcDesc = new FunctionDesc("sumtest", SumInt.class, FunctionType.AGGREGATION,
+        CatalogUtil.newSimpleDataType(Type.INT4),
+        CatalogUtil.newSimpleDataTypeArray(Type.INT4));
+
+
+    // TPC-H Schema for Complex Queries
+    String [] tpchTables = {
+        "part", "supplier", "partsupp", "nation", "region", "lineitem"
+    };
+    tpch = new TPCH();
+    tpch.loadSchemas();
+    tpch.loadOutSchema();
+    for (String table : tpchTables) {
+      TableMeta m = CatalogUtil.newTableMeta(StoreType.CSV);
+      TableDesc d = CatalogUtil.newTableDesc(
+          CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, table), tpch.getSchema(table), m,
+          CommonTestingUtil.getTestDir());
+      catalog.createTable(d);
+    }
+
+    catalog.createFunction(funcDesc);
+    sqlAnalyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  static String[] QUERIES = {
+      "select name, empid, deptname from employee where empId > 500", // 0
+      "select name, empid, e.deptname, manager from employee as e, dept as dp", // 1
+      "select name, empid, e.deptname, manager, score from employee as e, dept, score", // 2
+      "select p.deptname, sumtest(score) from dept as p, score group by p.deptName having sumtest(score) > 30", // 3
+      "select p.deptname, score from dept as p, score order by score asc", // 4
+      "select name from employee where empId = 100", // 5
+      "select name, score from employee, score", // 6
+      "select p.deptName, sumtest(score) from dept as p, score group by p.deptName", // 7
+      "create table store1 as select p.deptName, sumtest(score) from dept as p, score group by p.deptName", // 8
+      "select deptName, sumtest(score) from score group by deptName having sumtest(score) > 30", // 9
+      "select 7 + 8 as res1, 8 * 9 as res2, 10 * 10 as res3", // 10
+      "create index idx_employee on employee using bitmap (name null first, empId desc) with ('fillfactor' = 70)", // 11
+      "select name, score from employee, score order by score limit 3", // 12
+      "select length(name), length(deptname), *, empid+10 from employee where empId > 500", // 13
+  };
+
+  @Test
+  public final void testSingleRelation() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[0]);
+    LogicalPlan planNode = planner.createPlan(session, expr);
+    LogicalNode plan = planNode.getRootBlock().getRoot();
+    assertEquals(NodeType.ROOT, plan.getType());
+    TestLogicalNode.testCloneLogicalNode(plan);
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+
+    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
+    SelectionNode selNode = projNode.getChild();
+
+    assertEquals(NodeType.SCAN, selNode.getChild().getType());
+    ScanNode scanNode = selNode.getChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), scanNode.getTableName());
+  }
+
+  public static void assertSchema(Schema expected, Schema schema) {
+    Column expectedColumn;
+    Column column;
+    for (int i = 0; i < expected.size(); i++) {
+      expectedColumn = expected.getColumn(i);
+      column = schema.getColumn(expectedColumn.getSimpleName());
+      assertEquals(expectedColumn.getSimpleName(), column.getSimpleName());
+      assertEquals(expectedColumn.getDataType(), column.getDataType());
+    }
+  }
+
+  @Test
+  public final void testImplicityJoinPlan() throws CloneNotSupportedException, PlanningException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(QUERIES[1]);
+    LogicalPlan planNode = planner.createPlan(session, expr);
+    LogicalNode plan = planNode.getRootBlock().getRoot();
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+    TestLogicalNode.testCloneLogicalNode(root);
+
+    Schema expectedSchema = new Schema();
+    expectedSchema.addColumn("name", Type.TEXT);
+    expectedSchema.addColumn("empid", Type.INT4);
+    expectedSchema.addColumn("deptname", Type.TEXT);
+    expectedSchema.addColumn("manager", Type.TEXT);
+    for (int i = 0; i < expectedSchema.size(); i++) {
+      Column found = root.getOutSchema().getColumn(expectedSchema.getColumn(i).getSimpleName());
+      assertEquals(expectedSchema.getColumn(i).getDataType(), found.getDataType());
+    }
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+
+    assertEquals(NodeType.JOIN, projNode.getChild().getType());
+    JoinNode joinNode = projNode.getChild();
+
+    assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
+    ScanNode leftNode = joinNode.getLeftChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), leftNode.getTableName());
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+    ScanNode rightNode = joinNode.getRightChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dept"), rightNode.getTableName());
+
+    // three relations
+    expr = sqlAnalyzer.parse(QUERIES[2]);
+    plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    TestLogicalNode.testCloneLogicalNode(plan);
+
+    expectedSchema.addColumn("score", Type.INT4);
+    assertSchema(expectedSchema, plan.getOutSchema());
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    root = (LogicalRootNode) plan;
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    projNode = root.getChild();
+
+    assertEquals(NodeType.JOIN, projNode.getChild().getType());
+    joinNode = projNode.getChild();
+
+    assertEquals(NodeType.JOIN, joinNode.getLeftChild().getType());
+
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+    ScanNode scan1 = joinNode.getRightChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), scan1.getTableName());
+
+    JoinNode leftNode2 = joinNode.getLeftChild();
+    assertEquals(NodeType.JOIN, leftNode2.getType());
+
+    assertEquals(NodeType.SCAN, leftNode2.getLeftChild().getType());
+    ScanNode leftScan = leftNode2.getLeftChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), leftScan.getTableName());
+
+    assertEquals(NodeType.SCAN, leftNode2.getRightChild().getType());
+    ScanNode rightScan = leftNode2.getRightChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dept"), rightScan.getTableName());
+  }
+
+
+
+  String [] JOINS = {
+      "select name, dept.deptName, score from employee natural join dept natural join score", // 0
+      "select name, dept.deptName, score from employee inner join dept on employee.deptName = dept.deptName inner join score on dept.deptName = score.deptName", // 1
+      "select name, dept.deptName, score from employee left outer join dept on employee.deptName = dept.deptName right outer join score on dept.deptName = score.deptName" // 2
+  };
+
+  static Schema expectedJoinSchema;
+  static {
+    expectedJoinSchema = new Schema();
+    expectedJoinSchema.addColumn("name", Type.TEXT);
+    expectedJoinSchema.addColumn("deptname", Type.TEXT);
+    expectedJoinSchema.addColumn("score", Type.INT4);
+  }
+
+  @Test
+  public final void testNaturalJoinPlan() throws PlanningException {
+    // two relations
+    Expr context = sqlAnalyzer.parse(JOINS[0]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    assertSchema(expectedJoinSchema, plan.getOutSchema());
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode proj = root.getChild();
+    assertEquals(NodeType.JOIN, proj.getChild().getType());
+    JoinNode join = proj.getChild();
+    assertEquals(JoinType.INNER, join.getJoinType());
+    assertEquals(NodeType.SCAN, join.getRightChild().getType());
+    assertTrue(join.hasJoinQual());
+    ScanNode scan = join.getRightChild();
+    assertEquals("default.score", scan.getTableName());
+
+    assertEquals(NodeType.JOIN, join.getLeftChild().getType());
+    join = join.getLeftChild();
+    assertEquals(JoinType.INNER, join.getJoinType());
+    assertEquals(NodeType.SCAN, join.getLeftChild().getType());
+    ScanNode outer = join.getLeftChild();
+    assertEquals("default.employee", outer.getTableName());
+    assertEquals(NodeType.SCAN, join.getRightChild().getType());
+    ScanNode inner = join.getRightChild();
+    assertEquals("default.dept", inner.getTableName());
+  }
+
+  @Test
+  public final void testInnerJoinPlan() throws PlanningException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(JOINS[1]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalNode root = plan.getRootBlock().getRoot();
+    testJsonSerDerObject(root);
+    assertSchema(expectedJoinSchema, root.getOutSchema());
+
+    assertEquals(NodeType.ROOT, root.getType());
+    assertEquals(NodeType.PROJECTION, ((LogicalRootNode)root).getChild().getType());
+    ProjectionNode proj = ((LogicalRootNode)root).getChild();
+    assertEquals(NodeType.JOIN, proj.getChild().getType());
+    JoinNode join = proj.getChild();
+    assertEquals(JoinType.INNER, join.getJoinType());
+    assertEquals(NodeType.SCAN, join.getRightChild().getType());
+    ScanNode scan = join.getRightChild();
+    assertEquals("default.score", scan.getTableName());
+
+    assertEquals(NodeType.JOIN, join.getLeftChild().getType());
+    join = join.getLeftChild();
+    assertEquals(JoinType.INNER, join.getJoinType());
+    assertEquals(NodeType.SCAN, join.getLeftChild().getType());
+    ScanNode outer = join.getLeftChild();
+    assertEquals("default.employee", outer.getTableName());
+    assertEquals(NodeType.SCAN, join.getRightChild().getType());
+    ScanNode inner = join.getRightChild();
+    assertEquals("default.dept", inner.getTableName());
+    assertTrue(join.hasJoinQual());
+    assertEquals(EvalType.EQUAL, join.getJoinQual().getType());
+  }
+
+  @Test
+  public final void testOuterJoinPlan() throws PlanningException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(JOINS[2]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    assertSchema(expectedJoinSchema, plan.getOutSchema());
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode proj = root.getChild();
+    assertEquals(NodeType.JOIN, proj.getChild().getType());
+    JoinNode join = proj.getChild();
+    assertEquals(JoinType.RIGHT_OUTER, join.getJoinType());
+    assertEquals(NodeType.SCAN, join.getRightChild().getType());
+    ScanNode scan = join.getRightChild();
+    assertEquals("default.score", scan.getTableName());
+
+    assertEquals(NodeType.JOIN, join.getLeftChild().getType());
+    join = join.getLeftChild();
+    assertEquals(JoinType.LEFT_OUTER, join.getJoinType());
+    assertEquals(NodeType.SCAN, join.getLeftChild().getType());
+    ScanNode outer = join.getLeftChild();
+    assertEquals("default.employee", outer.getTableName());
+    assertEquals(NodeType.SCAN, join.getRightChild().getType());
+    ScanNode inner = join.getRightChild();
+    assertEquals("default.dept", inner.getTableName());
+    assertTrue(join.hasJoinQual());
+    assertEquals(EvalType.EQUAL, join.getJoinQual().getType());
+  }
+
+
+  @Test
+  public final void testGroupby() throws CloneNotSupportedException, PlanningException {
+    // without 'having clause'
+    Expr context = sqlAnalyzer.parse(QUERIES[7]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+    testQuery7(root.getChild());
+
+    // with having clause
+    context = sqlAnalyzer.parse(QUERIES[3]);
+    plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    TestLogicalNode.testCloneLogicalNode(plan);
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    root = (LogicalRootNode) plan;
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+    assertEquals(NodeType.HAVING, projNode.getChild().getType());
+    HavingNode havingNode = projNode.getChild();
+    assertEquals(NodeType.GROUP_BY, havingNode.getChild().getType());
+    GroupbyNode groupByNode =  havingNode.getChild();
+
+    assertEquals(NodeType.JOIN, groupByNode.getChild().getType());
+    JoinNode joinNode = groupByNode.getChild();
+
+    assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
+    ScanNode leftNode = joinNode.getLeftChild();
+    assertEquals("default.dept", leftNode.getTableName());
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+    ScanNode rightNode = joinNode.getRightChild();
+    assertEquals("default.score", rightNode.getTableName());
+
+    //LogicalOptimizer.optimize(context, plan);
+  }
+
+
+  @Test
+  public final void testMultipleJoin() throws IOException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(
+        FileUtil.readTextFile(new File("src/test/resources/queries/TestJoinQuery/testTPCHQ2Join.sql")));
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    Schema expected = tpch.getOutSchema("q2");
+    assertSchema(expected, plan.getOutSchema());
+  }
+
+  private final void findJoinQual(EvalNode evalNode, Map<BinaryEval, Boolean> qualMap,
+                                  EvalType leftType, EvalType rightType)
+      throws IOException, PlanningException {
+    Preconditions.checkArgument(evalNode instanceof BinaryEval);
+    BinaryEval qual = (BinaryEval)evalNode;
+
+    if (qual.getLeftExpr().getType() == leftType && qual.getRightExpr().getType() == rightType) {
+      assertEquals(qual.getLeftExpr().getType(), EvalType.FIELD);
+      FieldEval leftField = (FieldEval)qual.getLeftExpr();
+
+      for (Map.Entry<BinaryEval, Boolean> entry : qualMap.entrySet()) {
+        FieldEval leftJoinField = (FieldEval)entry.getKey().getLeftExpr();
+
+        if (qual.getRightExpr().getType() == entry.getKey().getRightExpr().getType()) {
+          if (rightType == EvalType.FIELD) {
+            FieldEval rightField = (FieldEval)qual.getRightExpr();
+            FieldEval rightJoinField = (FieldEval)entry.getKey().getRightExpr();
+
+            if (leftJoinField.getColumnRef().getQualifiedName().equals(leftField.getColumnRef().getQualifiedName())
+                && rightField.getColumnRef().getQualifiedName().equals(rightJoinField.getColumnRef().getQualifiedName())) {
+              qualMap.put(entry.getKey(), Boolean.TRUE);
+            }
+          } else if (rightType == EvalType.CONST) {
+            ConstEval rightField = (ConstEval)qual.getRightExpr();
+            ConstEval rightJoinField = (ConstEval)entry.getKey().getRightExpr();
+
+            if (leftJoinField.getColumnRef().getQualifiedName().equals(leftField.getColumnRef().getQualifiedName()) &&
+                rightField.getValue().equals(rightJoinField.getValue())) {
+              qualMap.put(entry.getKey(), Boolean.TRUE);
+            }
+          } else if (rightType == EvalType.ROW_CONSTANT) {
+            RowConstantEval rightField = (RowConstantEval)qual.getRightExpr();
+            RowConstantEval rightJoinField = (RowConstantEval)entry.getKey().getRightExpr();
+
+            if (leftJoinField.getColumnRef().getQualifiedName().equals(leftField.getColumnRef().getQualifiedName())) {
+              assertEquals(rightField.getValues().length, rightJoinField.getValues().length);
+              for (int i = 0; i < rightField.getValues().length; i++) {
+                assertEquals(rightField.getValues()[i], rightJoinField.getValues()[i]);
+              }
+              qualMap.put(entry.getKey(), Boolean.TRUE);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Test
+  public final void testJoinWithMultipleJoinQual1() throws IOException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(
+        FileUtil.readTextFile(new File
+            ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual1.sql")));
+
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalNode node = plan.getRootBlock().getRoot();
+    testJsonSerDerObject(node);
+
+    Schema expected = tpch.getOutSchema("q2");
+    assertSchema(expected, node.getOutSchema());
+
+    LogicalOptimizer optimizer = new LogicalOptimizer(util.getConfiguration());
+    optimizer.optimize(plan);
+
+    LogicalNode[] nodes = PlannerUtil.findAllNodes(node, NodeType.JOIN);
+    Map<BinaryEval, Boolean> qualMap = TUtil.newHashMap();
+    BinaryEval joinQual = new BinaryEval(EvalType.EQUAL
+        , new FieldEval(new Column("default.n.n_regionkey", Type.INT4))
+        , new FieldEval(new Column("default.ps.ps_suppkey", Type.INT4))
+        );
+    qualMap.put(joinQual, Boolean.FALSE);
+
+    for(LogicalNode eachNode : nodes) {
+      JoinNode joinNode = (JoinNode)eachNode;
+      EvalNode[] evalNodes = AlgebraicUtil.toConjunctiveNormalFormArray(joinNode.getJoinQual());
+
+      for(EvalNode evalNode : evalNodes) {
+        findJoinQual(evalNode, qualMap, EvalType.FIELD, EvalType.FIELD);
+      }
+    }
+
+    for (Map.Entry<BinaryEval, Boolean> entry : qualMap.entrySet()) {
+      if (!entry.getValue().booleanValue()) {
+        Preconditions.checkArgument(false,
+            "JoinQual not found. -> required JoinQual:" + entry.getKey().toJson());
+      }
+    }
+  }
+
+  @Test
+  public final void testJoinWithMultipleJoinQual2() throws IOException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(
+        FileUtil.readTextFile(new File
+            ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual2.sql")));
+
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalNode node = plan.getRootBlock().getRoot();
+    testJsonSerDerObject(node);
+
+    LogicalOptimizer optimizer = new LogicalOptimizer(util.getConfiguration());
+    optimizer.optimize(plan);
+
+    LogicalNode[] nodes = PlannerUtil.findAllNodes(node, NodeType.SCAN);
+    Map<BinaryEval, Boolean> qualMap = TUtil.newHashMap();
+    BinaryEval joinQual = new BinaryEval(EvalType.EQUAL
+        , new FieldEval(new Column("default.n.n_name", Type.TEXT))
+        , new ConstEval(new TextDatum("MOROCCO"))
+    );
+    qualMap.put(joinQual, Boolean.FALSE);
+
+    for(LogicalNode eachNode : nodes) {
+      ScanNode scanNode = (ScanNode)eachNode;
+      if (scanNode.hasQual()) {
+        EvalNode[] evalNodes = AlgebraicUtil.toConjunctiveNormalFormArray(scanNode.getQual());
+
+        for(EvalNode evalNode : evalNodes) {
+          findJoinQual(evalNode, qualMap, EvalType.FIELD, EvalType.CONST);
+        }
+      }
+    }
+
+    for (Map.Entry<BinaryEval, Boolean> entry : qualMap.entrySet()) {
+      if (!entry.getValue().booleanValue()) {
+        Preconditions.checkArgument(false,
+            "SelectionQual not found. -> required JoinQual:" + entry.getKey().toJson());
+      }
+    }
+  }
+
+  @Test
+  public final void testJoinWithMultipleJoinQual3() throws IOException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(
+        FileUtil.readTextFile(new File
+            ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual3.sql")));
+
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalNode node = plan.getRootBlock().getRoot();
+    testJsonSerDerObject(node);
+
+    LogicalOptimizer optimizer = new LogicalOptimizer(util.getConfiguration());
+    optimizer.optimize(plan);
+
+    LogicalNode[] nodes = PlannerUtil.findAllNodes(node, NodeType.SCAN);
+    Map<BinaryEval, Boolean> qualMap = TUtil.newHashMap();
+    TextDatum[] datums = new TextDatum[3];
+    datums[0] = new TextDatum("ARGENTINA");
+    datums[1] = new TextDatum("ETHIOPIA");
+    datums[2] = new TextDatum("MOROCCO");
+
+    BinaryEval joinQual = new BinaryEval(EvalType.EQUAL
+        , new FieldEval(new Column("default.n.n_name", Type.TEXT))
+        , new RowConstantEval(datums)
+    );
+    qualMap.put(joinQual, Boolean.FALSE);
+
+    for(LogicalNode eachNode : nodes) {
+      ScanNode scanNode = (ScanNode)eachNode;
+      if (scanNode.hasQual()) {
+        EvalNode[] evalNodes = AlgebraicUtil.toConjunctiveNormalFormArray(scanNode.getQual());
+
+        for(EvalNode evalNode : evalNodes) {
+          findJoinQual(evalNode, qualMap, EvalType.FIELD, EvalType.ROW_CONSTANT);
+        }
+      }
+    }
+
+    for (Map.Entry<BinaryEval, Boolean> entry : qualMap.entrySet()) {
+      if (!entry.getValue().booleanValue()) {
+        Preconditions.checkArgument(false,
+            "ScanQual not found. -> required JoinQual:" + entry.getKey().toJson());
+      }
+    }
+  }
+
+
+  @Test
+  public final void testJoinWithMultipleJoinQual4() throws IOException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(
+        FileUtil.readTextFile(new File
+            ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual4.sql")));
+
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalNode node = plan.getRootBlock().getRoot();
+    testJsonSerDerObject(node);
+
+    LogicalOptimizer optimizer = new LogicalOptimizer(util.getConfiguration());
+    optimizer.optimize(plan);
+
+    Map<BinaryEval, Boolean> scanMap = TUtil.newHashMap();
+    TextDatum[] datums = new TextDatum[3];
+    datums[0] = new TextDatum("ARGENTINA");
+    datums[1] = new TextDatum("ETHIOPIA");
+    datums[2] = new TextDatum("MOROCCO");
+
+    BinaryEval scanQual = new BinaryEval(EvalType.EQUAL
+        , new FieldEval(new Column("default.n.n_name", Type.TEXT))
+        , new RowConstantEval(datums)
+    );
+    scanMap.put(scanQual, Boolean.FALSE);
+
+    Map<BinaryEval, Boolean> joinQualMap = TUtil.newHashMap();
+    BinaryEval joinQual = new BinaryEval(EvalType.GTH
+        , new FieldEval(new Column("default.t.n_nationkey", Type.INT4))
+        , new FieldEval(new Column("default.s.s_suppkey", Type.INT4))
+    );
+    joinQualMap.put(joinQual, Boolean.FALSE);
+
+    LogicalNode[] nodes = PlannerUtil.findAllNodes(node, NodeType.JOIN);
+    for(LogicalNode eachNode : nodes) {
+      JoinNode joinNode = (JoinNode)eachNode;
+      if (joinNode.hasJoinQual()) {
+        EvalNode[] evalNodes = AlgebraicUtil.toConjunctiveNormalFormArray(joinNode.getJoinQual());
+
+        for(EvalNode evalNode : evalNodes) {
+          findJoinQual(evalNode, joinQualMap, EvalType.FIELD, EvalType.FIELD);
+        }
+      }
+    }
+
+    nodes = PlannerUtil.findAllNodes(node, NodeType.SCAN);
+    for(LogicalNode eachNode : nodes) {
+      ScanNode scanNode = (ScanNode)eachNode;
+      if (scanNode.hasQual()) {
+        EvalNode[] evalNodes = AlgebraicUtil.toConjunctiveNormalFormArray(scanNode.getQual());
+
+        for(EvalNode evalNode : evalNodes) {
+          findJoinQual(evalNode, scanMap, EvalType.FIELD, EvalType.ROW_CONSTANT);
+        }
+      }
+    }
+
+
+    for (Map.Entry<BinaryEval, Boolean> entry : joinQualMap.entrySet()) {
+      if (!entry.getValue().booleanValue()) {
+        Preconditions.checkArgument(false,
+            "JoinQual not found. -> required JoinQual:" + entry.getKey().toJson());
+      }
+    }
+
+    for (Map.Entry<BinaryEval, Boolean> entry : scanMap.entrySet()) {
+      if (!entry.getValue().booleanValue()) {
+        Preconditions.checkArgument(false,
+            "ScanQual not found. -> required JoinQual:" + entry.getKey().toJson());
+      }
+    }
+  }
+
+  static void testQuery7(LogicalNode plan) {
+    assertEquals(NodeType.PROJECTION, plan.getType());
+    ProjectionNode projNode = (ProjectionNode) plan;
+    assertEquals(NodeType.GROUP_BY, projNode.getChild().getType());
+    GroupbyNode groupByNode = projNode.getChild();
+
+    assertEquals(NodeType.JOIN, groupByNode.getChild().getType());
+    JoinNode joinNode = groupByNode.getChild();
+
+    assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
+    ScanNode leftNode = joinNode.getLeftChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dept"), leftNode.getTableName());
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+    ScanNode rightNode = joinNode.getRightChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), rightNode.getTableName());
+  }
+
+
+  @Test
+  public final void testStoreTable() throws CloneNotSupportedException, PlanningException {
+    Expr context = sqlAnalyzer.parse(QUERIES[8]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    TestLogicalNode.testCloneLogicalNode(plan);
+    testJsonSerDerObject(plan);
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+
+    assertEquals(NodeType.CREATE_TABLE, root.getChild().getType());
+    StoreTableNode storeNode = root.getChild();
+    testQuery7(storeNode.getChild());
+  }
+
+  @Test
+  public final void testOrderBy() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[4]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    TestLogicalNode.testCloneLogicalNode(plan);
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+
+    assertEquals(NodeType.SORT, projNode.getChild().getType());
+    SortNode sortNode = projNode.getChild();
+
+    assertEquals(NodeType.JOIN, sortNode.getChild().getType());
+    JoinNode joinNode = sortNode.getChild();
+
+    assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
+    ScanNode leftNode = joinNode.getLeftChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dept"), leftNode.getTableName());
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+    ScanNode rightNode = joinNode.getRightChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), rightNode.getTableName());
+  }
+
+  @Test
+  public final void testLimit() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[12]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    TestLogicalNode.testCloneLogicalNode(plan);
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+
+    assertEquals(NodeType.LIMIT, projNode.getChild().getType());
+    LimitNode limitNode = projNode.getChild();
+
+    assertEquals(NodeType.SORT, limitNode.getChild().getType());
+  }
+
+  @Test
+  public final void testSPJPush() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[5]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    TestLogicalNode.testCloneLogicalNode(plan);
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
+    SelectionNode selNode = projNode.getChild();
+    assertEquals(NodeType.SCAN, selNode.getChild().getType());
+    ScanNode scanNode = selNode.getChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), scanNode.getTableName());
+  }
+
+
+
+  @Test
+  public final void testSPJ() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[6]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    TestLogicalNode.testCloneLogicalNode(plan);
+  }
+
+  @Test
+  public final void testJson() throws PlanningException {
+	  Expr expr = sqlAnalyzer.parse(QUERIES[9]);
+	  LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+
+	  String json = plan.toJson();
+	  LogicalNode fromJson = CoreGsonHelper.fromJson(json, LogicalNode.class);
+	  assertEquals(NodeType.ROOT, fromJson.getType());
+	  LogicalNode project = ((LogicalRootNode)fromJson).getChild();
+	  assertEquals(NodeType.PROJECTION, project.getType());
+	  assertEquals(NodeType.HAVING, ((ProjectionNode) project).getChild().getType());
+    HavingNode havingNode = ((ProjectionNode) project).getChild();
+    assertEquals(NodeType.GROUP_BY, havingNode.getChild().getType());
+    GroupbyNode groupbyNode = havingNode.getChild();
+    assertEquals(NodeType.SCAN, groupbyNode.getChild().getType());
+	  LogicalNode scan = groupbyNode.getChild();
+	  assertEquals(NodeType.SCAN, scan.getType());
+  }
+
+  @Test
+  public final void testVisitor() throws PlanningException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    TestVisitor vis = new TestVisitor();
+    plan.postOrder(vis);
+
+    assertEquals(NodeType.ROOT, vis.stack.pop().getType());
+    assertEquals(NodeType.PROJECTION, vis.stack.pop().getType());
+    assertEquals(NodeType.JOIN, vis.stack.pop().getType());
+    assertEquals(NodeType.SCAN, vis.stack.pop().getType());
+    assertEquals(NodeType.SCAN, vis.stack.pop().getType());
+  }
+
+  private static class TestVisitor implements LogicalNodeVisitor {
+    Stack<LogicalNode> stack = new Stack<LogicalNode>();
+    @Override
+    public void visit(LogicalNode node) {
+      stack.push(node);
+    }
+  }
+
+
+  @Test
+  public final void testExprNode() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[10]);
+    LogicalPlan rootNode = planner.createPlan(session, expr);
+    LogicalNode plan = rootNode.getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    assertEquals(NodeType.EXPRS, root.getChild().getType());
+    Schema out = root.getOutSchema();
+
+    Iterator<Column> it = out.getColumns().iterator();
+    Column col = it.next();
+    assertEquals("res1", col.getSimpleName());
+    col = it.next();
+    assertEquals("res2", col.getSimpleName());
+    col = it.next();
+    assertEquals("res3", col.getSimpleName());
+  }
+
+  @Test
+  public final void testAsterisk() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[13]);
+    LogicalPlan planNode = planner.createPlan(session, expr);
+    LogicalNode plan = planNode.getRootBlock().getRoot();
+    assertEquals(NodeType.ROOT, plan.getType());
+    TestLogicalNode.testCloneLogicalNode(plan);
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+    assertEquals(6, projNode.getOutSchema().size());
+
+    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
+    SelectionNode selNode = projNode.getChild();
+
+    assertEquals(NodeType.SCAN, selNode.getChild().getType());
+    ScanNode scanNode = selNode.getChild();
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), scanNode.getTableName());
+  }
+
+  static final String ALIAS [] = {
+    "select deptName, sum(score) as total from score group by deptName",
+    "select em.empId as id, sum(score) as total from employee as em inner join score using (em.deptName) group by id"
+  };
+
+
+  @Test
+  public final void testAlias1() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(ALIAS[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+
+    Schema finalSchema = root.getOutSchema();
+    Iterator<Column> it = finalSchema.getColumns().iterator();
+    Column col = it.next();
+    assertEquals("deptname", col.getSimpleName());
+    col = it.next();
+    assertEquals("total", col.getSimpleName());
+
+    expr = sqlAnalyzer.parse(ALIAS[1]);
+    plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    root = (LogicalRootNode) plan;
+
+    finalSchema = root.getOutSchema();
+    it = finalSchema.getColumns().iterator();
+    col = it.next();
+    assertEquals("id", col.getSimpleName());
+    col = it.next();
+    assertEquals("total", col.getSimpleName());
+  }
+
+  @Test
+  public final void testAlias2() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(ALIAS[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+
+    Schema finalSchema = root.getOutSchema();
+    Iterator<Column> it = finalSchema.getColumns().iterator();
+    Column col = it.next();
+    assertEquals("id", col.getSimpleName());
+    col = it.next();
+    assertEquals("total", col.getSimpleName());
+  }
+
+  static final String CREATE_TABLE [] = {
+    "create external table table1 (name text, age int, earn bigint, score real) using csv with ('csv.delimiter'='|') location '/tmp/data'"
+  };
+
+  @Test
+  public final void testCreateTableDef() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(CREATE_TABLE[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+    assertEquals(NodeType.CREATE_TABLE, root.getChild().getType());
+    CreateTableNode createTable = root.getChild();
+
+    Schema def = createTable.getTableSchema();
+    assertEquals("name", def.getColumn(0).getSimpleName());
+    assertEquals(Type.TEXT, def.getColumn(0).getDataType().getType());
+    assertEquals("age", def.getColumn(1).getSimpleName());
+    assertEquals(Type.INT4, def.getColumn(1).getDataType().getType());
+    assertEquals("earn", def.getColumn(2).getSimpleName());
+    assertEquals(Type.INT8, def.getColumn(2).getDataType().getType());
+    assertEquals("score", def.getColumn(3).getSimpleName());
+    assertEquals(Type.FLOAT4, def.getColumn(3).getDataType().getType());
+    assertEquals(StoreType.CSV, createTable.getStorageType());
+    assertEquals("/tmp/data", createTable.getPath().toString());
+    assertTrue(createTable.hasOptions());
+    assertEquals("|", createTable.getOptions().get("csv.delimiter"));
+  }
+
+  private static final List<Set<Column>> testGenerateCuboidsResult
+    = Lists.newArrayList();
+  private static final int numCubeColumns = 3;
+  private static final Column [] testGenerateCuboids = new Column[numCubeColumns];
+
+  private static final List<Set<Column>> testCubeByResult
+    = Lists.newArrayList();
+  private static final Column [] testCubeByCuboids = new Column[2];
+  static {
+    testGenerateCuboids[0] = new Column("col1", Type.INT4);
+    testGenerateCuboids[1] = new Column("col2", Type.INT8);
+    testGenerateCuboids[2] = new Column("col3", Type.FLOAT4);
+
+    testGenerateCuboidsResult.add(new HashSet<Column>());
+    testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[0]));
+    testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[1]));
+    testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[2]));
+    testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[0],
+        testGenerateCuboids[1]));
+    testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[0],
+        testGenerateCuboids[2]));
+    testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[1],
+        testGenerateCuboids[2]));
+    testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[0],
+        testGenerateCuboids[1], testGenerateCuboids[2]));
+
+    testCubeByCuboids[0] = new Column("employee.name", Type.TEXT);
+    testCubeByCuboids[1] = new Column("employee.empid", Type.INT4);
+    testCubeByResult.add(new HashSet<Column>());
+    testCubeByResult.add(Sets.newHashSet(testCubeByCuboids[0]));
+    testCubeByResult.add(Sets.newHashSet(testCubeByCuboids[1]));
+    testCubeByResult.add(Sets.newHashSet(testCubeByCuboids[0],
+        testCubeByCuboids[1]));
+  }
+
+  @Test
+  public final void testGenerateCuboids() {
+    Column [] columns = new Column[3];
+
+    columns[0] = new Column("col1", Type.INT4);
+    columns[1] = new Column("col2", Type.INT8);
+    columns[2] = new Column("col3", Type.FLOAT4);
+
+    List<Column[]> cube = LogicalPlanner.generateCuboids(columns);
+    assertEquals(((int)Math.pow(2, numCubeColumns)), cube.size());
+
+    Set<Set<Column>> cuboids = Sets.newHashSet();
+    for (Column [] cols : cube) {
+      cuboids.add(Sets.newHashSet(cols));
+    }
+
+    for (Set<Column> result : testGenerateCuboidsResult) {
+      assertTrue(cuboids.contains(result));
+    }
+  }
+
+  static final String setStatements [] = {
+    "select deptName from employee where deptName like 'data%' union select deptName from score where deptName like 'data%'",
+  };
+
+  @Test
+  public final void testSetPlan() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(setStatements[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    assertEquals(NodeType.UNION, root.getChild().getType());
+    UnionNode union = root.getChild();
+    assertEquals(NodeType.PROJECTION, union.getLeftChild().getType());
+    assertEquals(NodeType.PROJECTION, union.getRightChild().getType());
+  }
+
+  static final String [] setQualifiers = {
+    "select name, empid from employee",
+    "select distinct name, empid from employee",
+    "select all name, empid from employee",
+  };
+
+  @Test
+  public void testSetQualifier() throws PlanningException {
+    Expr context = sqlAnalyzer.parse(setQualifiers[0]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projectionNode = root.getChild();
+    assertEquals(NodeType.SCAN, projectionNode.getChild().getType());
+
+    context = sqlAnalyzer.parse(setQualifiers[1]);
+    plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    assertEquals(NodeType.ROOT, plan.getType());
+    root = (LogicalRootNode) plan;
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    projectionNode = root.getChild();
+    assertEquals(NodeType.GROUP_BY, projectionNode.getChild().getType());
+
+    context = sqlAnalyzer.parse(setQualifiers[2]);
+    plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    testJsonSerDerObject(plan);
+    root = (LogicalRootNode) plan;
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    projectionNode = root.getChild();
+    assertEquals(NodeType.SCAN, projectionNode.getChild().getType());
+  }
+
+  public void testJsonSerDerObject(LogicalNode rootNode) {
+    String json = rootNode.toJson();
+    LogicalNode fromJson = CoreGsonHelper.fromJson(json, LogicalNode.class);
+    assertTrue("JSON (de) serialization equivalence check", rootNode.deepEquals(fromJson));
+  }
+
+  // Table descriptions
+  //
+  // employee (name text, empid int4, deptname text)
+  // dept (deptname text, nameger text)
+  // score (deptname text, score inet4)
+
+  static final String [] insertStatements = {
+      "insert into score select name from employee",                        // 0
+      "insert into score select name, empid from employee",                 // 1
+      "insert into employee (name, deptname) select * from dept",           // 2
+      "insert into location '/tmp/data' select name, empid from employee",  // 3
+      "insert overwrite into employee (name, deptname) select * from dept", // 4
+      "insert overwrite into LOCATION '/tmp/data' select * from dept"       // 5
+  };
+
+  @Test
+  public final void testInsertInto0() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(insertStatements[0]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    assertEquals(1, plan.getQueryBlocks().size());
+    InsertNode insertNode = getInsertNode(plan);
+    assertFalse(insertNode.isOverwrite());
+    assertTrue(insertNode.hasTargetTable());
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), insertNode.getTableName());
+  }
+
+  @Test
+  public final void testInsertInto1() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(insertStatements[1]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    assertEquals(1, plan.getQueryBlocks().size());
+    InsertNode insertNode = getInsertNode(plan);
+    assertFalse(insertNode.isOverwrite());
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), insertNode.getTableName());
+  }
+
+  @Test
+  public final void testInsertInto2() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(insertStatements[2]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    assertEquals(1, plan.getQueryBlocks().size());
+    InsertNode insertNode = getInsertNode(plan);
+    assertFalse(insertNode.isOverwrite());
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), insertNode.getTableName());
+    assertTrue(insertNode.hasTargetSchema());
+    assertEquals(insertNode.getTargetSchema().getColumn(0).getSimpleName(), "name");
+    assertEquals(insertNode.getTargetSchema().getColumn(1).getSimpleName(), "deptname");
+  }
+
+  @Test
+  public final void testInsertInto3() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(insertStatements[3]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    assertEquals(1, plan.getQueryBlocks().size());
+    InsertNode insertNode = getInsertNode(plan);
+    assertFalse(insertNode.isOverwrite());
+    assertTrue(insertNode.hasPath());
+  }
+
+  @Test
+  public final void testInsertInto4() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(insertStatements[4]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    assertEquals(1, plan.getQueryBlocks().size());
+    InsertNode insertNode = getInsertNode(plan);
+    assertTrue(insertNode.isOverwrite());
+    assertTrue(insertNode.hasTargetTable());
+    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), insertNode.getTableName());
+    assertTrue(insertNode.hasTargetSchema());
+    assertEquals(insertNode.getTargetSchema().getColumn(0).getSimpleName(), "name");
+    assertEquals(insertNode.getTargetSchema().getColumn(1).getSimpleName(), "deptname");
+  }
+
+  @Test
+  public final void testInsertInto5() throws PlanningException {
+    Expr expr = sqlAnalyzer.parse(insertStatements[5]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    assertEquals(1, plan.getQueryBlocks().size());
+    InsertNode insertNode = getInsertNode(plan);
+    assertTrue(insertNode.isOverwrite());
+    assertTrue(insertNode.hasPath());
+  }
+
+  private static InsertNode getInsertNode(LogicalPlan plan) {
+    LogicalRootNode root = plan.getRootBlock().getRoot();
+    assertEquals(NodeType.INSERT, root.getChild().getType());
+    return root.getChild();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
new file mode 100644
index 0000000..be4c133
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
@@ -0,0 +1,321 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.function.builtin.SumInt;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.*;
+
+public class TestPlannerUtil {
+  private static TajoTestingCluster util;
+  private static CatalogService catalog;
+  private static SQLAnalyzer analyzer;
+  private static LogicalPlanner planner;
+  private static Session session = LocalTajoTestingUtility.createDummySession();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.startCatalogCluster();
+    catalog = util.getMiniCatalogCluster().getCatalog();
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse");
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+
+    Schema schema = new Schema();
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("empid", CatalogUtil.newSimpleDataType(Type.INT4));
+    schema.addColumn("deptname", Type.TEXT);
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("deptname", Type.TEXT);
+    schema2.addColumn("manager", Type.TEXT);
+
+    Schema schema3 = new Schema();
+    schema3.addColumn("deptname", Type.TEXT);
+    schema3.addColumn("score", CatalogUtil.newSimpleDataType(Type.INT4));
+
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    TableDesc people = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta,
+        CommonTestingUtil.getTestDir());
+    catalog.createTable(people);
+
+    TableDesc student =
+        new TableDesc(
+            CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dept"), schema2, StoreType.CSV,
+            new Options(), CommonTestingUtil.getTestDir());
+    catalog.createTable(student);
+
+    TableDesc score =
+        new TableDesc(
+            CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), schema3, StoreType.CSV,
+            new Options(), CommonTestingUtil.getTestDir());
+    catalog.createTable(score);
+
+    FunctionDesc funcDesc = new FunctionDesc("sumtest", SumInt.class, FunctionType.AGGREGATION,
+        CatalogUtil.newSimpleDataType(Type.INT4),
+        CatalogUtil.newSimpleDataTypeArray(Type.INT4));
+
+    catalog.createFunction(funcDesc);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  @Test
+  public final void testFindTopNode() throws CloneNotSupportedException, PlanningException {
+    // two relations
+    Expr expr = analyzer.parse(TestLogicalPlanner.QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    TestLogicalNode.testCloneLogicalNode(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+
+    assertEquals(NodeType.JOIN, projNode.getChild().getType());
+    JoinNode joinNode = projNode.getChild();
+
+    assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
+    ScanNode leftNode = joinNode.getLeftChild();
+    assertEquals("default.employee", leftNode.getTableName());
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+    ScanNode rightNode = joinNode.getRightChild();
+    assertEquals("default.dept", rightNode.getTableName());
+    
+    LogicalNode node = PlannerUtil.findTopNode(root, NodeType.ROOT);
+    assertEquals(NodeType.ROOT, node.getType());
+    
+    node = PlannerUtil.findTopNode(root, NodeType.PROJECTION);
+    assertEquals(NodeType.PROJECTION, node.getType());
+    
+    node = PlannerUtil.findTopNode(root, NodeType.JOIN);
+    assertEquals(NodeType.JOIN, node.getType());
+    
+    node = PlannerUtil.findTopNode(root, NodeType.SCAN);
+    assertEquals(NodeType.SCAN, node.getType());
+  }
+
+  @Test
+  public final void testIsJoinQual() {
+    FieldEval f1 = new FieldEval("part.p_partkey", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f2 = new FieldEval("partsupp.ps_partkey",
+        CatalogUtil.newSimpleDataType(Type.INT4));
+
+
+    BinaryEval [] joinQuals = new BinaryEval[5];
+    int idx = 0;
+    joinQuals[idx++] = new BinaryEval(EvalType.EQUAL, f1, f2);
+    joinQuals[idx++] = new BinaryEval(EvalType.LEQ, f1, f2);
+    joinQuals[idx++] = new BinaryEval(EvalType.LTH, f1, f2);
+    joinQuals[idx++] = new BinaryEval(EvalType.GEQ, f1, f2);
+    joinQuals[idx] = new BinaryEval(EvalType.GTH, f1, f2);
+    for (int i = 0; i < idx; i++) {
+      assertTrue(EvalTreeUtil.isJoinQual(joinQuals[idx], true));
+    }
+
+    BinaryEval [] wrongJoinQuals = new BinaryEval[5];
+    idx = 0;
+    wrongJoinQuals[idx++] = new BinaryEval(EvalType.OR, f1, f2);
+    wrongJoinQuals[idx++] = new BinaryEval(EvalType.PLUS, f1, f2);
+    wrongJoinQuals[idx++] = new BinaryEval(EvalType.LIKE, f1, f2);
+
+    ConstEval f3 = new ConstEval(DatumFactory.createInt4(1));
+    wrongJoinQuals[idx] = new BinaryEval(EvalType.EQUAL, f1, f3);
+
+    for (int i = 0; i < idx; i++) {
+      assertFalse(EvalTreeUtil.isJoinQual(wrongJoinQuals[idx], true));
+    }
+  }
+
+  @Test
+  public final void testGetJoinKeyPairs() {
+    Schema outerSchema = new Schema();
+    outerSchema.addColumn("employee.id1", CatalogUtil.newSimpleDataType(Type.INT4));
+    outerSchema.addColumn("employee.id2", CatalogUtil.newSimpleDataType(Type.INT4));
+    Schema innerSchema = new Schema();
+    innerSchema.addColumn("people.fid1", CatalogUtil.newSimpleDataType(Type.INT4));
+    innerSchema.addColumn("people.fid2", CatalogUtil.newSimpleDataType(Type.INT4));
+
+    FieldEval f1 = new FieldEval("employee.id1", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f2 = new FieldEval("people.fid1", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f3 = new FieldEval("employee.id2", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f4 = new FieldEval("people.fid2", CatalogUtil.newSimpleDataType(Type.INT4));
+
+    EvalNode equiJoinQual = new BinaryEval(EvalType.EQUAL, f1, f2);
+
+    // the case where part is the outer and partsupp is the inner.
+    List<Column[]> pairs = PlannerUtil.getJoinKeyPairs(equiJoinQual, outerSchema,  innerSchema, false);
+    assertEquals(1, pairs.size());
+    assertEquals("employee.id1", pairs.get(0)[0].getQualifiedName());
+    assertEquals("people.fid1", pairs.get(0)[1].getQualifiedName());
+
+    // after exchange of outer and inner
+    pairs = PlannerUtil.getJoinKeyPairs(equiJoinQual, innerSchema, outerSchema, false);
+    assertEquals("people.fid1", pairs.get(0)[0].getQualifiedName());
+    assertEquals("employee.id1", pairs.get(0)[1].getQualifiedName());
+
+    // composited join key test
+    EvalNode joinQual2 = new BinaryEval(EvalType.EQUAL, f3, f4);
+    EvalNode compositedJoinQual = new BinaryEval(EvalType.AND, equiJoinQual, joinQual2);
+    pairs = PlannerUtil.getJoinKeyPairs(compositedJoinQual, outerSchema,  innerSchema, false);
+    assertEquals(2, pairs.size());
+    assertEquals("employee.id1", pairs.get(0)[0].getQualifiedName());
+    assertEquals("people.fid1", pairs.get(0)[1].getQualifiedName());
+    assertEquals("employee.id2", pairs.get(1)[0].getQualifiedName());
+    assertEquals("people.fid2", pairs.get(1)[1].getQualifiedName());
+
+    // after exchange of outer and inner
+    pairs = PlannerUtil.getJoinKeyPairs(compositedJoinQual, innerSchema,  outerSchema, false);
+    assertEquals(2, pairs.size());
+    assertEquals("people.fid1", pairs.get(0)[0].getQualifiedName());
+    assertEquals("employee.id1", pairs.get(0)[1].getQualifiedName());
+    assertEquals("people.fid2", pairs.get(1)[0].getQualifiedName());
+    assertEquals("employee.id2", pairs.get(1)[1].getQualifiedName());
+
+    // Theta join (f1 <= f2)
+    EvalNode thetaJoinQual = new BinaryEval(EvalType.LEQ, f1, f2);
+    pairs = PlannerUtil.getJoinKeyPairs(thetaJoinQual, outerSchema,  innerSchema, true);
+    assertEquals(1, pairs.size());
+    assertEquals("employee.id1", pairs.get(0)[0].getQualifiedName());
+    assertEquals("people.fid1", pairs.get(0)[1].getQualifiedName());
+
+    // Composite Theta join (f1 <= f2 AND f3 = f4)
+    EvalNode compositeThetaJoin = new BinaryEval(EvalType.AND, thetaJoinQual, joinQual2);
+    pairs = PlannerUtil.getJoinKeyPairs(compositeThetaJoin, outerSchema,  innerSchema, true);
+    assertEquals(2, pairs.size());
+    assertEquals("employee.id1", pairs.get(0)[0].getQualifiedName());
+    assertEquals("people.fid1", pairs.get(0)[1].getQualifiedName());
+    assertEquals("employee.id2", pairs.get(1)[0].getQualifiedName());
+    assertEquals("people.fid2", pairs.get(1)[1].getQualifiedName());
+  }
+
+  @Test
+  public final void testGetSortKeysFromJoinQual() {
+    Schema outerSchema = new Schema();
+    outerSchema.addColumn("employee.id1", CatalogUtil.newSimpleDataType(Type.INT4));
+    outerSchema.addColumn("employee.id2", CatalogUtil.newSimpleDataType(Type.INT4));
+    Schema innerSchema = new Schema();
+    innerSchema.addColumn("people.fid1", CatalogUtil.newSimpleDataType(Type.INT4));
+    innerSchema.addColumn("people.fid2", CatalogUtil.newSimpleDataType(Type.INT4));
+
+    FieldEval f1 = new FieldEval("employee.id1", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f2 = new FieldEval("people.fid1", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f3 = new FieldEval("employee.id2", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f4 = new FieldEval("people.fid2", CatalogUtil.newSimpleDataType(Type.INT4));
+
+    EvalNode joinQual = new BinaryEval(EvalType.EQUAL, f1, f2);
+    SortSpec[][] sortSpecs = PlannerUtil.getSortKeysFromJoinQual(joinQual, outerSchema, innerSchema);
+    assertEquals(2, sortSpecs.length);
+    assertEquals(1, sortSpecs[0].length);
+    assertEquals(1, sortSpecs[1].length);
+    assertEquals(outerSchema.getColumn("id1"), sortSpecs[0][0].getSortKey());
+    assertEquals(innerSchema.getColumn("fid1"), sortSpecs[1][0].getSortKey());
+
+    // tests for composited join key
+    EvalNode joinQual2 = new BinaryEval(EvalType.EQUAL, f3, f4);
+    EvalNode compositedJoinQual = new BinaryEval(EvalType.AND, joinQual, joinQual2);
+
+    sortSpecs = PlannerUtil.getSortKeysFromJoinQual(compositedJoinQual, outerSchema, innerSchema);
+    assertEquals(2, sortSpecs.length);
+    assertEquals(2, sortSpecs[0].length);
+    assertEquals(2, sortSpecs[1].length);
+    assertEquals(outerSchema.getColumn("id1"), sortSpecs[0][0].getSortKey());
+    assertEquals(outerSchema.getColumn("id2"), sortSpecs[0][1].getSortKey());
+    assertEquals(innerSchema.getColumn("fid1"), sortSpecs[1][0].getSortKey());
+    assertEquals(innerSchema.getColumn("fid2"), sortSpecs[1][1].getSortKey());
+  }
+
+  @Test
+  public final void testComparatorsFromJoinQual() {
+    Schema outerSchema = new Schema();
+    outerSchema.addColumn("employee.id1", CatalogUtil.newSimpleDataType(Type.INT4));
+    outerSchema.addColumn("employee.id2", CatalogUtil.newSimpleDataType(Type.INT4));
+    Schema innerSchema = new Schema();
+    innerSchema.addColumn("people.fid1", CatalogUtil.newSimpleDataType(Type.INT4));
+    innerSchema.addColumn("people.fid2", CatalogUtil.newSimpleDataType(Type.INT4));
+
+    FieldEval f1 = new FieldEval("employee.id1", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f2 = new FieldEval("people.fid1", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f3 = new FieldEval("employee.id2", CatalogUtil.newSimpleDataType(Type.INT4));
+    FieldEval f4 = new FieldEval("people.fid2", CatalogUtil.newSimpleDataType(Type.INT4));
+
+    EvalNode joinQual = new BinaryEval(EvalType.EQUAL, f1, f2);
+    TupleComparator [] comparators = PlannerUtil.getComparatorsFromJoinQual(joinQual, outerSchema, innerSchema);
+
+    Tuple t1 = new VTuple(2);
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+
+    Tuple t2 = new VTuple(2);
+    t2.put(0, DatumFactory.createInt4(2));
+    t2.put(1, DatumFactory.createInt4(3));
+
+    TupleComparator outerComparator = comparators[0];
+    assertTrue(outerComparator.compare(t1, t2) < 0);
+    assertTrue(outerComparator.compare(t2, t1) > 0);
+
+    TupleComparator innerComparator = comparators[1];
+    assertTrue(innerComparator.compare(t1, t2) < 0);
+    assertTrue(innerComparator.compare(t2, t1) > 0);
+
+    // tests for composited join key
+    EvalNode joinQual2 = new BinaryEval(EvalType.EQUAL, f3, f4);
+    EvalNode compositedJoinQual = new BinaryEval(EvalType.AND, joinQual, joinQual2);
+    comparators = PlannerUtil.getComparatorsFromJoinQual(compositedJoinQual, outerSchema, innerSchema);
+
+    outerComparator = comparators[0];
+    assertTrue(outerComparator.compare(t1, t2) < 0);
+    assertTrue(outerComparator.compare(t2, t1) > 0);
+
+    innerComparator = comparators[1];
+    assertTrue(innerComparator.compare(t1, t2) < 0);
+    assertTrue(innerComparator.compare(t2, t1) > 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleDirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleDirectedGraph.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleDirectedGraph.java
new file mode 100644
index 0000000..607a6ae
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleDirectedGraph.java
@@ -0,0 +1,79 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.engine.planner.graph.DirectedGraphVisitor;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
+import org.junit.Test;
+
+import java.util.Stack;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestSimpleDirectedGraph {
+
+  @Test
+  public final void test() {
+    SimpleDirectedGraph<String, Integer> graph = new SimpleDirectedGraph<String, Integer>();
+
+    //     root
+    //     /  \
+    // (1)/    \ (2)
+    //   /      \
+    // child1  child2
+    //           / \
+    //       (3)/   \(4)
+    //         /     \
+    //    child3   child4
+    //
+    String root = "root";
+    String child1 = "child1";
+    String child2 = "child2";
+    String child3 = "child3";
+    String child4 = "child4";
+
+    graph.addEdge(child1, root, 1);
+    graph.addEdge(child2, root, 2);
+    graph.addEdge(child3, child2, 3);
+    graph.addEdge(child4, child2, 4);
+
+    assertEquals(4, graph.getEdgeNum());
+    assertEquals(4, graph.getEdgesAll().size());
+
+    // tree features
+    assertTrue(graph.isRoot(root));
+    assertFalse(graph.isLeaf(root));
+
+    assertEquals(2, graph.getChildCount(root));
+    assertEquals(2, graph.getChildCount(child2));
+
+    // visitor
+    graph.accept(root, new Visitor());
+  }
+
+  private class Visitor implements DirectedGraphVisitor<String> {
+
+    @Override
+    public void visit(Stack<String> stack, String s) {
+      System.out.println("===> " + s);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleUndirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleUndirectedGraph.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleUndirectedGraph.java
new file mode 100644
index 0000000..1cd2389
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestSimpleUndirectedGraph.java
@@ -0,0 +1,96 @@
+/**
+ * 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.planner;
+
+import com.google.common.collect.Sets;
+import org.apache.tajo.engine.planner.graph.SimpleUndirectedGraph;
+import org.apache.tajo.engine.planner.graph.UndirectedGraph;
+import org.junit.Test;
+
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+public class TestSimpleUndirectedGraph {
+
+  @Test
+  public final void test() {
+    UndirectedGraph<String, Integer> graph = new SimpleUndirectedGraph<String, Integer>();
+
+    //     root
+    //     /  \
+    // (1)/    \ (2)
+    //   /      \
+    // child1  child2
+    //           / \
+    //       (3)/   \(4)
+    //         /     \
+    //    child3   child4
+    //
+    String root = "root";
+    String child1 = "child1";
+    String child2 = "child2";
+    String child3 = "child3";
+    String child4 = "child4";
+
+    graph.addEdge(child1, root, 1);
+    graph.addEdge(child2, root, 2);
+    graph.addEdge(child3, child2, 3);
+    graph.addEdge(child4, child2, 4);
+
+    // for connected edges
+    assertNotNull(graph.getEdge(child1, root));
+    assertNotNull(graph.getEdge(root, child1));
+
+    assertNotNull(graph.getEdge(root, child2));
+    assertNotNull(graph.getEdge(child2, root));
+
+    assertNotNull(graph.getEdge(child2, child3));
+    assertNotNull(graph.getEdge(child3, child2));
+
+    assertNotNull(graph.getEdge(child2, child4));
+    assertNotNull(graph.getEdge(child4, child2));
+
+    // for not-connected edges
+    assertNull(graph.getEdge(root, child4));
+    assertNull(graph.getEdge(child4, root));
+
+    assertNull(graph.getEdge(root, child3));
+    assertNull(graph.getEdge(child3, root));
+
+    assertNull(graph.getEdge(child1, child2));
+    assertNull(graph.getEdge(child2, child1));
+
+    assertNull(graph.getEdge(child3, child4));
+    assertNull(graph.getEdge(child4, child3));
+
+    // number
+    assertEquals(4, graph.getEdgeNum());
+    assertEquals(4, graph.getEdgesAll().size());
+
+    assertEquals(2, graph.getDegree(root));
+    assertEquals(1, graph.getDegree(child1));
+    assertEquals(3, graph.getDegree(child2));
+    assertEquals(1, graph.getDegree(child3));
+    assertEquals(1, graph.getDegree(child4));
+
+    Set<Integer> edges = Sets.newHashSet(2, 3, 4);
+    assertEquals(edges, Sets.newHashSet(graph.getEdges(child2)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java
new file mode 100644
index 0000000..f4c114f
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java
@@ -0,0 +1,442 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.storage.VTuple;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestUniformRangePartition {
+  /**
+   * It verify overflow and increment.
+   */
+  @Test
+  public void testIncrement1() {
+    Schema schema = new Schema()
+        .addColumn("l_returnflag", Type.TEXT)
+        .addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec[] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("A"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createText("D"));
+    e.put(1, DatumFactory.createText("C"));
+
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+
+    UniformRangePartition partitioner = new UniformRangePartition(expected, sortSpecs);
+    assertEquals(12, partitioner.getTotalCardinality().intValue());
+
+    String [] result = new String[12];
+    result[0] = "AA";
+    result[1] = "AB";
+    result[2] = "AC";
+    result[3] = "BA";
+    result[4] = "BB";
+    result[5] = "BC";
+    result[6] = "CA";
+    result[7] = "CB";
+    result[8] = "CC";
+    result[9] = "DA";
+    result[10] = "DB";
+    result[11] = "DC";
+
+    Tuple end = partitioner.increment(s, 1, 1);
+    assertEquals("A", end.get(0).asChars());
+    assertEquals("B", end.get(1).asChars());
+    for (int i = 2; i < 11; i++ ) {
+      end = partitioner.increment(end, 1, 1);
+      assertEquals(result[i].charAt(0), end.get(0).asChars().charAt(0));
+      assertEquals(result[i].charAt(1), end.get(1).asChars().charAt(0));
+    }
+  }
+
+  /**
+   * It verify overflow with the number that exceeds the last digit.
+   */
+  @Test
+  public void testIncrement2() {
+    Schema schema = new Schema()
+        .addColumn("l_returnflag", Type.TEXT)
+        .addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("A"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createText("D"));
+    e.put(1, DatumFactory.createText("C"));
+
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+
+    UniformRangePartition partitioner = new UniformRangePartition(expected, sortSpecs);
+    assertEquals(12, partitioner.getTotalCardinality().intValue());
+
+    String [] result = new String[12];
+    result[0] = "AA";
+    result[1] = "AB";
+    result[2] = "AC";
+    result[3] = "BA";
+    result[4] = "BB";
+    result[5] = "BC";
+    result[6] = "CA";
+    result[7] = "CB";
+    result[8] = "CC";
+    result[9] = "DA";
+    result[10] = "DB";
+    result[11] = "DC";
+
+    Tuple end = partitioner.increment(s, 6, 1);
+    assertEquals("C", end.get(0).asChars());
+    assertEquals("A", end.get(1).asChars());
+    end = partitioner.increment(end, 5, 1);
+    assertEquals("D", end.get(0).asChars());
+    assertEquals("C", end.get(1).asChars());
+  }
+
+  /**
+   * It verify the case where two or more digits are overflow.
+   */
+  @Test
+  public void testIncrement3() {
+    Schema schema = new Schema()
+        .addColumn("l_returnflag", Type.TEXT)
+        .addColumn("l_linestatus", Type.TEXT)
+        .addColumn("final", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(3);
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("A"));
+    s.put(2, DatumFactory.createText("A"));
+    Tuple e = new VTuple(3);
+    e.put(0, DatumFactory.createText("D")); //  4
+    e.put(1, DatumFactory.createText("B")); //  2
+    e.put(2, DatumFactory.createText("C")); // x3 = 24
+
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+
+    UniformRangePartition partitioner = new UniformRangePartition(expected, sortSpecs);
+    assertEquals(24, partitioner.getTotalCardinality().intValue());
+
+    Tuple overflowBefore = partitioner.increment(s, 5, 2);
+    assertEquals("A", overflowBefore.get(0).asChars());
+    assertEquals("B", overflowBefore.get(1).asChars());
+    assertEquals("C", overflowBefore.get(2).asChars());
+    Tuple overflowed = partitioner.increment(overflowBefore, 1, 2);
+    assertEquals("B", overflowed.get(0).asChars());
+    assertEquals("A", overflowed.get(1).asChars());
+    assertEquals("A", overflowed.get(2).asChars());
+  }
+
+  @Test
+  public void testIncrement4() {
+    Schema schema = new Schema()
+        .addColumn("l_orderkey", Type.INT8)
+        .addColumn("l_linenumber", Type.INT8);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createInt8(10));
+    s.put(1, DatumFactory.createInt8(20));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createInt8(19));
+    e.put(1, DatumFactory.createInt8(39));
+
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+
+    UniformRangePartition partitioner = new UniformRangePartition(expected, sortSpecs);
+    assertEquals(200, partitioner.getTotalCardinality().longValue());
+
+    Tuple range2 = partitioner.increment(s, 100, 1);
+    assertEquals(15, range2.get(0).asInt4());
+    assertEquals(20, range2.get(1).asInt4());
+    Tuple range3 = partitioner.increment(range2, 99, 1);
+    assertEquals(19, range3.get(0).asInt4());
+    assertEquals(39, range3.get(1).asInt4());
+  }
+
+  @Test public void testIncrement5() {
+    Schema schema = new Schema()
+        .addColumn("l_orderkey", Type.INT8)
+        .addColumn("l_linenumber", Type.INT8)
+        .addColumn("final", Type.INT8);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(3);
+    s.put(0, DatumFactory.createInt8(1));
+    s.put(1, DatumFactory.createInt8(1));
+    s.put(2, DatumFactory.createInt8(1));
+    Tuple e = new VTuple(3);
+    e.put(0, DatumFactory.createInt8(4)); // 4
+    e.put(1, DatumFactory.createInt8(2)); // 2
+    e.put(2, DatumFactory.createInt8(3)); //x3 = 24
+
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+
+    UniformRangePartition partitioner = new UniformRangePartition(expected, sortSpecs);
+    assertEquals(24, partitioner.getTotalCardinality().longValue());
+
+    Tuple beforeOverflow = partitioner.increment(s, 5, 2);
+    assertEquals(1, beforeOverflow.get(0).asInt8());
+    assertEquals(2, beforeOverflow.get(1).asInt8());
+    assertEquals(3, beforeOverflow.get(2).asInt8());
+    Tuple overflow = partitioner.increment(beforeOverflow, 1, 2);
+    assertEquals(2, overflow.get(0).asInt8());
+    assertEquals(1, overflow.get(1).asInt8());
+    assertEquals(1, overflow.get(2).asInt8());
+  }
+
+  @Test
+  public void testIncrement6() {
+    Schema schema = new Schema()
+        .addColumn("l_orderkey", Type.FLOAT8)
+        .addColumn("l_linenumber", Type.FLOAT8)
+        .addColumn("final", Type.FLOAT8);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(3);
+    s.put(0, DatumFactory.createFloat8(1.1d));
+    s.put(1, DatumFactory.createFloat8(1.1d));
+    s.put(2, DatumFactory.createFloat8(1.1d));
+    Tuple e = new VTuple(3);
+    e.put(0, DatumFactory.createFloat8(4.1d)); // 4
+    e.put(1, DatumFactory.createFloat8(2.1d)); // 2
+    e.put(2, DatumFactory.createFloat8(3.1d)); //x3 = 24
+
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+
+    UniformRangePartition partitioner = new UniformRangePartition(expected, sortSpecs);
+    assertEquals(24, partitioner.getTotalCardinality().longValue());
+
+    Tuple beforeOverflow = partitioner.increment(s, 5, 2);
+    assertTrue(1.1d == beforeOverflow.get(0).asFloat8());
+    assertTrue(2.1d == beforeOverflow.get(1).asFloat8());
+    assertTrue(3.1d == beforeOverflow.get(2).asFloat8());
+    Tuple overflow = partitioner.increment(beforeOverflow, 1, 2);
+    assertTrue(2.1d == overflow.get(0).asFloat8());
+    assertTrue(1.1d == overflow.get(1).asFloat8());
+    assertTrue(1.1d == overflow.get(2).asFloat8());
+  }
+
+  @Test
+  public void testIncrement7() {
+    Schema schema = new Schema()
+        .addColumn("l_orderkey", Type.INET4)
+        .addColumn("l_linenumber", Type.INET4)
+        .addColumn("final", Type.INET4);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(3);
+    s.put(0, DatumFactory.createInet4("127.0.1.1"));
+    s.put(1, DatumFactory.createInet4("127.0.0.1"));
+    s.put(2, DatumFactory.createInet4("128.0.0.253"));
+    Tuple e = new VTuple(3);
+    e.put(0, DatumFactory.createInet4("127.0.1.4")); // 4
+    e.put(1, DatumFactory.createInet4("127.0.0.2")); // 2
+    e.put(2, DatumFactory.createInet4("128.0.0.255")); //x3 = 24
+
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+
+    UniformRangePartition partitioner = new UniformRangePartition(expected, sortSpecs);
+    assertEquals(24, partitioner.getTotalCardinality().longValue());
+
+    Tuple beforeOverflow = partitioner.increment(s, 5, 2);
+    assertTrue("127.0.1.1".equals(beforeOverflow.get(0).asChars()));
+    assertTrue("127.0.0.2".equals(beforeOverflow.get(1).asChars()));
+    assertTrue("128.0.0.255".equals(beforeOverflow.get(2).asChars()));
+    Tuple overflow = partitioner.increment(beforeOverflow, 1, 2);
+    assertTrue("127.0.1.2".equals(overflow.get(0).asChars()));
+    assertTrue("127.0.0.1".equals(overflow.get(1).asChars()));
+    assertTrue("128.0.0.253".equals(overflow.get(2).asChars()));
+  }
+
+  @Test
+  public void testPartition() {
+    Schema schema = new Schema();
+    schema.addColumn("l_returnflag", Type.TEXT);
+    schema.addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("F"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createText("R"));
+    e.put(1, DatumFactory.createText("O"));
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner
+        = new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(31);
+
+
+    TupleRange prev = null;
+    for (TupleRange r : ranges) {
+      if (prev == null) {
+        prev = r;
+      } else {
+        assertTrue(prev.compareTo(r) < 0);
+      }
+    }
+  }
+
+  @Test
+  public void testPartitionForOnePartNum() {
+    Schema schema = new Schema()
+        .addColumn("l_returnflag", Type.TEXT)
+        .addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("F"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createText("R"));
+    e.put(1, DatumFactory.createText("O"));
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner =
+        new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(1);
+
+    assertEquals(expected, ranges[0]);
+  }
+
+  @Test
+  public void testPartitionForOnePartNumWithOneOfTheValueNull() {
+    Schema schema = new Schema()
+        .addColumn("l_returnflag", Type.TEXT)
+        .addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createNullDatum());
+    s.put(1, DatumFactory.createText("F"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createText("R"));
+    e.put(1, DatumFactory.createNullDatum());
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner =
+        new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(1);
+
+    assertEquals(expected, ranges[0]);
+  }
+
+  @Test
+  public void testPartitionForOnePartNumWithBothValueNull() {
+    Schema schema = new Schema()
+        .addColumn("l_returnflag", Type.TEXT)
+        .addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createNullDatum());
+    s.put(1, DatumFactory.createNullDatum());
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createNullDatum());
+    e.put(1, DatumFactory.createNullDatum());
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner =
+        new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(1);
+
+    assertEquals(expected, ranges[0]);
+  }
+
+  @Test
+  public void testPartitionWithNull() {
+    Schema schema = new Schema();
+    schema.addColumn("l_returnflag", Type.TEXT);
+    schema.addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createNullDatum());
+    s.put(1, DatumFactory.createText("F"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createNullDatum());
+    e.put(1, DatumFactory.createText("O"));
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner
+        = new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(10);
+
+
+    TupleRange prev = null;
+    for (TupleRange r : ranges) {
+      if (prev == null) {
+        prev = r;
+      } else {
+        assertTrue(prev.compareTo(r) > 0);
+      }
+    }
+  }
+
+  @Test
+  public void testPartitionWithINET4() {
+    Schema schema = new Schema();
+    schema.addColumn("l_returnflag", Type.INET4);
+    schema.addColumn("l_linestatus", Type.INET4);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createInet4("127.0.1.10"));
+    s.put(1, DatumFactory.createInet4("127.0.2.10"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createInet4("127.0.1.20"));
+    e.put(1, DatumFactory.createInet4("127.0.2.20"));
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner
+        = new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(10);
+
+    TupleRange prev = null;
+    for (TupleRange r : ranges) {
+      if (prev == null) {
+        prev = r;
+      } else {
+        assertTrue(prev.compareTo(r) < 0);
+      }
+    }
+  }
+}


[15/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/js/jquery-ui.min.js
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/js/jquery-ui.min.js b/tajo-core/src/main/resources/webapps/static/js/jquery-ui.min.js
new file mode 100644
index 0000000..76a3285
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/static/js/jquery-ui.min.js
@@ -0,0 +1,5 @@
+/*! jQuery UI - v1.8.23 - 2012-08-15
+ * https://github.com/jquery/jquery-ui
+ * Includes: jquery.ui.core.js, jquery.ui.widget.js, jquery.ui.mouse.js, jquery.ui.draggable.js, jquery.ui.droppable.js, jquery.ui.resizable.js, jquery.ui.selectable.js, jquery.ui.sortable.js, jquery.effects.core.js, jquery.effects.blind.js, jquery.effects.bounce.js, jquery.effects.clip.js, jquery.effects.drop.js, jquery.effects.explode.js, jquery.effects.fade.js, jquery.effects.fold.js, jquery.effects.highlight.js, jquery.effects.pulsate.js, jquery.effects.scale.js, jquery.effects.shake.js, jquery.effects.slide.js, jquery.effects.transfer.js, jquery.ui.accordion.js, jquery.ui.autocomplete.js, jquery.ui.button.js, jquery.ui.datepicker.js, jquery.ui.dialog.js, jquery.ui.position.js, jquery.ui.progressbar.js, jquery.ui.slider.js, jquery.ui.tabs.js
+ * Copyright (c) 2012 AUTHORS.txt; Licensed MIT, GPL */
+(function(a,b){function c(b,c){var e=b.nodeName.toLowerCase();if("area"===e){var f=b.parentNode,g=f.name,h;return!b.href||!g||f.nodeName.toLowerCase()!=="map"?!1:(h=a("img[usemap=#"+g+"]")[0],!!h&&d(h))}return(/input|select|textarea|button|object/.test(e)?!b.disabled:"a"==e?b.href||c:c)&&d(b)}function d(b){return!a(b).parents().andSelf().filter(function(){return a.curCSS(this,"visibility")==="hidden"||a.expr.filters.hidden(this)}).length}a.ui=a.ui||{};if(a.ui.version)return;a.extend(a.ui,{version:"1.8.23",keyCode:{ALT:18,BACKSPACE:8,CAPS_LOCK:20,COMMA:188,COMMAND:91,COMMAND_LEFT:91,COMMAND_RIGHT:93,CONTROL:17,DELETE:46,DOWN:40,END:35,ENTER:13,ESCAPE:27,HOME:36,INSERT:45,LEFT:37,MENU:93,NUMPAD_ADD:107,NUMPAD_DECIMAL:110,NUMPAD_DIVIDE:111,NUMPAD_ENTER:108,NUMPAD_MULTIPLY:106,NUMPAD_SUBTRACT:109,PAGE_DOWN:34,PAGE_UP:33,PERIOD:190,RIGHT:39,SHIFT:16,SPACE:32,TAB:9,UP:38,WINDOWS:91}}),a.fn.extend({propAttr:a.fn.prop||a.fn.attr,_focus:a.fn.focus,focus:function(b,c){return typeof b=="number
 "?this.each(function(){var d=this;setTimeout(function(){a(d).focus(),c&&c.call(d)},b)}):this._focus.apply(this,arguments)},scrollParent:function(){var b;return a.browser.msie&&/(static|relative)/.test(this.css("position"))||/absolute/.test(this.css("position"))?b=this.parents().filter(function(){return/(relative|absolute|fixed)/.test(a.curCSS(this,"position",1))&&/(auto|scroll)/.test(a.curCSS(this,"overflow",1)+a.curCSS(this,"overflow-y",1)+a.curCSS(this,"overflow-x",1))}).eq(0):b=this.parents().filter(function(){return/(auto|scroll)/.test(a.curCSS(this,"overflow",1)+a.curCSS(this,"overflow-y",1)+a.curCSS(this,"overflow-x",1))}).eq(0),/fixed/.test(this.css("position"))||!b.length?a(document):b},zIndex:function(c){if(c!==b)return this.css("zIndex",c);if(this.length){var d=a(this[0]),e,f;while(d.length&&d[0]!==document){e=d.css("position");if(e==="absolute"||e==="relative"||e==="fixed"){f=parseInt(d.css("zIndex"),10);if(!isNaN(f)&&f!==0)return f}d=d.parent()}}return 0},disableSelectio
 n:function(){return this.bind((a.support.selectstart?"selectstart":"mousedown")+".ui-disableSelection",function(a){a.preventDefault()})},enableSelection:function(){return this.unbind(".ui-disableSelection")}}),a("<a>").outerWidth(1).jquery||a.each(["Width","Height"],function(c,d){function h(b,c,d,f){return a.each(e,function(){c-=parseFloat(a.curCSS(b,"padding"+this,!0))||0,d&&(c-=parseFloat(a.curCSS(b,"border"+this+"Width",!0))||0),f&&(c-=parseFloat(a.curCSS(b,"margin"+this,!0))||0)}),c}var e=d==="Width"?["Left","Right"]:["Top","Bottom"],f=d.toLowerCase(),g={innerWidth:a.fn.innerWidth,innerHeight:a.fn.innerHeight,outerWidth:a.fn.outerWidth,outerHeight:a.fn.outerHeight};a.fn["inner"+d]=function(c){return c===b?g["inner"+d].call(this):this.each(function(){a(this).css(f,h(this,c)+"px")})},a.fn["outer"+d]=function(b,c){return typeof b!="number"?g["outer"+d].call(this,b):this.each(function(){a(this).css(f,h(this,b,!0,c)+"px")})}}),a.extend(a.expr[":"],{data:a.expr.createPseudo?a.expr.cre
 atePseudo(function(b){return function(c){return!!a.data(c,b)}}):function(b,c,d){return!!a.data(b,d[3])},focusable:function(b){return c(b,!isNaN(a.attr(b,"tabindex")))},tabbable:function(b){var d=a.attr(b,"tabindex"),e=isNaN(d);return(e||d>=0)&&c(b,!e)}}),a(function(){var b=document.body,c=b.appendChild(c=document.createElement("div"));c.offsetHeight,a.extend(c.style,{minHeight:"100px",height:"auto",padding:0,borderWidth:0}),a.support.minHeight=c.offsetHeight===100,a.support.selectstart="onselectstart"in c,b.removeChild(c).style.display="none"}),a.curCSS||(a.curCSS=a.css),a.extend(a.ui,{plugin:{add:function(b,c,d){var e=a.ui[b].prototype;for(var f in d)e.plugins[f]=e.plugins[f]||[],e.plugins[f].push([c,d[f]])},call:function(a,b,c){var d=a.plugins[b];if(!d||!a.element[0].parentNode)return;for(var e=0;e<d.length;e++)a.options[d[e][0]]&&d[e][1].apply(a.element,c)}},contains:function(a,b){return document.compareDocumentPosition?a.compareDocumentPosition(b)&16:a!==b&&a.contains(b)},hasScr
 oll:function(b,c){if(a(b).css("overflow")==="hidden")return!1;var d=c&&c==="left"?"scrollLeft":"scrollTop",e=!1;return b[d]>0?!0:(b[d]=1,e=b[d]>0,b[d]=0,e)},isOverAxis:function(a,b,c){return a>b&&a<b+c},isOver:function(b,c,d,e,f,g){return a.ui.isOverAxis(b,d,f)&&a.ui.isOverAxis(c,e,g)}})})(jQuery),function(a,b){if(a.cleanData){var c=a.cleanData;a.cleanData=function(b){for(var d=0,e;(e=b[d])!=null;d++)try{a(e).triggerHandler("remove")}catch(f){}c(b)}}else{var d=a.fn.remove;a.fn.remove=function(b,c){return this.each(function(){return c||(!b||a.filter(b,[this]).length)&&a("*",this).add([this]).each(function(){try{a(this).triggerHandler("remove")}catch(b){}}),d.call(a(this),b,c)})}}a.widget=function(b,c,d){var e=b.split(".")[0],f;b=b.split(".")[1],f=e+"-"+b,d||(d=c,c=a.Widget),a.expr[":"][f]=function(c){return!!a.data(c,b)},a[e]=a[e]||{},a[e][b]=function(a,b){arguments.length&&this._createWidget(a,b)};var g=new c;g.options=a.extend(!0,{},g.options),a[e][b].prototype=a.extend(!0,g,{names
 pace:e,widgetName:b,widgetEventPrefix:a[e][b].prototype.widgetEventPrefix||b,widgetBaseClass:f},d),a.widget.bridge(b,a[e][b])},a.widget.bridge=function(c,d){a.fn[c]=function(e){var f=typeof e=="string",g=Array.prototype.slice.call(arguments,1),h=this;return e=!f&&g.length?a.extend.apply(null,[!0,e].concat(g)):e,f&&e.charAt(0)==="_"?h:(f?this.each(function(){var d=a.data(this,c),f=d&&a.isFunction(d[e])?d[e].apply(d,g):d;if(f!==d&&f!==b)return h=f,!1}):this.each(function(){var b=a.data(this,c);b?b.option(e||{})._init():a.data(this,c,new d(e,this))}),h)}},a.Widget=function(a,b){arguments.length&&this._createWidget(a,b)},a.Widget.prototype={widgetName:"widget",widgetEventPrefix:"",options:{disabled:!1},_createWidget:function(b,c){a.data(c,this.widgetName,this),this.element=a(c),this.options=a.extend(!0,{},this.options,this._getCreateOptions(),b);var d=this;this.element.bind("remove."+this.widgetName,function(){d.destroy()}),this._create(),this._trigger("create"),this._init()},_getCreate
 Options:function(){return a.metadata&&a.metadata.get(this.element[0])[this.widgetName]},_create:function(){},_init:function(){},destroy:function(){this.element.unbind("."+this.widgetName).removeData(this.widgetName),this.widget().unbind("."+this.widgetName).removeAttr("aria-disabled").removeClass(this.widgetBaseClass+"-disabled "+"ui-state-disabled")},widget:function(){return this.element},option:function(c,d){var e=c;if(arguments.length===0)return a.extend({},this.options);if(typeof c=="string"){if(d===b)return this.options[c];e={},e[c]=d}return this._setOptions(e),this},_setOptions:function(b){var c=this;return a.each(b,function(a,b){c._setOption(a,b)}),this},_setOption:function(a,b){return this.options[a]=b,a==="disabled"&&this.widget()[b?"addClass":"removeClass"](this.widgetBaseClass+"-disabled"+" "+"ui-state-disabled").attr("aria-disabled",b),this},enable:function(){return this._setOption("disabled",!1)},disable:function(){return this._setOption("disabled",!0)},_trigger:functio
 n(b,c,d){var e,f,g=this.options[b];d=d||{},c=a.Event(c),c.type=(b===this.widgetEventPrefix?b:this.widgetEventPrefix+b).toLowerCase(),c.target=this.element[0],f=c.originalEvent;if(f)for(e in f)e in c||(c[e]=f[e]);return this.element.trigger(c,d),!(a.isFunction(g)&&g.call(this.element[0],c,d)===!1||c.isDefaultPrevented())}}}(jQuery),function(a,b){var c=!1;a(document).mouseup(function(a){c=!1}),a.widget("ui.mouse",{options:{cancel:":input,option",distance:1,delay:0},_mouseInit:function(){var b=this;this.element.bind("mousedown."+this.widgetName,function(a){return b._mouseDown(a)}).bind("click."+this.widgetName,function(c){if(!0===a.data(c.target,b.widgetName+".preventClickEvent"))return a.removeData(c.target,b.widgetName+".preventClickEvent"),c.stopImmediatePropagation(),!1}),this.started=!1},_mouseDestroy:function(){this.element.unbind("."+this.widgetName),this._mouseMoveDelegate&&a(document).unbind("mousemove."+this.widgetName,this._mouseMoveDelegate).unbind("mouseup."+this.widgetNam
 e,this._mouseUpDelegate)},_mouseDown:function(b){if(c)return;this._mouseStarted&&this._mouseUp(b),this._mouseDownEvent=b;var d=this,e=b.which==1,f=typeof this.options.cancel=="string"&&b.target.nodeName?a(b.target).closest(this.options.cancel).length:!1;if(!e||f||!this._mouseCapture(b))return!0;this.mouseDelayMet=!this.options.delay,this.mouseDelayMet||(this._mouseDelayTimer=setTimeout(function(){d.mouseDelayMet=!0},this.options.delay));if(this._mouseDistanceMet(b)&&this._mouseDelayMet(b)){this._mouseStarted=this._mouseStart(b)!==!1;if(!this._mouseStarted)return b.preventDefault(),!0}return!0===a.data(b.target,this.widgetName+".preventClickEvent")&&a.removeData(b.target,this.widgetName+".preventClickEvent"),this._mouseMoveDelegate=function(a){return d._mouseMove(a)},this._mouseUpDelegate=function(a){return d._mouseUp(a)},a(document).bind("mousemove."+this.widgetName,this._mouseMoveDelegate).bind("mouseup."+this.widgetName,this._mouseUpDelegate),b.preventDefault(),c=!0,!0},_mouseMove
 :function(b){return!a.browser.msie||document.documentMode>=9||!!b.button?this._mouseStarted?(this._mouseDrag(b),b.preventDefault()):(this._mouseDistanceMet(b)&&this._mouseDelayMet(b)&&(this._mouseStarted=this._mouseStart(this._mouseDownEvent,b)!==!1,this._mouseStarted?this._mouseDrag(b):this._mouseUp(b)),!this._mouseStarted):this._mouseUp(b)},_mouseUp:function(b){return a(document).unbind("mousemove."+this.widgetName,this._mouseMoveDelegate).unbind("mouseup."+this.widgetName,this._mouseUpDelegate),this._mouseStarted&&(this._mouseStarted=!1,b.target==this._mouseDownEvent.target&&a.data(b.target,this.widgetName+".preventClickEvent",!0),this._mouseStop(b)),!1},_mouseDistanceMet:function(a){return Math.max(Math.abs(this._mouseDownEvent.pageX-a.pageX),Math.abs(this._mouseDownEvent.pageY-a.pageY))>=this.options.distance},_mouseDelayMet:function(a){return this.mouseDelayMet},_mouseStart:function(a){},_mouseDrag:function(a){},_mouseStop:function(a){},_mouseCapture:function(a){return!0}})}(j
 Query),function(a,b){a.widget("ui.draggable",a.ui.mouse,{widgetEventPrefix:"drag",options:{addClasses:!0,appendTo:"parent",axis:!1,connectToSortable:!1,containment:!1,cursor:"auto",cursorAt:!1,grid:!1,handle:!1,helper:"original",iframeFix:!1,opacity:!1,refreshPositions:!1,revert:!1,revertDuration:500,scope:"default",scroll:!0,scrollSensitivity:20,scrollSpeed:20,snap:!1,snapMode:"both",snapTolerance:20,stack:!1,zIndex:!1},_create:function(){this.options.helper=="original"&&!/^(?:r|a|f)/.test(this.element.css("position"))&&(this.element[0].style.position="relative"),this.options.addClasses&&this.element.addClass("ui-draggable"),this.options.disabled&&this.element.addClass("ui-draggable-disabled"),this._mouseInit()},destroy:function(){if(!this.element.data("draggable"))return;return this.element.removeData("draggable").unbind(".draggable").removeClass("ui-draggable ui-draggable-dragging ui-draggable-disabled"),this._mouseDestroy(),this},_mouseCapture:function(b){var c=this.options;retu
 rn this.helper||c.disabled||a(b.target).is(".ui-resizable-handle")?!1:(this.handle=this._getHandle(b),this.handle?(c.iframeFix&&a(c.iframeFix===!0?"iframe":c.iframeFix).each(function(){a('<div class="ui-draggable-iframeFix" style="background: #fff;"></div>').css({width:this.offsetWidth+"px",height:this.offsetHeight+"px",position:"absolute",opacity:"0.001",zIndex:1e3}).css(a(this).offset()).appendTo("body")}),!0):!1)},_mouseStart:function(b){var c=this.options;return this.helper=this._createHelper(b),this.helper.addClass("ui-draggable-dragging"),this._cacheHelperProportions(),a.ui.ddmanager&&(a.ui.ddmanager.current=this),this._cacheMargins(),this.cssPosition=this.helper.css("position"),this.scrollParent=this.helper.scrollParent(),this.offset=this.positionAbs=this.element.offset(),this.offset={top:this.offset.top-this.margins.top,left:this.offset.left-this.margins.left},a.extend(this.offset,{click:{left:b.pageX-this.offset.left,top:b.pageY-this.offset.top},parent:this._getParentOffset
 (),relative:this._getRelativeOffset()}),this.originalPosition=this.position=this._generatePosition(b),this.originalPageX=b.pageX,this.originalPageY=b.pageY,c.cursorAt&&this._adjustOffsetFromHelper(c.cursorAt),c.containment&&this._setContainment(),this._trigger("start",b)===!1?(this._clear(),!1):(this._cacheHelperProportions(),a.ui.ddmanager&&!c.dropBehaviour&&a.ui.ddmanager.prepareOffsets(this,b),this._mouseDrag(b,!0),a.ui.ddmanager&&a.ui.ddmanager.dragStart(this,b),!0)},_mouseDrag:function(b,c){this.position=this._generatePosition(b),this.positionAbs=this._convertPositionTo("absolute");if(!c){var d=this._uiHash();if(this._trigger("drag",b,d)===!1)return this._mouseUp({}),!1;this.position=d.position}if(!this.options.axis||this.options.axis!="y")this.helper[0].style.left=this.position.left+"px";if(!this.options.axis||this.options.axis!="x")this.helper[0].style.top=this.position.top+"px";return a.ui.ddmanager&&a.ui.ddmanager.drag(this,b),!1},_mouseStop:function(b){var c=!1;a.ui.ddmana
 ger&&!this.options.dropBehaviour&&(c=a.ui.ddmanager.drop(this,b)),this.dropped&&(c=this.dropped,this.dropped=!1);var d=this.element[0],e=!1;while(d&&(d=d.parentNode))d==document&&(e=!0);if(!e&&this.options.helper==="original")return!1;if(this.options.revert=="invalid"&&!c||this.options.revert=="valid"&&c||this.options.revert===!0||a.isFunction(this.options.revert)&&this.options.revert.call(this.element,c)){var f=this;a(this.helper).animate(this.originalPosition,parseInt(this.options.revertDuration,10),function(){f._trigger("stop",b)!==!1&&f._clear()})}else this._trigger("stop",b)!==!1&&this._clear();return!1},_mouseUp:function(b){return this.options.iframeFix===!0&&a("div.ui-draggable-iframeFix").each(function(){this.parentNode.removeChild(this)}),a.ui.ddmanager&&a.ui.ddmanager.dragStop(this,b),a.ui.mouse.prototype._mouseUp.call(this,b)},cancel:function(){return this.helper.is(".ui-draggable-dragging")?this._mouseUp({}):this._clear(),this},_getHandle:function(b){var c=!this.options.
 handle||!a(this.options.handle,this.element).length?!0:!1;return a(this.options.handle,this.element).find("*").andSelf().each(function(){this==b.target&&(c=!0)}),c},_createHelper:function(b){var c=this.options,d=a.isFunction(c.helper)?a(c.helper.apply(this.element[0],[b])):c.helper=="clone"?this.element.clone().removeAttr("id"):this.element;return d.parents("body").length||d.appendTo(c.appendTo=="parent"?this.element[0].parentNode:c.appendTo),d[0]!=this.element[0]&&!/(fixed|absolute)/.test(d.css("position"))&&d.css("position","absolute"),d},_adjustOffsetFromHelper:function(b){typeof b=="string"&&(b=b.split(" ")),a.isArray(b)&&(b={left:+b[0],top:+b[1]||0}),"left"in b&&(this.offset.click.left=b.left+this.margins.left),"right"in b&&(this.offset.click.left=this.helperProportions.width-b.right+this.margins.left),"top"in b&&(this.offset.click.top=b.top+this.margins.top),"bottom"in b&&(this.offset.click.top=this.helperProportions.height-b.bottom+this.margins.top)},_getParentOffset:function
 (){this.offsetParent=this.helper.offsetParent();var b=this.offsetParent.offset();this.cssPosition=="absolute"&&this.scrollParent[0]!=document&&a.ui.contains(this.scrollParent[0],this.offsetParent[0])&&(b.left+=this.scrollParent.scrollLeft(),b.top+=this.scrollParent.scrollTop());if(this.offsetParent[0]==document.body||this.offsetParent[0].tagName&&this.offsetParent[0].tagName.toLowerCase()=="html"&&a.browser.msie)b={top:0,left:0};return{top:b.top+(parseInt(this.offsetParent.css("borderTopWidth"),10)||0),left:b.left+(parseInt(this.offsetParent.css("borderLeftWidth"),10)||0)}},_getRelativeOffset:function(){if(this.cssPosition=="relative"){var a=this.element.position();return{top:a.top-(parseInt(this.helper.css("top"),10)||0)+this.scrollParent.scrollTop(),left:a.left-(parseInt(this.helper.css("left"),10)||0)+this.scrollParent.scrollLeft()}}return{top:0,left:0}},_cacheMargins:function(){this.margins={left:parseInt(this.element.css("marginLeft"),10)||0,top:parseInt(this.element.css("margi
 nTop"),10)||0,right:parseInt(this.element.css("marginRight"),10)||0,bottom:parseInt(this.element.css("marginBottom"),10)||0}},_cacheHelperProportions:function(){this.helperProportions={width:this.helper.outerWidth(),height:this.helper.outerHeight()}},_setContainment:function(){var b=this.options;b.containment=="parent"&&(b.containment=this.helper[0].parentNode);if(b.containment=="document"||b.containment=="window")this.containment=[b.containment=="document"?0:a(window).scrollLeft()-this.offset.relative.left-this.offset.parent.left,b.containment=="document"?0:a(window).scrollTop()-this.offset.relative.top-this.offset.parent.top,(b.containment=="document"?0:a(window).scrollLeft())+a(b.containment=="document"?document:window).width()-this.helperProportions.width-this.margins.left,(b.containment=="document"?0:a(window).scrollTop())+(a(b.containment=="document"?document:window).height()||document.body.parentNode.scrollHeight)-this.helperProportions.height-this.margins.top];if(!/^(documen
 t|window|parent)$/.test(b.containment)&&b.containment.constructor!=Array){var c=a(b.containment),d=c[0];if(!d)return;var e=c.offset(),f=a(d).css("overflow")!="hidden";this.containment=[(parseInt(a(d).css("borderLeftWidth"),10)||0)+(parseInt(a(d).css("paddingLeft"),10)||0),(parseInt(a(d).css("borderTopWidth"),10)||0)+(parseInt(a(d).css("paddingTop"),10)||0),(f?Math.max(d.scrollWidth,d.offsetWidth):d.offsetWidth)-(parseInt(a(d).css("borderLeftWidth"),10)||0)-(parseInt(a(d).css("paddingRight"),10)||0)-this.helperProportions.width-this.margins.left-this.margins.right,(f?Math.max(d.scrollHeight,d.offsetHeight):d.offsetHeight)-(parseInt(a(d).css("borderTopWidth"),10)||0)-(parseInt(a(d).css("paddingBottom"),10)||0)-this.helperProportions.height-this.margins.top-this.margins.bottom],this.relative_container=c}else b.containment.constructor==Array&&(this.containment=b.containment)},_convertPositionTo:function(b,c){c||(c=this.position);var d=b=="absolute"?1:-1,e=this.options,f=this.cssPosition
 =="absolute"&&(this.scrollParent[0]==document||!a.ui.contains(this.scrollParent[0],this.offsetParent[0]))?this.offsetParent:this.scrollParent,g=/(html|body)/i.test(f[0].tagName);return{top:c.top+this.offset.relative.top*d+this.offset.parent.top*d-(a.browser.safari&&a.browser.version<526&&this.cssPosition=="fixed"?0:(this.cssPosition=="fixed"?-this.scrollParent.scrollTop():g?0:f.scrollTop())*d),left:c.left+this.offset.relative.left*d+this.offset.parent.left*d-(a.browser.safari&&a.browser.version<526&&this.cssPosition=="fixed"?0:(this.cssPosition=="fixed"?-this.scrollParent.scrollLeft():g?0:f.scrollLeft())*d)}},_generatePosition:function(b){var c=this.options,d=this.cssPosition=="absolute"&&(this.scrollParent[0]==document||!a.ui.contains(this.scrollParent[0],this.offsetParent[0]))?this.offsetParent:this.scrollParent,e=/(html|body)/i.test(d[0].tagName),f=b.pageX,g=b.pageY;if(this.originalPosition){var h;if(this.containment){if(this.relative_container){var i=this.relative_container.offs
 et();h=[this.containment[0]+i.left,this.containment[1]+i.top,this.containment[2]+i.left,this.containment[3]+i.top]}else h=this.containment;b.pageX-this.offset.click.left<h[0]&&(f=h[0]+this.offset.click.left),b.pageY-this.offset.click.top<h[1]&&(g=h[1]+this.offset.click.top),b.pageX-this.offset.click.left>h[2]&&(f=h[2]+this.offset.click.left),b.pageY-this.offset.click.top>h[3]&&(g=h[3]+this.offset.click.top)}if(c.grid){var j=c.grid[1]?this.originalPageY+Math.round((g-this.originalPageY)/c.grid[1])*c.grid[1]:this.originalPageY;g=h?j-this.offset.click.top<h[1]||j-this.offset.click.top>h[3]?j-this.offset.click.top<h[1]?j+c.grid[1]:j-c.grid[1]:j:j;var k=c.grid[0]?this.originalPageX+Math.round((f-this.originalPageX)/c.grid[0])*c.grid[0]:this.originalPageX;f=h?k-this.offset.click.left<h[0]||k-this.offset.click.left>h[2]?k-this.offset.click.left<h[0]?k+c.grid[0]:k-c.grid[0]:k:k}}return{top:g-this.offset.click.top-this.offset.relative.top-this.offset.parent.top+(a.browser.safari&&a.browser.v
 ersion<526&&this.cssPosition=="fixed"?0:this.cssPosition=="fixed"?-this.scrollParent.scrollTop():e?0:d.scrollTop()),left:f-this.offset.click.left-this.offset.relative.left-this.offset.parent.left+(a.browser.safari&&a.browser.version<526&&this.cssPosition=="fixed"?0:this.cssPosition=="fixed"?-this.scrollParent.scrollLeft():e?0:d.scrollLeft())}},_clear:function(){this.helper.removeClass("ui-draggable-dragging"),this.helper[0]!=this.element[0]&&!this.cancelHelperRemoval&&this.helper.remove(),this.helper=null,this.cancelHelperRemoval=!1},_trigger:function(b,c,d){return d=d||this._uiHash(),a.ui.plugin.call(this,b,[c,d]),b=="drag"&&(this.positionAbs=this._convertPositionTo("absolute")),a.Widget.prototype._trigger.call(this,b,c,d)},plugins:{},_uiHash:function(a){return{helper:this.helper,position:this.position,originalPosition:this.originalPosition,offset:this.positionAbs}}}),a.extend(a.ui.draggable,{version:"1.8.23"}),a.ui.plugin.add("draggable","connectToSortable",{start:function(b,c){va
 r d=a(this).data("draggable"),e=d.options,f=a.extend({},c,{item:d.element});d.sortables=[],a(e.connectToSortable).each(function(){var c=a.data(this,"sortable");c&&!c.options.disabled&&(d.sortables.push({instance:c,shouldRevert:c.options.revert}),c.refreshPositions(),c._trigger("activate",b,f))})},stop:function(b,c){var d=a(this).data("draggable"),e=a.extend({},c,{item:d.element});a.each(d.sortables,function(){this.instance.isOver?(this.instance.isOver=0,d.cancelHelperRemoval=!0,this.instance.cancelHelperRemoval=!1,this.shouldRevert&&(this.instance.options.revert=!0),this.instance._mouseStop(b),this.instance.options.helper=this.instance.options._helper,d.options.helper=="original"&&this.instance.currentItem.css({top:"auto",left:"auto"})):(this.instance.cancelHelperRemoval=!1,this.instance._trigger("deactivate",b,e))})},drag:function(b,c){var d=a(this).data("draggable"),e=this,f=function(b){var c=this.offset.click.top,d=this.offset.click.left,e=this.positionAbs.top,f=this.positionAbs.
 left,g=b.height,h=b.width,i=b.top,j=b.left;return a.ui.isOver(e+c,f+d,i,j,g,h)};a.each(d.sortables,function(f){this.instance.positionAbs=d.positionAbs,this.instance.helperProportions=d.helperProportions,this.instance.offset.click=d.offset.click,this.instance._intersectsWith(this.instance.containerCache)?(this.instance.isOver||(this.instance.isOver=1,this.instance.currentItem=a(e).clone().removeAttr("id").appendTo(this.instance.element).data("sortable-item",!0),this.instance.options._helper=this.instance.options.helper,this.instance.options.helper=function(){return c.helper[0]},b.target=this.instance.currentItem[0],this.instance._mouseCapture(b,!0),this.instance._mouseStart(b,!0,!0),this.instance.offset.click.top=d.offset.click.top,this.instance.offset.click.left=d.offset.click.left,this.instance.offset.parent.left-=d.offset.parent.left-this.instance.offset.parent.left,this.instance.offset.parent.top-=d.offset.parent.top-this.instance.offset.parent.top,d._trigger("toSortable",b),d.dr
 opped=this.instance.element,d.currentItem=d.element,this.instance.fromOutside=d),this.instance.currentItem&&this.instance._mouseDrag(b)):this.instance.isOver&&(this.instance.isOver=0,this.instance.cancelHelperRemoval=!0,this.instance.options.revert=!1,this.instance._trigger("out",b,this.instance._uiHash(this.instance)),this.instance._mouseStop(b,!0),this.instance.options.helper=this.instance.options._helper,this.instance.currentItem.remove(),this.instance.placeholder&&this.instance.placeholder.remove(),d._trigger("fromSortable",b),d.dropped=!1)})}}),a.ui.plugin.add("draggable","cursor",{start:function(b,c){var d=a("body"),e=a(this).data("draggable").options;d.css("cursor")&&(e._cursor=d.css("cursor")),d.css("cursor",e.cursor)},stop:function(b,c){var d=a(this).data("draggable").options;d._cursor&&a("body").css("cursor",d._cursor)}}),a.ui.plugin.add("draggable","opacity",{start:function(b,c){var d=a(c.helper),e=a(this).data("draggable").options;d.css("opacity")&&(e._opacity=d.css("opa
 city")),d.css("opacity",e.opacity)},stop:function(b,c){var d=a(this).data("draggable").options;d._opacity&&a(c.helper).css("opacity",d._opacity)}}),a.ui.plugin.add("draggable","scroll",{start:function(b,c){var d=a(this).data("draggable");d.scrollParent[0]!=document&&d.scrollParent[0].tagName!="HTML"&&(d.overflowOffset=d.scrollParent.offset())},drag:function(b,c){var d=a(this).data("draggable"),e=d.options,f=!1;if(d.scrollParent[0]!=document&&d.scrollParent[0].tagName!="HTML"){if(!e.axis||e.axis!="x")d.overflowOffset.top+d.scrollParent[0].offsetHeight-b.pageY<e.scrollSensitivity?d.scrollParent[0].scrollTop=f=d.scrollParent[0].scrollTop+e.scrollSpeed:b.pageY-d.overflowOffset.top<e.scrollSensitivity&&(d.scrollParent[0].scrollTop=f=d.scrollParent[0].scrollTop-e.scrollSpeed);if(!e.axis||e.axis!="y")d.overflowOffset.left+d.scrollParent[0].offsetWidth-b.pageX<e.scrollSensitivity?d.scrollParent[0].scrollLeft=f=d.scrollParent[0].scrollLeft+e.scrollSpeed:b.pageX-d.overflowOffset.left<e.scroll
 Sensitivity&&(d.scrollParent[0].scrollLeft=f=d.scrollParent[0].scrollLeft-e.scrollSpeed)}else{if(!e.axis||e.axis!="x")b.pageY-a(document).scrollTop()<e.scrollSensitivity?f=a(document).scrollTop(a(document).scrollTop()-e.scrollSpeed):a(window).height()-(b.pageY-a(document).scrollTop())<e.scrollSensitivity&&(f=a(document).scrollTop(a(document).scrollTop()+e.scrollSpeed));if(!e.axis||e.axis!="y")b.pageX-a(document).scrollLeft()<e.scrollSensitivity?f=a(document).scrollLeft(a(document).scrollLeft()-e.scrollSpeed):a(window).width()-(b.pageX-a(document).scrollLeft())<e.scrollSensitivity&&(f=a(document).scrollLeft(a(document).scrollLeft()+e.scrollSpeed))}f!==!1&&a.ui.ddmanager&&!e.dropBehaviour&&a.ui.ddmanager.prepareOffsets(d,b)}}),a.ui.plugin.add("draggable","snap",{start:function(b,c){var d=a(this).data("draggable"),e=d.options;d.snapElements=[],a(e.snap.constructor!=String?e.snap.items||":data(draggable)":e.snap).each(function(){var b=a(this),c=b.offset();this!=d.element[0]&&d.snapEleme
 nts.push({item:this,width:b.outerWidth(),height:b.outerHeight(),top:c.top,left:c.left})})},drag:function(b,c){var d=a(this).data("draggable"),e=d.options,f=e.snapTolerance,g=c.offset.left,h=g+d.helperProportions.width,i=c.offset.top,j=i+d.helperProportions.height;for(var k=d.snapElements.length-1;k>=0;k--){var l=d.snapElements[k].left,m=l+d.snapElements[k].width,n=d.snapElements[k].top,o=n+d.snapElements[k].height;if(!(l-f<g&&g<m+f&&n-f<i&&i<o+f||l-f<g&&g<m+f&&n-f<j&&j<o+f||l-f<h&&h<m+f&&n-f<i&&i<o+f||l-f<h&&h<m+f&&n-f<j&&j<o+f)){d.snapElements[k].snapping&&d.options.snap.release&&d.options.snap.release.call(d.element,b,a.extend(d._uiHash(),{snapItem:d.snapElements[k].item})),d.snapElements[k].snapping=!1;continue}if(e.snapMode!="inner"){var p=Math.abs(n-j)<=f,q=Math.abs(o-i)<=f,r=Math.abs(l-h)<=f,s=Math.abs(m-g)<=f;p&&(c.position.top=d._convertPositionTo("relative",{top:n-d.helperProportions.height,left:0}).top-d.margins.top),q&&(c.position.top=d._convertPositionTo("relative",{top:
 o,left:0}).top-d.margins.top),r&&(c.position.left=d._convertPositionTo("relative",{top:0,left:l-d.helperProportions.width}).left-d.margins.left),s&&(c.position.left=d._convertPositionTo("relative",{top:0,left:m}).left-d.margins.left)}var t=p||q||r||s;if(e.snapMode!="outer"){var p=Math.abs(n-i)<=f,q=Math.abs(o-j)<=f,r=Math.abs(l-g)<=f,s=Math.abs(m-h)<=f;p&&(c.position.top=d._convertPositionTo("relative",{top:n,left:0}).top-d.margins.top),q&&(c.position.top=d._convertPositionTo("relative",{top:o-d.helperProportions.height,left:0}).top-d.margins.top),r&&(c.position.left=d._convertPositionTo("relative",{top:0,left:l}).left-d.margins.left),s&&(c.position.left=d._convertPositionTo("relative",{top:0,left:m-d.helperProportions.width}).left-d.margins.left)}!d.snapElements[k].snapping&&(p||q||r||s||t)&&d.options.snap.snap&&d.options.snap.snap.call(d.element,b,a.extend(d._uiHash(),{snapItem:d.snapElements[k].item})),d.snapElements[k].snapping=p||q||r||s||t}}}),a.ui.plugin.add("draggable","stac
 k",{start:function(b,c){var d=a(this).data("draggable").options,e=a.makeArray(a(d.stack)).sort(function(b,c){return(parseInt(a(b).css("zIndex"),10)||0)-(parseInt(a(c).css("zIndex"),10)||0)});if(!e.length)return;var f=parseInt(e[0].style.zIndex)||0;a(e).each(function(a){this.style.zIndex=f+a}),this[0].style.zIndex=f+e.length}}),a.ui.plugin.add("draggable","zIndex",{start:function(b,c){var d=a(c.helper),e=a(this).data("draggable").options;d.css("zIndex")&&(e._zIndex=d.css("zIndex")),d.css("zIndex",e.zIndex)},stop:function(b,c){var d=a(this).data("draggable").options;d._zIndex&&a(c.helper).css("zIndex",d._zIndex)}})}(jQuery),function(a,b){a.widget("ui.droppable",{widgetEventPrefix:"drop",options:{accept:"*",activeClass:!1,addClasses:!0,greedy:!1,hoverClass:!1,scope:"default",tolerance:"intersect"},_create:function(){var b=this.options,c=b.accept;this.isover=0,this.isout=1,this.accept=a.isFunction(c)?c:function(a){return a.is(c)},this.proportions={width:this.element[0].offsetWidth,heigh
 t:this.element[0].offsetHeight},a.ui.ddmanager.droppables[b.scope]=a.ui.ddmanager.droppables[b.scope]||[],a.ui.ddmanager.droppables[b.scope].push(this),b.addClasses&&this.element.addClass("ui-droppable")},destroy:function(){var b=a.ui.ddmanager.droppables[this.options.scope];for(var c=0;c<b.length;c++)b[c]==this&&b.splice(c,1);return this.element.removeClass("ui-droppable ui-droppable-disabled").removeData("droppable").unbind(".droppable"),this},_setOption:function(b,c){b=="accept"&&(this.accept=a.isFunction(c)?c:function(a){return a.is(c)}),a.Widget.prototype._setOption.apply(this,arguments)},_activate:function(b){var c=a.ui.ddmanager.current;this.options.activeClass&&this.element.addClass(this.options.activeClass),c&&this._trigger("activate",b,this.ui(c))},_deactivate:function(b){var c=a.ui.ddmanager.current;this.options.activeClass&&this.element.removeClass(this.options.activeClass),c&&this._trigger("deactivate",b,this.ui(c))},_over:function(b){var c=a.ui.ddmanager.current;if(!c|
 |(c.currentItem||c.element)[0]==this.element[0])return;this.accept.call(this.element[0],c.currentItem||c.element)&&(this.options.hoverClass&&this.element.addClass(this.options.hoverClass),this._trigger("over",b,this.ui(c)))},_out:function(b){var c=a.ui.ddmanager.current;if(!c||(c.currentItem||c.element)[0]==this.element[0])return;this.accept.call(this.element[0],c.currentItem||c.element)&&(this.options.hoverClass&&this.element.removeClass(this.options.hoverClass),this._trigger("out",b,this.ui(c)))},_drop:function(b,c){var d=c||a.ui.ddmanager.current;if(!d||(d.currentItem||d.element)[0]==this.element[0])return!1;var e=!1;return this.element.find(":data(droppable)").not(".ui-draggable-dragging").each(function(){var b=a.data(this,"droppable");if(b.options.greedy&&!b.options.disabled&&b.options.scope==d.options.scope&&b.accept.call(b.element[0],d.currentItem||d.element)&&a.ui.intersect(d,a.extend(b,{offset:b.element.offset()}),b.options.tolerance))return e=!0,!1}),e?!1:this.accept.call(
 this.element[0],d.currentItem||d.element)?(this.options.activeClass&&this.element.removeClass(this.options.activeClass),this.options.hoverClass&&this.element.removeClass(this.options.hoverClass),this._trigger("drop",b,this.ui(d)),this.element):!1},ui:function(a){return{draggable:a.currentItem||a.element,helper:a.helper,position:a.position,offset:a.positionAbs}}}),a.extend(a.ui.droppable,{version:"1.8.23"}),a.ui.intersect=function(b,c,d){if(!c.offset)return!1;var e=(b.positionAbs||b.position.absolute).left,f=e+b.helperProportions.width,g=(b.positionAbs||b.position.absolute).top,h=g+b.helperProportions.height,i=c.offset.left,j=i+c.proportions.width,k=c.offset.top,l=k+c.proportions.height;switch(d){case"fit":return i<=e&&f<=j&&k<=g&&h<=l;case"intersect":return i<e+b.helperProportions.width/2&&f-b.helperProportions.width/2<j&&k<g+b.helperProportions.height/2&&h-b.helperProportions.height/2<l;case"pointer":var m=(b.positionAbs||b.position.absolute).left+(b.clickOffset||b.offset.click).le
 ft,n=(b.positionAbs||b.position.absolute).top+(b.clickOffset||b.offset.click).top,o=a.ui.isOver(n,m,k,i,c.proportions.height,c.proportions.width);return o;case"touch":return(g>=k&&g<=l||h>=k&&h<=l||g<k&&h>l)&&(e>=i&&e<=j||f>=i&&f<=j||e<i&&f>j);default:return!1}},a.ui.ddmanager={current:null,droppables:{"default":[]},prepareOffsets:function(b,c){var d=a.ui.ddmanager.droppables[b.options.scope]||[],e=c?c.type:null,f=(b.currentItem||b.element).find(":data(droppable)").andSelf();g:for(var h=0;h<d.length;h++){if(d[h].options.disabled||b&&!d[h].accept.call(d[h].element[0],b.currentItem||b.element))continue;for(var i=0;i<f.length;i++)if(f[i]==d[h].element[0]){d[h].proportions.height=0;continue g}d[h].visible=d[h].element.css("display")!="none";if(!d[h].visible)continue;e=="mousedown"&&d[h]._activate.call(d[h],c),d[h].offset=d[h].element.offset(),d[h].proportions={width:d[h].element[0].offsetWidth,height:d[h].element[0].offsetHeight}}},drop:function(b,c){var d=!1;return a.each(a.ui.ddmanage
 r.droppables[b.options.scope]||[],function(){if(!this.options)return;!this.options.disabled&&this.visible&&a.ui.intersect(b,this,this.options.tolerance)&&(d=this._drop.call(this,c)||d),!this.options.disabled&&this.visible&&this.accept.call(this.element[0],b.currentItem||b.element)&&(this.isout=1,this.isover=0,this._deactivate.call(this,c))}),d},dragStart:function(b,c){b.element.parents(":not(body,html)").bind("scroll.droppable",function(){b.options.refreshPositions||a.ui.ddmanager.prepareOffsets(b,c)})},drag:function(b,c){b.options.refreshPositions&&a.ui.ddmanager.prepareOffsets(b,c),a.each(a.ui.ddmanager.droppables[b.options.scope]||[],function(){if(this.options.disabled||this.greedyChild||!this.visible)return;var d=a.ui.intersect(b,this,this.options.tolerance),e=!d&&this.isover==1?"isout":d&&this.isover==0?"isover":null;if(!e)return;var f;if(this.options.greedy){var g=this.element.parents(":data(droppable):eq(0)");g.length&&(f=a.data(g[0],"droppable"),f.greedyChild=e=="isover"?1:0
 )}f&&e=="isover"&&(f.isover=0,f.isout=1,f._out.call(f,c)),this[e]=1,this[e=="isout"?"isover":"isout"]=0,this[e=="isover"?"_over":"_out"].call(this,c),f&&e=="isout"&&(f.isout=0,f.isover=1,f._over.call(f,c))})},dragStop:function(b,c){b.element.parents(":not(body,html)").unbind("scroll.droppable"),b.options.refreshPositions||a.ui.ddmanager.prepareOffsets(b,c)}}}(jQuery),function(a,b){a.widget("ui.resizable",a.ui.mouse,{widgetEventPrefix:"resize",options:{alsoResize:!1,animate:!1,animateDuration:"slow",animateEasing:"swing",aspectRatio:!1,autoHide:!1,containment:!1,ghost:!1,grid:!1,handles:"e,s,se",helper:!1,maxHeight:null,maxWidth:null,minHeight:10,minWidth:10,zIndex:1e3},_create:function(){var b=this,c=this.options;this.element.addClass("ui-resizable"),a.extend(this,{_aspectRatio:!!c.aspectRatio,aspectRatio:c.aspectRatio,originalElement:this.element,_proportionallyResizeElements:[],_helper:c.helper||c.ghost||c.animate?c.helper||"ui-resizable-helper":null}),this.element[0].nodeName.mat
 ch(/canvas|textarea|input|select|button|img/i)&&(this.element.wrap(a('<div class="ui-wrapper" style="overflow: hidden;"></div>').css({position:this.element.css("position"),width:this.element.outerWidth(),height:this.element.outerHeight(),top:this.element.css("top"),left:this.element.css("left")})),this.element=this.element.parent().data("resizable",this.element.data("resizable")),this.elementIsWrapper=!0,this.element.css({marginLeft:this.originalElement.css("marginLeft"),marginTop:this.originalElement.css("marginTop"),marginRight:this.originalElement.css("marginRight"),marginBottom:this.originalElement.css("marginBottom")}),this.originalElement.css({marginLeft:0,marginTop:0,marginRight:0,marginBottom:0}),this.originalResizeStyle=this.originalElement.css("resize"),this.originalElement.css("resize","none"),this._proportionallyResizeElements.push(this.originalElement.css({position:"static",zoom:1,display:"block"})),this.originalElement.css({margin:this.originalElement.css("margin")}),t
 his._proportionallyResize()),this.handles=c.handles||(a(".ui-resizable-handle",this.element).length?{n:".ui-resizable-n",e:".ui-resizable-e",s:".ui-resizable-s",w:".ui-resizable-w",se:".ui-resizable-se",sw:".ui-resizable-sw",ne:".ui-resizable-ne",nw:".ui-resizable-nw"}:"e,s,se");if(this.handles.constructor==String){this.handles=="all"&&(this.handles="n,e,s,w,se,sw,ne,nw");var d=this.handles.split(",");this.handles={};for(var e=0;e<d.length;e++){var f=a.trim(d[e]),g="ui-resizable-"+f,h=a('<div class="ui-resizable-handle '+g+'"></div>');h.css({zIndex:c.zIndex}),"se"==f&&h.addClass("ui-icon ui-icon-gripsmall-diagonal-se"),this.handles[f]=".ui-resizable-"+f,this.element.append(h)}}this._renderAxis=function(b){b=b||this.element;for(var c in this.handles){this.handles[c].constructor==String&&(this.handles[c]=a(this.handles[c],this.element).show());if(this.elementIsWrapper&&this.originalElement[0].nodeName.match(/textarea|input|select|button/i)){var d=a(this.handles[c],this.element),e=0;e=
 /sw|ne|nw|se|n|s/.test(c)?d.outerHeight():d.outerWidth();var f=["padding",/ne|nw|n/.test(c)?"Top":/se|sw|s/.test(c)?"Bottom":/^e$/.test(c)?"Right":"Left"].join("");b.css(f,e),this._proportionallyResize()}if(!a(this.handles[c]).length)continue}},this._renderAxis(this.element),this._handles=a(".ui-resizable-handle",this.element).disableSelection(),this._handles.mouseover(function(){if(!b.resizing){if(this.className)var a=this.className.match(/ui-resizable-(se|sw|ne|nw|n|e|s|w)/i);b.axis=a&&a[1]?a[1]:"se"}}),c.autoHide&&(this._handles.hide(),a(this.element).addClass("ui-resizable-autohide").hover(function(){if(c.disabled)return;a(this).removeClass("ui-resizable-autohide"),b._handles.show()},function(){if(c.disabled)return;b.resizing||(a(this).addClass("ui-resizable-autohide"),b._handles.hide())})),this._mouseInit()},destroy:function(){this._mouseDestroy();var b=function(b){a(b).removeClass("ui-resizable ui-resizable-disabled ui-resizable-resizing").removeData("resizable").unbind(".resi
 zable").find(".ui-resizable-handle").remove()};if(this.elementIsWrapper){b(this.element);var c=this.element;c.after(this.originalElement.css({position:c.css("position"),width:c.outerWidth(),height:c.outerHeight(),top:c.css("top"),left:c.css("left")})).remove()}return this.originalElement.css("resize",this.originalResizeStyle),b(this.originalElement),this},_mouseCapture:function(b){var c=!1;for(var d in this.handles)a(this.handles[d])[0]==b.target&&(c=!0);return!this.options.disabled&&c},_mouseStart:function(b){var d=this.options,e=this.element.position(),f=this.element;this.resizing=!0,this.documentScroll={top:a(document).scrollTop(),left:a(document).scrollLeft()},(f.is(".ui-draggable")||/absolute/.test(f.css("position")))&&f.css({position:"absolute",top:e.top,left:e.left}),this._renderProxy();var g=c(this.helper.css("left")),h=c(this.helper.css("top"));d.containment&&(g+=a(d.containment).scrollLeft()||0,h+=a(d.containment).scrollTop()||0),this.offset=this.helper.offset(),this.posit
 ion={left:g,top:h},this.size=this._helper?{width:f.outerWidth(),height:f.outerHeight()}:{width:f.width(),height:f.height()},this.originalSize=this._helper?{width:f.outerWidth(),height:f.outerHeight()}:{width:f.width(),height:f.height()},this.originalPosition={left:g,top:h},this.sizeDiff={width:f.outerWidth()-f.width(),height:f.outerHeight()-f.height()},this.originalMousePosition={left:b.pageX,top:b.pageY},this.aspectRatio=typeof d.aspectRatio=="number"?d.aspectRatio:this.originalSize.width/this.originalSize.height||1;var i=a(".ui-resizable-"+this.axis).css("cursor");return a("body").css("cursor",i=="auto"?this.axis+"-resize":i),f.addClass("ui-resizable-resizing"),this._propagate("start",b),!0},_mouseDrag:function(b){var c=this.helper,d=this.options,e={},f=this,g=this.originalMousePosition,h=this.axis,i=b.pageX-g.left||0,j=b.pageY-g.top||0,k=this._change[h];if(!k)return!1;var l=k.apply(this,[b,i,j]),m=a.browser.msie&&a.browser.version<7,n=this.sizeDiff;this._updateVirtualBoundaries(b
 .shiftKey);if(this._aspectRatio||b.shiftKey)l=this._updateRatio(l,b);return l=this._respectSize(l,b),this._propagate("resize",b),c.css({top:this.position.top+"px",left:this.position.left+"px",width:this.size.width+"px",height:this.size.height+"px"}),!this._helper&&this._proportionallyResizeElements.length&&this._proportionallyResize(),this._updateCache(l),this._trigger("resize",b,this.ui()),!1},_mouseStop:function(b){this.resizing=!1;var c=this.options,d=this;if(this._helper){var e=this._proportionallyResizeElements,f=e.length&&/textarea/i.test(e[0].nodeName),g=f&&a.ui.hasScroll(e[0],"left")?0:d.sizeDiff.height,h=f?0:d.sizeDiff.width,i={width:d.helper.width()-h,height:d.helper.height()-g},j=parseInt(d.element.css("left"),10)+(d.position.left-d.originalPosition.left)||null,k=parseInt(d.element.css("top"),10)+(d.position.top-d.originalPosition.top)||null;c.animate||this.element.css(a.extend(i,{top:k,left:j})),d.helper.height(d.size.height),d.helper.width(d.size.width),this._helper&&!c
 .animate&&this._proportionallyResize()}return a("body").css("cursor","auto"),this.element.removeClass("ui-resizable-resizing"),this._propagate("stop",b),this._helper&&this.helper.remove(),!1},_updateVirtualBoundaries:function(a){var b=this.options,c,e,f,g,h;h={minWidth:d(b.minWidth)?b.minWidth:0,maxWidth:d(b.maxWidth)?b.maxWidth:Infinity,minHeight:d(b.minHeight)?b.minHeight:0,maxHeight:d(b.maxHeight)?b.maxHeight:Infinity};if(this._aspectRatio||a)c=h.minHeight*this.aspectRatio,f=h.minWidth/this.aspectRatio,e=h.maxHeight*this.aspectRatio,g=h.maxWidth/this.aspectRatio,c>h.minWidth&&(h.minWidth=c),f>h.minHeight&&(h.minHeight=f),e<h.maxWidth&&(h.maxWidth=e),g<h.maxHeight&&(h.maxHeight=g);this._vBoundaries=h},_updateCache:function(a){var b=this.options;this.offset=this.helper.offset(),d(a.left)&&(this.position.left=a.left),d(a.top)&&(this.position.top=a.top),d(a.height)&&(this.size.height=a.height),d(a.width)&&(this.size.width=a.width)},_updateRatio:function(a,b){var c=this.options,e=this
 .position,f=this.size,g=this.axis;return d(a.height)?a.width=a.height*this.aspectRatio:d(a.width)&&(a.height=a.width/this.aspectRatio),g=="sw"&&(a.left=e.left+(f.width-a.width),a.top=null),g=="nw"&&(a.top=e.top+(f.height-a.height),a.left=e.left+(f.width-a.width)),a},_respectSize:function(a,b){var c=this.helper,e=this._vBoundaries,f=this._aspectRatio||b.shiftKey,g=this.axis,h=d(a.width)&&e.maxWidth&&e.maxWidth<a.width,i=d(a.height)&&e.maxHeight&&e.maxHeight<a.height,j=d(a.width)&&e.minWidth&&e.minWidth>a.width,k=d(a.height)&&e.minHeight&&e.minHeight>a.height;j&&(a.width=e.minWidth),k&&(a.height=e.minHeight),h&&(a.width=e.maxWidth),i&&(a.height=e.maxHeight);var l=this.originalPosition.left+this.originalSize.width,m=this.position.top+this.size.height,n=/sw|nw|w/.test(g),o=/nw|ne|n/.test(g);j&&n&&(a.left=l-e.minWidth),h&&n&&(a.left=l-e.maxWidth),k&&o&&(a.top=m-e.minHeight),i&&o&&(a.top=m-e.maxHeight);var p=!a.width&&!a.height;return p&&!a.left&&a.top?a.top=null:p&&!a.top&&a.left&&(a.lef
 t=null),a},_proportionallyResize:function(){var b=this.options;if(!this._proportionallyResizeElements.length)return;var c=this.helper||this.element;for(var d=0;d<this._proportionallyResizeElements.length;d++){var e=this._proportionallyResizeElements[d];if(!this.borderDif){var f=[e.css("borderTopWidth"),e.css("borderRightWidth"),e.css("borderBottomWidth"),e.css("borderLeftWidth")],g=[e.css("paddingTop"),e.css("paddingRight"),e.css("paddingBottom"),e.css("paddingLeft")];this.borderDif=a.map(f,function(a,b){var c=parseInt(a,10)||0,d=parseInt(g[b],10)||0;return c+d})}if(!a.browser.msie||!a(c).is(":hidden")&&!a(c).parents(":hidden").length)e.css({height:c.height()-this.borderDif[0]-this.borderDif[2]||0,width:c.width()-this.borderDif[1]-this.borderDif[3]||0});else continue}},_renderProxy:function(){var b=this.element,c=this.options;this.elementOffset=b.offset();if(this._helper){this.helper=this.helper||a('<div style="overflow:hidden;"></div>');var d=a.browser.msie&&a.browser.version<7,e=d
 ?1:0,f=d?2:-1;this.helper.addClass(this._helper).css({width:this.element.outerWidth()+f,height:this.element.outerHeight()+f,position:"absolute",left:this.elementOffset.left-e+"px",top:this.elementOffset.top-e+"px",zIndex:++c.zIndex}),this.helper.appendTo("body").disableSelection()}else this.helper=this.element},_change:{e:function(a,b,c){return{width:this.originalSize.width+b}},w:function(a,b,c){var d=this.options,e=this.originalSize,f=this.originalPosition;return{left:f.left+b,width:e.width-b}},n:function(a,b,c){var d=this.options,e=this.originalSize,f=this.originalPosition;return{top:f.top+c,height:e.height-c}},s:function(a,b,c){return{height:this.originalSize.height+c}},se:function(b,c,d){return a.extend(this._change.s.apply(this,arguments),this._change.e.apply(this,[b,c,d]))},sw:function(b,c,d){return a.extend(this._change.s.apply(this,arguments),this._change.w.apply(this,[b,c,d]))},ne:function(b,c,d){return a.extend(this._change.n.apply(this,arguments),this._change.e.apply(this
 ,[b,c,d]))},nw:function(b,c,d){return a.extend(this._change.n.apply(this,arguments),this._change.w.apply(this,[b,c,d]))}},_propagate:function(b,c){a.ui.plugin.call(this,b,[c,this.ui()]),b!="resize"&&this._trigger(b,c,this.ui())},plugins:{},ui:function(){return{originalElement:this.originalElement,element:this.element,helper:this.helper,position:this.position,size:this.size,originalSize:this.originalSize,originalPosition:this.originalPosition}}}),a.extend(a.ui.resizable,{version:"1.8.23"}),a.ui.plugin.add("resizable","alsoResize",{start:function(b,c){var d=a(this).data("resizable"),e=d.options,f=function(b){a(b).each(function(){var b=a(this);b.data("resizable-alsoresize",{width:parseInt(b.width(),10),height:parseInt(b.height(),10),left:parseInt(b.css("left"),10),top:parseInt(b.css("top"),10)})})};typeof e.alsoResize=="object"&&!e.alsoResize.parentNode?e.alsoResize.length?(e.alsoResize=e.alsoResize[0],f(e.alsoResize)):a.each(e.alsoResize,function(a){f(a)}):f(e.alsoResize)},resize:func
 tion(b,c){var d=a(this).data("resizable"),e=d.options,f=d.originalSize,g=d.originalPosition,h={height:d.size.height-f.height||0,width:d.size.width-f.width||0,top:d.position.top-g.top||0,left:d.position.left-g.left||0},i=function(b,d){a(b).each(function(){var b=a(this),e=a(this).data("resizable-alsoresize"),f={},g=d&&d.length?d:b.parents(c.originalElement[0]).length?["width","height"]:["width","height","top","left"];a.each(g,function(a,b){var c=(e[b]||0)+(h[b]||0);c&&c>=0&&(f[b]=c||null)}),b.css(f)})};typeof e.alsoResize=="object"&&!e.alsoResize.nodeType?a.each(e.alsoResize,function(a,b){i(a,b)}):i(e.alsoResize)},stop:function(b,c){a(this).removeData("resizable-alsoresize")}}),a.ui.plugin.add("resizable","animate",{stop:function(b,c){var d=a(this).data("resizable"),e=d.options,f=d._proportionallyResizeElements,g=f.length&&/textarea/i.test(f[0].nodeName),h=g&&a.ui.hasScroll(f[0],"left")?0:d.sizeDiff.height,i=g?0:d.sizeDiff.width,j={width:d.size.width-i,height:d.size.height-h},k=parseI
 nt(d.element.css("left"),10)+(d.position.left-d.originalPosition.left)||null,l=parseInt(d.element.css("top"),10)+(d.position.top-d.originalPosition.top)||null;d.element.animate(a.extend(j,l&&k?{top:l,left:k}:{}),{duration:e.animateDuration,easing:e.animateEasing,step:function(){var c={width:parseInt(d.element.css("width"),10),height:parseInt(d.element.css("height"),10),top:parseInt(d.element.css("top"),10),left:parseInt(d.element.css("left"),10)};f&&f.length&&a(f[0]).css({width:c.width,height:c.height}),d._updateCache(c),d._propagate("resize",b)}})}}),a.ui.plugin.add("resizable","containment",{start:function(b,d){var e=a(this).data("resizable"),f=e.options,g=e.element,h=f.containment,i=h instanceof a?h.get(0):/parent/.test(h)?g.parent().get(0):h;if(!i)return;e.containerElement=a(i);if(/document/.test(h)||h==document)e.containerOffset={left:0,top:0},e.containerPosition={left:0,top:0},e.parentData={element:a(document),left:0,top:0,width:a(document).width(),height:a(document).height()|
 |document.body.parentNode.scrollHeight};else{var j=a(i),k=[];a(["Top","Right","Left","Bottom"]).each(function(a,b){k[a]=c(j.css("padding"+b))}),e.containerOffset=j.offset(),e.containerPosition=j.position(),e.containerSize={height:j.innerHeight()-k[3],width:j.innerWidth()-k[1]};var l=e.containerOffset,m=e.containerSize.height,n=e.containerSize.width,o=a.ui.hasScroll(i,"left")?i.scrollWidth:n,p=a.ui.hasScroll(i)?i.scrollHeight:m;e.parentData={element:i,left:l.left,top:l.top,width:o,height:p}}},resize:function(b,c){var d=a(this).data("resizable"),e=d.options,f=d.containerSize,g=d.containerOffset,h=d.size,i=d.position,j=d._aspectRatio||b.shiftKey,k={top:0,left:0},l=d.containerElement;l[0]!=document&&/static/.test(l.css("position"))&&(k=g),i.left<(d._helper?g.left:0)&&(d.size.width=d.size.width+(d._helper?d.position.left-g.left:d.position.left-k.left),j&&(d.size.height=d.size.width/d.aspectRatio),d.position.left=e.helper?g.left:0),i.top<(d._helper?g.top:0)&&(d.size.height=d.size.height+(
 d._helper?d.position.top-g.top:d.position.top),j&&(d.size.width=d.size.height*d.aspectRatio),d.position.top=d._helper?g.top:0),d.offset.left=d.parentData.left+d.position.left,d.offset.top=d.parentData.top+d.position.top;var m=Math.abs((d._helper?d.offset.left-k.left:d.offset.left-k.left)+d.sizeDiff.width),n=Math.abs((d._helper?d.offset.top-k.top:d.offset.top-g.top)+d.sizeDiff.height),o=d.containerElement.get(0)==d.element.parent().get(0),p=/relative|absolute/.test(d.containerElement.css("position"));o&&p&&(m-=d.parentData.left),m+d.size.width>=d.parentData.width&&(d.size.width=d.parentData.width-m,j&&(d.size.height=d.size.width/d.aspectRatio)),n+d.size.height>=d.parentData.height&&(d.size.height=d.parentData.height-n,j&&(d.size.width=d.size.height*d.aspectRatio))},stop:function(b,c){var d=a(this).data("resizable"),e=d.options,f=d.position,g=d.containerOffset,h=d.containerPosition,i=d.containerElement,j=a(d.helper),k=j.offset(),l=j.outerWidth()-d.sizeDiff.width,m=j.outerHeight()-d.si
 zeDiff.height;d._helper&&!e.animate&&/relative/.test(i.css("position"))&&a(this).css({left:k.left-h.left-g.left,width:l,height:m}),d._helper&&!e.animate&&/static/.test(i.css("position"))&&a(this).css({left:k.left-h.left-g.left,width:l,height:m})}}),a.ui.plugin.add("resizable","ghost",{start:function(b,c){var d=a(this).data("resizable"),e=d.options,f=d.size;d.ghost=d.originalElement.clone(),d.ghost.css({opacity:.25,display:"block",position:"relative",height:f.height,width:f.width,margin:0,left:0,top:0}).addClass("ui-resizable-ghost").addClass(typeof e.ghost=="string"?e.ghost:""),d.ghost.appendTo(d.helper)},resize:function(b,c){var d=a(this).data("resizable"),e=d.options;d.ghost&&d.ghost.css({position:"relative",height:d.size.height,width:d.size.width})},stop:function(b,c){var d=a(this).data("resizable"),e=d.options;d.ghost&&d.helper&&d.helper.get(0).removeChild(d.ghost.get(0))}}),a.ui.plugin.add("resizable","grid",{resize:function(b,c){var d=a(this).data("resizable"),e=d.options,f=d.
 size,g=d.originalSize,h=d.originalPosition,i=d.axis,j=e._aspectRatio||b.shiftKey;e.grid=typeof e.grid=="number"?[e.grid,e.grid]:e.grid;var k=Math.round((f.width-g.width)/(e.grid[0]||1))*(e.grid[0]||1),l=Math.round((f.height-g.height)/(e.grid[1]||1))*(e.grid[1]||1);/^(se|s|e)$/.test(i)?(d.size.width=g.width+k,d.size.height=g.height+l):/^(ne)$/.test(i)?(d.size.width=g.width+k,d.size.height=g.height+l,d.position.top=h.top-l):/^(sw)$/.test(i)?(d.size.width=g.width+k,d.size.height=g.height+l,d.position.left=h.left-k):(d.size.width=g.width+k,d.size.height=g.height+l,d.position.top=h.top-l,d.position.left=h.left-k)}});var c=function(a){return parseInt(a,10)||0},d=function(a){return!isNaN(parseInt(a,10))}}(jQuery),function(a,b){a.widget("ui.selectable",a.ui.mouse,{options:{appendTo:"body",autoRefresh:!0,distance:0,filter:"*",tolerance:"touch"},_create:function(){var b=this;this.element.addClass("ui-selectable"),this.dragged=!1;var c;this.refresh=function(){c=a(b.options.filter,b.element[0])
 ,c.addClass("ui-selectee"),c.each(function(){var b=a(this),c=b.offset();a.data(this,"selectable-item",{element:this,$element:b,left:c.left,top:c.top,right:c.left+b.outerWidth(),bottom:c.top+b.outerHeight(),startselected:!1,selected:b.hasClass("ui-selected"),selecting:b.hasClass("ui-selecting"),unselecting:b.hasClass("ui-unselecting")})})},this.refresh(),this.selectees=c.addClass("ui-selectee"),this._mouseInit(),this.helper=a("<div class='ui-selectable-helper'></div>")},destroy:function(){return this.selectees.removeClass("ui-selectee").removeData("selectable-item"),this.element.removeClass("ui-selectable ui-selectable-disabled").removeData("selectable").unbind(".selectable"),this._mouseDestroy(),this},_mouseStart:function(b){var c=this;this.opos=[b.pageX,b.pageY];if(this.options.disabled)return;var d=this.options;this.selectees=a(d.filter,this.element[0]),this._trigger("start",b),a(d.appendTo).append(this.helper),this.helper.css({left:b.clientX,top:b.clientY,width:0,height:0}),d.aut
 oRefresh&&this.refresh(),this.selectees.filter(".ui-selected").each(function(){var d=a.data(this,"selectable-item");d.startselected=!0,!b.metaKey&&!b.ctrlKey&&(d.$element.removeClass("ui-selected"),d.selected=!1,d.$element.addClass("ui-unselecting"),d.unselecting=!0,c._trigger("unselecting",b,{unselecting:d.element}))}),a(b.target).parents().andSelf().each(function(){var d=a.data(this,"selectable-item");if(d){var e=!b.metaKey&&!b.ctrlKey||!d.$element.hasClass("ui-selected");return d.$element.removeClass(e?"ui-unselecting":"ui-selected").addClass(e?"ui-selecting":"ui-unselecting"),d.unselecting=!e,d.selecting=e,d.selected=e,e?c._trigger("selecting",b,{selecting:d.element}):c._trigger("unselecting",b,{unselecting:d.element}),!1}})},_mouseDrag:function(b){var c=this;this.dragged=!0;if(this.options.disabled)return;var d=this.options,e=this.opos[0],f=this.opos[1],g=b.pageX,h=b.pageY;if(e>g){var i=g;g=e,e=i}if(f>h){var i=h;h=f,f=i}return this.helper.css({left:e,top:f,width:g-e,height:h-f}
 ),this.selectees.each(function(){var i=a.data(this,"selectable-item");if(!i||i.element==c.element[0])return;var j=!1;d.tolerance=="touch"?j=!(i.left>g||i.right<e||i.top>h||i.bottom<f):d.tolerance=="fit"&&(j=i.left>e&&i.right<g&&i.top>f&&i.bottom<h),j?(i.selected&&(i.$element.removeClass("ui-selected"),i.selected=!1),i.unselecting&&(i.$element.removeClass("ui-unselecting"),i.unselecting=!1),i.selecting||(i.$element.addClass("ui-selecting"),i.selecting=!0,c._trigger("selecting",b,{selecting:i.element}))):(i.selecting&&((b.metaKey||b.ctrlKey)&&i.startselected?(i.$element.removeClass("ui-selecting"),i.selecting=!1,i.$element.addClass("ui-selected"),i.selected=!0):(i.$element.removeClass("ui-selecting"),i.selecting=!1,i.startselected&&(i.$element.addClass("ui-unselecting"),i.unselecting=!0),c._trigger("unselecting",b,{unselecting:i.element}))),i.selected&&!b.metaKey&&!b.ctrlKey&&!i.startselected&&(i.$element.removeClass("ui-selected"),i.selected=!1,i.$element.addClass("ui-unselecting"),i
 .unselecting=!0,c._trigger("unselecting",b,{unselecting:i.element})))}),!1},_mouseStop:function(b){var c=this;this.dragged=!1;var d=this.options;return a(".ui-unselecting",this.element[0]).each(function(){var d=a.data(this,"selectable-item");d.$element.removeClass("ui-unselecting"),d.unselecting=!1,d.startselected=!1,c._trigger("unselected",b,{unselected:d.element})}),a(".ui-selecting",this.element[0]).each(function(){var d=a.data(this,"selectable-item");d.$element.removeClass("ui-selecting").addClass("ui-selected"),d.selecting=!1,d.selected=!0,d.startselected=!0,c._trigger("selected",b,{selected:d.element})}),this._trigger("stop",b),this.helper.remove(),!1}}),a.extend(a.ui.selectable,{version:"1.8.23"})}(jQuery),function(a,b){a.widget("ui.sortable",a.ui.mouse,{widgetEventPrefix:"sort",ready:!1,options:{appendTo:"parent",axis:!1,connectWith:!1,containment:!1,cursor:"auto",cursorAt:!1,dropOnEmpty:!0,forcePlaceholderSize:!1,forceHelperSize:!1,grid:!1,handle:!1,helper:"original",items:
 "> *",opacity:!1,placeholder:!1,revert:!1,scroll:!0,scrollSensitivity:20,scrollSpeed:20,scope:"default",tolerance:"intersect",zIndex:1e3},_create:function(){var a=this.options;this.containerCache={},this.element.addClass("ui-sortable"),this.refresh(),this.floating=this.items.length?a.axis==="x"||/left|right/.test(this.items[0].item.css("float"))||/inline|table-cell/.test(this.items[0].item.css("display")):!1,this.offset=this.element.offset(),this._mouseInit(),this.ready=!0},destroy:function(){a.Widget.prototype.destroy.call(this),this.element.removeClass("ui-sortable ui-sortable-disabled"),this._mouseDestroy();for(var b=this.items.length-1;b>=0;b--)this.items[b].item.removeData(this.widgetName+"-item");return this},_setOption:function(b,c){b==="disabled"?(this.options[b]=c,this.widget()[c?"addClass":"removeClass"]("ui-sortable-disabled")):a.Widget.prototype._setOption.apply(this,arguments)},_mouseCapture:function(b,c){var d=this;if(this.reverting)return!1;if(this.options.disabled||t
 his.options.type=="static")return!1;this._refreshItems(b);var e=null,f=this,g=a(b.target).parents().each(function(){if(a.data(this,d.widgetName+"-item")==f)return e=a(this),!1});a.data(b.target,d.widgetName+"-item")==f&&(e=a(b.target));if(!e)return!1;if(this.options.handle&&!c){var h=!1;a(this.options.handle,e).find("*").andSelf().each(function(){this==b.target&&(h=!0)});if(!h)return!1}return this.currentItem=e,this._removeCurrentsFromItems(),!0},_mouseStart:function(b,c,d){var e=this.options,f=this;this.currentContainer=this,this.refreshPositions(),this.helper=this._createHelper(b),this._cacheHelperProportions(),this._cacheMargins(),this.scrollParent=this.helper.scrollParent(),this.offset=this.currentItem.offset(),this.offset={top:this.offset.top-this.margins.top,left:this.offset.left-this.margins.left},a.extend(this.offset,{click:{left:b.pageX-this.offset.left,top:b.pageY-this.offset.top},parent:this._getParentOffset(),relative:this._getRelativeOffset()}),this.helper.css("position
 ","absolute"),this.cssPosition=this.helper.css("position"),this.originalPosition=this._generatePosition(b),this.originalPageX=b.pageX,this.originalPageY=b.pageY,e.cursorAt&&this._adjustOffsetFromHelper(e.cursorAt),this.domPosition={prev:this.currentItem.prev()[0],parent:this.currentItem.parent()[0]},this.helper[0]!=this.currentItem[0]&&this.currentItem.hide(),this._createPlaceholder(),e.containment&&this._setContainment(),e.cursor&&(a("body").css("cursor")&&(this._storedCursor=a("body").css("cursor")),a("body").css("cursor",e.cursor)),e.opacity&&(this.helper.css("opacity")&&(this._storedOpacity=this.helper.css("opacity")),this.helper.css("opacity",e.opacity)),e.zIndex&&(this.helper.css("zIndex")&&(this._storedZIndex=this.helper.css("zIndex")),this.helper.css("zIndex",e.zIndex)),this.scrollParent[0]!=document&&this.scrollParent[0].tagName!="HTML"&&(this.overflowOffset=this.scrollParent.offset()),this._trigger("start",b,this._uiHash()),this._preserveHelperProportions||this._cacheHelpe
 rProportions();if(!d)for(var g=this.containers.length-1;g>=0;g--)this.containers[g]._trigger("activate",b,f._uiHash(this));return a.ui.ddmanager&&(a.ui.ddmanager.current=this),a.ui.ddmanager&&!e.dropBehaviour&&a.ui.ddmanager.prepareOffsets(this,b),this.dragging=!0,this.helper.addClass("ui-sortable-helper"),this._mouseDrag(b),!0},_mouseDrag:function(b){this.position=this._generatePosition(b),this.positionAbs=this._convertPositionTo("absolute"),this.lastPositionAbs||(this.lastPositionAbs=this.positionAbs);if(this.options.scroll){var c=this.options,d=!1;this.scrollParent[0]!=document&&this.scrollParent[0].tagName!="HTML"?(this.overflowOffset.top+this.scrollParent[0].offsetHeight-b.pageY<c.scrollSensitivity?this.scrollParent[0].scrollTop=d=this.scrollParent[0].scrollTop+c.scrollSpeed:b.pageY-this.overflowOffset.top<c.scrollSensitivity&&(this.scrollParent[0].scrollTop=d=this.scrollParent[0].scrollTop-c.scrollSpeed),this.overflowOffset.left+this.scrollParent[0].offsetWidth-b.pageX<c.scrol
 lSensitivity?this.scrollParent[0].scrollLeft=d=this.scrollParent[0].scrollLeft+c.scrollSpeed:b.pageX-this.overflowOffset.left<c.scrollSensitivity&&(this.scrollParent[0].scrollLeft=d=this.scrollParent[0].scrollLeft-c.scrollSpeed)):(b.pageY-a(document).scrollTop()<c.scrollSensitivity?d=a(document).scrollTop(a(document).scrollTop()-c.scrollSpeed):a(window).height()-(b.pageY-a(document).scrollTop())<c.scrollSensitivity&&(d=a(document).scrollTop(a(document).scrollTop()+c.scrollSpeed)),b.pageX-a(document).scrollLeft()<c.scrollSensitivity?d=a(document).scrollLeft(a(document).scrollLeft()-c.scrollSpeed):a(window).width()-(b.pageX-a(document).scrollLeft())<c.scrollSensitivity&&(d=a(document).scrollLeft(a(document).scrollLeft()+c.scrollSpeed))),d!==!1&&a.ui.ddmanager&&!c.dropBehaviour&&a.ui.ddmanager.prepareOffsets(this,b)}this.positionAbs=this._convertPositionTo("absolute");if(!this.options.axis||this.options.axis!="y")this.helper[0].style.left=this.position.left+"px";if(!this.options.axis||
 this.options.axis!="x")this.helper[0].style.top=this.position.top+"px";for(var e=this.items.length-1;e>=0;e--){var f=this.items[e],g=f.item[0],h=this._intersectsWithPointer(f);if(!h)continue;if(g!=this.currentItem[0]&&this.placeholder[h==1?"next":"prev"]()[0]!=g&&!a.ui.contains(this.placeholder[0],g)&&(this.options.type=="semi-dynamic"?!a.ui.contains(this.element[0],g):!0)){this.direction=h==1?"down":"up";if(this.options.tolerance=="pointer"||this._intersectsWithSides(f))this._rearrange(b,f);else break;this._trigger("change",b,this._uiHash());break}}return this._contactContainers(b),a.ui.ddmanager&&a.ui.ddmanager.drag(this,b),this._trigger("sort",b,this._uiHash()),this.lastPositionAbs=this.positionAbs,!1},_mouseStop:function(b,c){if(!b)return;a.ui.ddmanager&&!this.options.dropBehaviour&&a.ui.ddmanager.drop(this,b);if(this.options.revert){var d=this,e=d.placeholder.offset();d.reverting=!0,a(this.helper).animate({left:e.left-this.offset.parent.left-d.margins.left+(this.offsetParent[0]
 ==document.body?0:this.offsetParent[0].scrollLeft),top:e.top-this.offset.parent.top-d.margins.top+(this.offsetParent[0]==document.body?0:this.offsetParent[0].scrollTop)},parseInt(this.options.revert,10)||500,function(){d._clear(b)})}else this._clear(b,c);return!1},cancel:function(){var b=this;if(this.dragging){this._mouseUp({target:null}),this.options.helper=="original"?this.currentItem.css(this._storedCSS).removeClass("ui-sortable-helper"):this.currentItem.show();for(var c=this.containers.length-1;c>=0;c--)this.containers[c]._trigger("deactivate",null,b._uiHash(this)),this.containers[c].containerCache.over&&(this.containers[c]._trigger("out",null,b._uiHash(this)),this.containers[c].containerCache.over=0)}return this.placeholder&&(this.placeholder[0].parentNode&&this.placeholder[0].parentNode.removeChild(this.placeholder[0]),this.options.helper!="original"&&this.helper&&this.helper[0].parentNode&&this.helper.remove(),a.extend(this,{helper:null,dragging:!1,reverting:!1,_noFinalSort:n
 ull}),this.domPosition.prev?a(this.domPosition.prev).after(this.currentItem):a(this.domPosition.parent).prepend(this.currentItem)),this},serialize:function(b){var c=this._getItemsAsjQuery(b&&b.connected),d=[];return b=b||{},a(c).each(function(){var c=(a(b.item||this).attr(b.attribute||"id")||"").match(b.expression||/(.+)[-=_](.+)/);c&&d.push((b.key||c[1]+"[]")+"="+(b.key&&b.expression?c[1]:c[2]))}),!d.length&&b.key&&d.push(b.key+"="),d.join("&")},toArray:function(b){var c=this._getItemsAsjQuery(b&&b.connected),d=[];return b=b||{},c.each(function(){d.push(a(b.item||this).attr(b.attribute||"id")||"")}),d},_intersectsWith:function(a){var b=this.positionAbs.left,c=b+this.helperProportions.width,d=this.positionAbs.top,e=d+this.helperProportions.height,f=a.left,g=f+a.width,h=a.top,i=h+a.height,j=this.offset.click.top,k=this.offset.click.left,l=d+j>h&&d+j<i&&b+k>f&&b+k<g;return this.options.tolerance=="pointer"||this.options.forcePointerForContainers||this.options.tolerance!="pointer"&&thi
 s.helperProportions[this.floating?"width":"height"]>a[this.floating?"width":"height"]?l:f<b+this.helperProportions.width/2&&c-this.helperProportions.width/2<g&&h<d+this.helperProportions.height/2&&e-this.helperProportions.height/2<i},_intersectsWithPointer:function(b){var c=this.options.axis==="x"||a.ui.isOverAxis(this.positionAbs.top+this.offset.click.top,b.top,b.height),d=this.options.axis==="y"||a.ui.isOverAxis(this.positionAbs.left+this.offset.click.left,b.left,b.width),e=c&&d,f=this._getDragVerticalDirection(),g=this._getDragHorizontalDirection();return e?this.floating?g&&g=="right"||f=="down"?2:1:f&&(f=="down"?2:1):!1},_intersectsWithSides:function(b){var c=a.ui.isOverAxis(this.positionAbs.top+this.offset.click.top,b.top+b.height/2,b.height),d=a.ui.isOverAxis(this.positionAbs.left+this.offset.click.left,b.left+b.width/2,b.width),e=this._getDragVerticalDirection(),f=this._getDragHorizontalDirection();return this.floating&&f?f=="right"&&d||f=="left"&&!d:e&&(e=="down"&&c||e=="up"
 &&!c)},_getDragVerticalDirection:function(){var a=this.positionAbs.top-this.lastPositionAbs.top;return a!=0&&(a>0?"down":"up")},_getDragHorizontalDirection:function(){var a=this.positionAbs.left-this.lastPositionAbs.left;return a!=0&&(a>0?"right":"left")},refresh:function(a){return this._refreshItems(a),this.refreshPositions(),this},_connectWith:function(){var a=this.options;return a.connectWith.constructor==String?[a.connectWith]:a.connectWith},_getItemsAsjQuery:function(b){var c=this,d=[],e=[],f=this._connectWith();if(f&&b)for(var g=f.length-1;g>=0;g--){var h=a(f[g]);for(var i=h.length-1;i>=0;i--){var j=a.data(h[i],this.widgetName);j&&j!=this&&!j.options.disabled&&e.push([a.isFunction(j.options.items)?j.options.items.call(j.element):a(j.options.items,j.element).not(".ui-sortable-helper").not(".ui-sortable-placeholder"),j])}}e.push([a.isFunction(this.options.items)?this.options.items.call(this.element,null,{options:this.options,item:this.currentItem}):a(this.options.items,this.elem
 ent).not(".ui-sortable-helper").not(".ui-sortable-placeholder"),this]);for(var g=e.length-1;g>=0;g--)e[g][0].each(function(){d.push(this)});return a(d)},_removeCurrentsFromItems:function(){var a=this.currentItem.find(":data("+this.widgetName+"-item)");for(var b=0;b<this.items.length;b++)for(var c=0;c<a.length;c++)a[c]==this.items[b].item[0]&&this.items.splice(b,1)},_refreshItems:function(b){this.items=[],this.containers=[this];var c=this.items,d=this,e=[[a.isFunction(this.options.items)?this.options.items.call(this.element[0],b,{item:this.currentItem}):a(this.options.items,this.element),this]],f=this._connectWith();if(f&&this.ready)for(var g=f.length-1;g>=0;g--){var h=a(f[g]);for(var i=h.length-1;i>=0;i--){var j=a.data(h[i],this.widgetName);j&&j!=this&&!j.options.disabled&&(e.push([a.isFunction(j.options.items)?j.options.items.call(j.element[0],b,{item:this.currentItem}):a(j.options.items,j.element),j]),this.containers.push(j))}}for(var g=e.length-1;g>=0;g--){var k=e[g][1],l=e[g][0]
 ;for(var i=0,m=l.length;i<m;i++){var n=a(l[i]);n.data(this.widgetName+"-item",k),c.push({item:n,instance:k,width:0,height:0,left:0,top:0})}}},refreshPositions:function(b){this.offsetParent&&this.helper&&(this.offset.parent=this._getParentOffset());for(var c=this.items.length-1;c>=0;c--){var d=this.items[c];if(d.instance!=this.currentContainer&&this.currentContainer&&d.item[0]!=this.currentItem[0])continue;var e=this.options.toleranceElement?a(this.options.toleranceElement,d.item):d.item;b||(d.width=e.outerWidth(),d.height=e.outerHeight());var f=e.offset();d.left=f.left,d.top=f.top}if(this.options.custom&&this.options.custom.refreshContainers)this.options.custom.refreshContainers.call(this);else for(var c=this.containers.length-1;c>=0;c--){var f=this.containers[c].element.offset();this.containers[c].containerCache.left=f.left,this.containers[c].containerCache.top=f.top,this.containers[c].containerCache.width=this.containers[c].element.outerWidth(),this.containers[c].containerCache.he
 ight=this.containers[c].element.outerHeight()}return this},_createPlaceholder:function(b){var c=b||this,d=c.options;if(!d.placeholder||d.placeholder.constructor==String){var e=d.placeholder;d.placeholder={element:function(){var b=a(document.createElement(c.currentItem[0].nodeName)).addClass(e||c.currentItem[0].className+" ui-sortable-placeholder").removeClass("ui-sortable-helper")[0];return e||(b.style.visibility="hidden"),b},update:function(a,b){if(e&&!d.forcePlaceholderSize)return;b.height()||b.height(c.currentItem.innerHeight()-parseInt(c.currentItem.css("paddingTop")||0,10)-parseInt(c.currentItem.css("paddingBottom")||0,10)),b.width()||b.width(c.currentItem.innerWidth()-parseInt(c.currentItem.css("paddingLeft")||0,10)-parseInt(c.currentItem.css("paddingRight")||0,10))}}}c.placeholder=a(d.placeholder.element.call(c.element,c.currentItem)),c.currentItem.after(c.placeholder),d.placeholder.update(c,c.placeholder)},_contactContainers:function(b){var c=null,d=null;for(var e=this.conta
 iners.length-1;e>=0;e--){if(a.ui.contains(this.currentItem[0],this.containers[e].element[0]))continue;if(this._intersectsWith(this.containers[e].containerCache)){if(c&&a.ui.contains(this.containers[e].element[0],c.element[0]))continue;c=this.containers[e],d=e}else this.containers[e].containerCache.over&&(this.containers[e]._trigger("out",b,this._uiHash(this)),this.containers[e].containerCache.over=0)}if(!c)return;if(this.containers.length===1)this.containers[d]._trigger("over",b,this._uiHash(this)),this.containers[d].containerCache.over=1;else if(this.currentContainer!=this.containers[d]){var f=1e4,g=null,h=this.positionAbs[this.containers[d].floating?"left":"top"];for(var i=this.items.length-1;i>=0;i--){if(!a.ui.contains(this.containers[d].element[0],this.items[i].item[0]))continue;var j=this.containers[d].floating?this.items[i].item.offset().left:this.items[i].item.offset().top;Math.abs(j-h)<f&&(f=Math.abs(j-h),g=this.items[i],this.direction=j-h>0?"down":"up")}if(!g&&!this.options
 .dropOnEmpty)return;this.currentContainer=this.containers[d],g?this._rearrange(b,g,null,!0):this._rearrange(b,null,this.containers[d].element,!0),this._trigger("change",b,this._uiHash()),this.containers[d]._trigger("change",b,this._uiHash(this)),this.options.placeholder.update(this.currentContainer,this.placeholder),this.containers[d]._trigger("over",b,this._uiHash(this)),this.containers[d].containerCache.over=1}},_createHelper:function(b){var c=this.options,d=a.isFunction(c.helper)?a(c.helper.apply(this.element[0],[b,this.currentItem])):c.helper=="clone"?this.currentItem.clone():this.currentItem;return d.parents("body").length||a(c.appendTo!="parent"?c.appendTo:this.currentItem[0].parentNode)[0].appendChild(d[0]),d[0]==this.currentItem[0]&&(this._storedCSS={width:this.currentItem[0].style.width,height:this.currentItem[0].style.height,position:this.currentItem.css("position"),top:this.currentItem.css("top"),left:this.currentItem.css("left")}),(d[0].style.width==""||c.forceHelperSize
 )&&d.width(this.currentItem.width()),(d[0].style.height==""||c.forceHelperSize)&&d.height(this.currentItem.height()),d},_adjustOffsetFromHelper:function(b){typeof b=="string"&&(b=b.split(" ")),a.isArray(b)&&(b={left:+b[0],top:+b[1]||0}),"left"in b&&(this.offset.click.left=b.left+this.margins.left),"right"in b&&(this.offset.click.left=this.helperProportions.width-b.right+this.margins.left),"top"in b&&(this.offset.click.top=b.top+this.margins.top),"bottom"in b&&(this.offset.click.top=this.helperProportions.height-b.bottom+this.margins.top)},_getParentOffset:function(){this.offsetParent=this.helper.offsetParent();var b=this.offsetParent.offset();this.cssPosition=="absolute"&&this.scrollParent[0]!=document&&a.ui.contains(this.scrollParent[0],this.offsetParent[0])&&(b.left+=this.scrollParent.scrollLeft(),b.top+=this.scrollParent.scrollTop());if(this.offsetParent[0]==document.body||this.offsetParent[0].tagName&&this.offsetParent[0].tagName.toLowerCase()=="html"&&a.browser.msie)b={top:0,le
 ft:0};return{top:b.top+(parseInt(this.offsetParent.css("borderTopWidth"),10)||0),left:b.left+(parseInt(this.offsetParent.css("borderLeftWidth"),10)||0)}},_getRelativeOffset:function(){if(this.cssPosition=="relative"){var a=this.currentItem.position();return{top:a.top-(parseInt(this.helper.css("top"),10)||0)+this.scrollParent.scrollTop(),left:a.left-(parseInt(this.helper.css("left"),10)||0)+this.scrollParent.scrollLeft()}}return{top:0,left:0}},_cacheMargins:function(){this.margins={left:parseInt(this.currentItem.css("marginLeft"),10)||0,top:parseInt(this.currentItem.css("marginTop"),10)||0}},_cacheHelperProportions:function(){this.helperProportions={width:this.helper.outerWidth(),height:this.helper.outerHeight()}},_setContainment:function(){var b=this.options;b.containment=="parent"&&(b.containment=this.helper[0].parentNode);if(b.containment=="document"||b.containment=="window")this.containment=[0-this.offset.relative.left-this.offset.parent.left,0-this.offset.relative.top-this.offse
 t.parent.top,a(b.containment=="document"?document:window).width()-this.helperProportions.width-this.margins.left,(a(b.containment=="document"?document:window).height()||document.body.parentNode.scrollHeight)-this.helperProportions.height-this.margins.top];if(!/^(document|window|parent)$/.test(b.containment)){var c=a(b.containment)[0],d=a(b.containment).offset(),e=a(c).css("overflow")!="hidden";this.containment=[d.left+(parseInt(a(c).css("borderLeftWidth"),10)||0)+(parseInt(a(c).css("paddingLeft"),10)||0)-this.margins.left,d.top+(parseInt(a(c).css("borderTopWidth"),10)||0)+(parseInt(a(c).css("paddingTop"),10)||0)-this.margins.top,d.left+(e?Math.max(c.scrollWidth,c.offsetWidth):c.offsetWidth)-(parseInt(a(c).css("borderLeftWidth"),10)||0)-(parseInt(a(c).css("paddingRight"),10)||0)-this.helperProportions.width-this.margins.left,d.top+(e?Math.max(c.scrollHeight,c.offsetHeight):c.offsetHeight)-(parseInt(a(c).css("borderTopWidth"),10)||0)-(parseInt(a(c).css("paddingBottom"),10)||0)-this.he
 lperProportions.height-this.margins.top]}},_convertPositionTo:function(b,c){c||(c=this.position);var d=b=="absolute"?1:-1,e=this.options,f=this.cssPosition=="absolute"&&(this.scrollParent[0]==document||!a.ui.contains(this.scrollParent[0],this.offsetParent[0]))?this.offsetParent:this.scrollParent,g=/(html|body)/i.test(f[0].tagName);return{top:c.top+this.offset.relative.top*d+this.offset.parent.top*d-(a.browser.safari&&this.cssPosition=="fixed"?0:(this.cssPosition=="fixed"?-this.scrollParent.scrollTop():g?0:f.scrollTop())*d),left:c.left+this.offset.relative.left*d+this.offset.parent.left*d-(a.browser.safari&&this.cssPosition=="fixed"?0:(this.cssPosition=="fixed"?-this.scrollParent.scrollLeft():g?0:f.scrollLeft())*d)}},_generatePosition:function(b){var c=this.options,d=this.cssPosition=="absolute"&&(this.scrollParent[0]==document||!a.ui.contains(this.scrollParent[0],this.offsetParent[0]))?this.offsetParent:this.scrollParent,e=/(html|body)/i.test(d[0].tagName);this.cssPosition=="relativ
 e"&&(this.scrollParent[0]==document||this.scrollParent[0]==this.offsetParent[0])&&(this.offset.relative=this._getRelativeOffset());var f=b.pageX,g=b.pageY;if(this.originalPosition){this.containment&&(b.pageX-this.offset.click.left<this.containment[0]&&(f=this.containment[0]+this.offset.click.left),b.pageY-this.offset.click.top<this.containment[1]&&(g=this.containment[1]+this.offset.click.top),b.pageX-this.offset.click.left>this.containment[2]&&(f=this.containment[2]+this.offset.click.left),b.pageY-this.offset.click.top>this.containment[3]&&(g=this.containment[3]+this.offset.click.top));if(c.grid){var h=this.originalPageY+Math.round((g-this.originalPageY)/c.grid[1])*c.grid[1];g=this.containment?h-this.offset.click.top<this.containment[1]||h-this.offset.click.top>this.containment[3]?h-this.offset.click.top<this.containment[1]?h+c.grid[1]:h-c.grid[1]:h:h;var i=this.originalPageX+Math.round((f-this.originalPageX)/c.grid[0])*c.grid[0];f=this.containment?i-this.offset.click.left<this.cont
 ainment[0]||i-this.offset.click.left>this.containment[2]?i-this.offset.click.left<this.containment[0]?i+c.grid[0]:i-c.grid[0]:i:i}}return{top:g-this.offset.click.top-this.offset.relative.top-this.offset.parent.top+(a.browser.safari&&this.cssPosition=="fixed"?0:this.cssPosition=="fixed"?-this.scrollParent.scrollTop():e?0:d.scrollTop()),left:f-this.offset.click.left-this.offset.relative.left-this.offset.parent.left+(a.browser.safari&&this.cssPosition=="fixed"?0:this.cssPosition=="fixed"?-this.scrollParent.scrollLeft():e?0:d.scrollLeft())}},_rearrange:function(a,b,c,d){c?c[0].appendChild(this.placeholder[0]):b.item[0].parentNode.insertBefore(this.placeholder[0],this.direction=="down"?b.item[0]:b.item[0].nextSibling),this.counter=this.counter?++this.counter:1;var e=this,f=this.counter;window.setTimeout(function(){f==e.counter&&e.refreshPositions(!d)},0)},_clear:function(b,c){this.reverting=!1;var d=[],e=this;!this._noFinalSort&&this.currentItem.parent().length&&this.placeholder.before(t
 his.currentItem),this._noFinalSort=null;if(this.helper[0]==this.currentItem[0]){for(var f in this._storedCSS)if(this._storedCSS[f]=="auto"||this._storedCSS[f]=="static")this._storedCSS[f]="";this.currentItem.css(this._storedCSS).removeClass("ui-sortable-helper")}else this.currentItem.show();this.fromOutside&&!c&&d.push(function(a){this._trigger("receive",a,this._uiHash(this.fromOutside))}),(this.fromOutside||this.domPosition.prev!=this.currentItem.prev().not(".ui-sortable-helper")[0]||this.domPosition.parent!=this.currentItem.parent()[0])&&!c&&d.push(function(a){this._trigger("update",a,this._uiHash())});if(!a.ui.contains(this.element[0],this.currentItem[0])){c||d.push(function(a){this._trigger("remove",a,this._uiHash())});for(var f=this.containers.length-1;f>=0;f--)a.ui.contains(this.containers[f].element[0],this.currentItem[0])&&!c&&(d.push(function(a){return function(b){a._trigger("receive",b,this._uiHash(this))}}.call(this,this.containers[f])),d.push(function(a){return function(
 b){a._trigger("update",b,this._uiHash(this))}}.call(this,this.containers[f])))}for(var f=this.containers.length-1;f>=0;f--)c||d.push(function(a){return function(b){a._trigger("deactivate",b,this._uiHash(this))}}.call(this,this.containers[f])),this.containers[f].containerCache.over&&(d.push(function(a){return function(b){a._trigger("out",b,this._uiHash(this))}}.call(this,this.containers[f])),this.containers[f].containerCache.over=0);this._storedCursor&&a("body").css("cursor",this._storedCursor),this._storedOpacity&&this.helper.css("opacity",this._storedOpacity),this._storedZIndex&&this.helper.css("zIndex",this._storedZIndex=="auto"?"":this._storedZIndex),this.dragging=!1;if(this.cancelHelperRemoval){if(!c){this._trigger("beforeStop",b,this._uiHash());for(var f=0;f<d.length;f++)d[f].call(this,b);this._trigger("stop",b,this._uiHash())}return this.fromOutside=!1,!1}c||this._trigger("beforeStop",b,this._uiHash()),this.placeholder[0].parentNode.removeChild(this.placeholder[0]),this.helper
 [0]!=this.currentItem[0]&&this.helper.remove(),this.helper=null;if(!c){for(var f=0;f<d.length;f++)d[f].call(this,b);this._trigger("stop",b,this._uiHash())}return this.fromOutside=!1,!0},_trigger:function(){a.Widget.prototype._trigger.apply(this,arguments)===!1&&this.cancel()},_uiHash:function(b){var c=b||this;return{helper:c.helper,placeholder:c.placeholder||a([]),position:c.position,originalPosition:c.originalPosition,offset:c.positionAbs,item:c.currentItem,sender:b?b.element:null}}}),a.extend(a.ui.sortable,{version:"1.8.23"})}(jQuery),jQuery.effects||function(a,b){function c(b){var c;return b&&b.constructor==Array&&b.length==3?b:(c=/rgb\(\s*([0-9]{1,3})\s*,\s*([0-9]{1,3})\s*,\s*([0-9]{1,3})\s*\)/.exec(b))?[parseInt(c[1],10),parseInt(c[2],10),parseInt(c[3],10)]:(c=/rgb\(\s*([0-9]+(?:\.[0-9]+)?)\%\s*,\s*([0-9]+(?:\.[0-9]+)?)\%\s*,\s*([0-9]+(?:\.[0-9]+)?)\%\s*\)/.exec(b))?[parseFloat(c[1])*2.55,parseFloat(c[2])*2.55,parseFloat(c[3])*2.55]:(c=/#([a-fA-F0-9]{2})([a-fA-F0-9]{2})([a-fA-F
 0-9]{2})/.exec(b))?[parseInt(c[1],16),parseInt(c[2],16),parseInt(c[3],16)]:(c=/#([a-fA-F0-9])([a-fA-F0-9])([a-fA-F0-9])/.exec(b))?[parseInt(c[1]+c[1],16),parseInt(c[2]+c[2],16),parseInt(c[3]+c[3],16)]:(c=/rgba\(0, 0, 0, 0\)/.exec(b))?e.transparent:e[a.trim(b).toLowerCase()]}function d(b,d){var e;do{e=(a.curCSS||a.css)(b,d);if(e!=""&&e!="transparent"||a.nodeName(b,"body"))break;d="backgroundColor"}while(b=b.parentNode);return c(e)}function h(){var a=document.defaultView?document.defaultView.getComputedStyle(this,null):this.currentStyle,b={},c,d;if(a&&a.length&&a[0]&&a[a[0]]){var e=a.length;while(e--)c=a[e],typeof a[c]=="string"&&(d=c.replace(/\-(\w)/g,function(a,b){return b.toUpperCase()}),b[d]=a[c])}else for(c in a)typeof a[c]=="string"&&(b[c]=a[c]);return b}function i(b){var c,d;for(c in b)d=b[c],(d==null||a.isFunction(d)||c in g||/scrollbar/.test(c)||!/color/i.test(c)&&isNaN(parseFloat(d)))&&delete b[c];return b}function j(a,b){var c={_:0},d;for(d in b)a[d]!=b[d]&&(c[d]=b[d]);retu
 rn c}function k(b,c,d,e){typeof b=="object"&&(e=c,d=null,c=b,b=c.effect),a.isFunction(c)&&(e=c,d=null,c={});if(typeof c=="number"||a.fx.speeds[c])e=d,d=c,c={};return a.isFunction(d)&&(e=d,d=null),c=c||{},d=d||c.duration,d=a.fx.off?0:typeof d=="number"?d:d in a.fx.speeds?a.fx.speeds[d]:a.fx.speeds._default,e=e||c.complete,[b,c,d,e]}function l(b){return!b||typeof b=="number"||a.fx.speeds[b]?!0:typeof b=="string"&&!a.effects[b]?!0:!1}a.effects={},a.each(["backgroundColor","borderBottomColor","borderLeftColor","borderRightColor","borderTopColor","borderColor","color","outlineColor"],function(b,e){a.fx.step[e]=function(a){a.colorInit||(a.start=d(a.elem,e),a.end=c(a.end),a.colorInit=!0),a.elem.style[e]="rgb("+Math.max(Math.min(parseInt(a.pos*(a.end[0]-a.start[0])+a.start[0],10),255),0)+","+Math.max(Math.min(parseInt(a.pos*(a.end[1]-a.start[1])+a.start[1],10),255),0)+","+Math.max(Math.min(parseInt(a.pos*(a.end[2]-a.start[2])+a.start[2],10),255),0)+")"}});var e={aqua:[0,255,255],azure:[240,
 255,255],beige:[245,245,220],black:[0,0,0],blue:[0,0,255],brown:[165,42,42],cyan:[0,255,255],darkblue:[0,0,139],darkcyan:[0,139,139],darkgrey:[169,169,169],darkgreen:[0,100,0],darkkhaki:[189,183,107],darkmagenta:[139,0,139],darkolivegreen:[85,107,47],darkorange:[255,140,0],darkorchid:[153,50,204],darkred:[139,0,0],darksalmon:[233,150,122],darkviolet:[148,0,211],fuchsia:[255,0,255],gold:[255,215,0],green:[0,128,0],indigo:[75,0,130],khaki:[240,230,140],lightblue:[173,216,230],lightcyan:[224,255,255],lightgreen:[144,238,144],lightgrey:[211,211,211],lightpink:[255,182,193],lightyellow:[255,255,224],lime:[0,255,0],magenta:[255,0,255],maroon:[128,0,0],navy:[0,0,128],olive:[128,128,0],orange:[255,165,0],pink:[255,192,203],purple:[128,0,128],violet:[128,0,128],red:[255,0,0],silver:[192,192,192],white:[255,255,255],yellow:[255,255,0],transparent:[255,255,255]},f=["add","remove","toggle"],g={border:1,borderBottom:1,borderColor:1,borderLeft:1,borderRight:1,borderTop:1,borderWidth:1,margin:1,pa
 dding:1};a.effects.animateClass=function(b,c,d,e){return a.isFunction(d)&&(e=d,d=null),this.queue(function(){var g=a(this),k=g.attr("style")||" ",l=i(h.call(this)),m,n=g.attr("class")||"";a.each(f,function(a,c){b[c]&&g[c+"Class"](b[c])}),m=i(h.call(this)),g.attr("class",n),g.animate(j(l,m),{queue:!1,duration:c,easing:d,complete:function(){a.each(f,function(a,c){b[c]&&g[c+"Class"](b[c])}),typeof g.attr("style")=="object"?(g.attr("style").cssText="",g.attr("style").cssText=k):g.attr("style",k),e&&e.apply(this,arguments),a.dequeue(this)}})})},a.fn.extend({_addClass:a.fn.addClass,addClass:function(b,c,d,e){return c?a.effects.animateClass.apply(this,[{add:b},c,d,e]):this._addClass(b)},_removeClass:a.fn.removeClass,removeClass:function(b,c,d,e){return c?a.effects.animateClass.apply(this,[{remove:b},c,d,e]):this._removeClass(b)},_toggleClass:a.fn.toggleClass,toggleClass:function(c,d,e,f,g){return typeof d=="boolean"||d===b?e?a.effects.animateClass.apply(this,[d?{add:c}:{remove:c},e,f,g]):t
 his._toggleClass(c,d):a.effects.animateClass.apply(this,[{toggle:c},d,e,f])},switchClass:function(b,c,d,e,f){return a.effects.animateClass.apply(this,[{add:c,remove:b},d,e,f])}}),a.extend(a.effects,{version:"1.8.23",save:function(a,b){for(var c=0;c<b.length;c++)b[c]!==null&&a.data("ec.storage."+b[c],a[0].style[b[c]])},restore:function(a,b){for(var c=0;c<b.length;c++)b[c]!==null&&a.css(b[c],a.data("ec.storage."+b[c]))},setMode:function(a,b){return b=="toggle"&&(b=a.is(":hidden")?"show":"hide"),b},getBaseline:function(a,b){var c,d;switch(a[0]){case"top":c=0;break;case"middle":c=.5;break;case"bottom":c=1;break;default:c=a[0]/b.height}switch(a[1]){case"left":d=0;break;case"center":d=.5;break;case"right":d=1;break;default:d=a[1]/b.width}return{x:d,y:c}},createWrapper:function(b){if(b.parent().is(".ui-effects-wrapper"))return b.parent();var c={width:b.outerWidth(!0),height:b.outerHeight(!0),"float":b.css("float")},d=a("<div></div>").addClass("ui-effects-wrapper").css({fontSize:"100%",back
 ground:"transparent",border:"none",margin:0,padding:0}),e=document.activeElement;try{e.id}catch(f){e=document.body}return b.wrap(d),(b[0]===e||a.contains(b[0],e))&&a(e).focus(),d=b.parent(),b.css("position")=="static"?(d.css({position:"relative"}),b.css({position:"relative"})):(a.extend(c,{position:b.css("position"),zIndex:b.css("z-index")}),a.each(["top","left","bottom","right"],function(a,d){c[d]=b.css(d),isNaN(parseInt(c[d],10))&&(c[d]="auto")}),b.css({position:"relative",top:0,left:0,right:"auto",bottom:"auto"})),d.css(c).show()},removeWrapper:function(b){var c,d=document.activeElement;return b.parent().is(".ui-effects-wrapper")?(c=b.parent().replaceWith(b),(b[0]===d||a.contains(b[0],d))&&a(d).focus(),c):b},setTransition:function(b,c,d,e){return e=e||{},a.each(c,function(a,c){var f=b.cssUnit(c);f[0]>0&&(e[c]=f[0]*d+f[1])}),e}}),a.fn.extend({effect:function(b,c,d,e){var f=k.apply(this,arguments),g={options:f[1],duration:f[2],callback:f[3]},h=g.options.mode,i=a.effects[b];return a
 .fx.off||!i?h?this[h](g.duration,g.callback):this.each(function(){g.callback&&g.callback.call(this)}):i.call(this,g)},_show:a.fn.show,show:function(a){if(l(a))return this._show.apply(this,arguments);var b=k.apply(this,arguments);return b[1].mode="show",this.effect.apply(this,b)},_hide:a.fn.hide,hide:function(a){if(l(a))return this._hide.apply(this,arguments);var b=k.apply(this,arguments);return b[1].mode="hide",this.effect.apply(this,b)},__toggle:a.fn.toggle,toggle:function(b){if(l(b)||typeof b=="boolean"||a.isFunction(b))return this.__toggle.apply(this,arguments);var c=k.apply(this,arguments);return c[1].mode="toggle",this.effect.apply(this,c)},cssUnit:function(b){var c=this.css(b),d=[];return a.each(["em","px","%","pt"],function(a,b){c.indexOf(b)>0&&(d=[parseFloat(c),b])}),d}});var m={};a.each(["Quad","Cubic","Quart","Quint","Expo"],function(a,b){m[b]=function(b){return Math.pow(b,a+2)}}),a.extend(m,{Sine:function(a){return 1-Math.cos(a*Math.PI/2)},Circ:function(a){return 1-Math.s
 qrt(1-a*a)},Elastic:function(a){return a===0||a===1?a:-Math.pow(2,8*(a-1))*Math.sin(((a-1)*80-7.5)*Math.PI/15)},Back:function(a){return a*a*(3*a-2)},Bounce:function(a){var b,c=4;while(a<((b=Math.pow(2,--c))-1)/11);return 1/Math.pow(4,3-c)-7.5625*Math.pow((b*3-2)/22-a,2)}}),a.each(m,function(b,c){a.easing["easeIn"+b]=c,a.easing["easeOut"+b]=function(a){return 1-c(1-a)},a.easing["easeInOut"+b]=function(a){return a<.5?c(a*2)/2:c(a*-2+2)/-2+1}})}(jQuery),function(a,b){a.effects.blind=function(b){return this.queue(function(){var c=a(this),d=["position","top","bottom","left","right"],e=a.effects.setMode(c,b.options.mode||"hide"),f=b.options.direction||"vertical";a.effects.save(c,d),c.show();var g=a.effects.createWrapper(c).css({overflow:"hidden"}),h=f=="vertical"?"height":"width",i=f=="vertical"?g.height():g.width();e=="show"&&g.css(h,0);var j={};j[h]=e=="show"?i:0,g.animate(j,b.duration,b.options.easing,function(){e=="hide"&&c.hide(),a.effects.restore(c,d),a.effects.removeWrapper(c),b.ca
 llback&&b.callback.apply(c[0],arguments),c.dequeue()})})}}(jQuery),function(a,b){a.effects.bounce=function(b){return this.queue(function(){var c=a(this),d=["position","top","bottom","left","right"],e=a.effects.setMode(c,b.options.mode||"effect"),f=b.options.direction||"up",g=b.options.distance||20,h=b.options.times||5,i=b.duration||250;/show|hide/.test(e)&&d.push("opacity"),a.

<TRUNCATED>

[16/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/proto/TajoWorkerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto
new file mode 100644
index 0000000..7c94e33
--- /dev/null
+++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto
@@ -0,0 +1,258 @@
+/**
+ * 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.
+ */
+
+// TajoMaster -> TajoWorker, TajoWorker(QueryMaster) <-> TajoWorker Protocol
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "TajoWorkerProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "yarn_protos.proto";
+import "tajo_protos.proto";
+import "TajoIdProtos.proto";
+import "CatalogProtos.proto";
+import "PrimitiveProtos.proto";
+
+message SessionProto {
+  required string session_id = 1;
+  required string username = 2;
+  required string current_database = 3;
+  required int64 last_access_time = 4;
+  required KeyValueSetProto variables = 5;
+}
+
+message TaskStatusProto {
+  required QueryUnitAttemptIdProto id = 1;
+  required string workerName = 2;
+  required float progress = 3;
+  required TaskAttemptState state = 4;
+  optional StatSetProto stats = 5;
+  optional TableStatsProto inputStats = 6;
+  optional TableStatsProto resultStats = 7;
+  repeated ShuffleFileOutput shuffleFileOutputs = 8;
+}
+
+message TaskCompletionReport {
+  required QueryUnitAttemptIdProto id = 1;
+  optional StatSetProto stats = 2;
+  optional TableStatsProto inputStats = 3;
+  optional TableStatsProto resultStats = 4;
+  repeated ShuffleFileOutput shuffleFileOutputs = 5;
+}
+
+message TaskFatalErrorReport {
+  required QueryUnitAttemptIdProto id = 1;
+  optional string error_message = 2;
+}
+
+message QueryUnitRequestProto {
+    required QueryUnitAttemptIdProto id = 1;
+    repeated FragmentProto fragments = 2;
+    required string outputTable = 3;
+    required bool clusteredOutput = 4;
+    required string serializedData = 5;
+    optional bool interQuery = 6 [default = false];
+    repeated Fetch fetches = 7;
+    optional bool shouldDie = 8;
+    optional KeyValueSetProto queryContext = 9;
+    optional DataChannelProto dataChannel = 10;
+    optional EnforcerProto enforcer = 11;
+}
+
+message Fetch {
+    required string name = 1;
+    required string urls = 2;
+}
+
+message QueryUnitResponseProto {
+    required string id = 1;
+    required QueryState status = 2;
+}
+
+message StatusReportProto {
+  required int64 timestamp = 1;
+  required string serverName = 2;
+  repeated TaskStatusProto status = 3;
+  repeated QueryUnitAttemptIdProto pings = 4;
+}
+
+message CommandRequestProto {
+    repeated Command command = 1;
+}
+
+message CommandResponseProto {
+}
+
+message Command {
+    required QueryUnitAttemptIdProto id = 1;
+    required CommandType type = 2;
+}
+
+enum CommandType {
+    PREPARE = 0;
+    LAUNCH = 1;
+    STOP = 2;
+    FINALIZE = 3;
+}
+
+message ShuffleFileOutput {
+    required int32 partId = 1;
+    optional string fileName = 2;
+}
+
+message QueryExecutionRequestProto {
+    required QueryIdProto queryId = 1;
+    required SessionProto session = 2;
+    required KeyValueSetProto queryContext = 3;
+    required StringProto sql = 4;
+    optional StringProto logicalPlanJson = 5;
+}
+
+message GetTaskRequestProto {
+    required hadoop.yarn.ContainerIdProto containerId = 1;
+    required ExecutionBlockIdProto executionBlockId = 2;
+}
+
+enum ShuffleType {
+  NONE_SHUFFLE = 0;
+  HASH_SHUFFLE = 1;
+  RANGE_SHUFFLE = 2;
+}
+
+enum TransmitType {
+  PUSH_TRANSMIT = 0;
+  PULL_TRANSMIT = 1;
+  FILE_WRITE = 2;
+}
+
+message DataChannelProto {
+  required ExecutionBlockIdProto srcId = 1;
+  required ExecutionBlockIdProto targetId = 2;
+
+  required TransmitType transmitType = 3 [default = PULL_TRANSMIT];
+  required ShuffleType shuffleType = 4;
+
+  optional SchemaProto schema = 5;
+
+  repeated ColumnProto shuffleKeys = 7;
+  optional int32 numOutputs = 9 [default = 1];
+
+  optional StoreType storeType = 10 [default = CSV];
+}
+
+message RunExecutionBlockRequestProto {
+    required string executionBlockId = 1;
+    required string queryMasterHost = 2;
+    required int32 queryMasterPort = 3;
+    required string nodeId = 4;
+    required string containerId = 5;
+    optional string queryOutputPath = 6;
+}
+
+service TajoWorkerProtocolService {
+  rpc ping (QueryUnitAttemptIdProto) returns (BoolProto);
+
+  // from QueryMaster(Worker)
+  rpc executeExecutionBlock(RunExecutionBlockRequestProto) returns (BoolProto);
+  rpc killTaskAttempt(QueryUnitAttemptIdProto) returns (BoolProto);
+  rpc cleanup(QueryIdProto) returns (BoolProto);
+}
+
+message EnforceProperty {
+  enum EnforceType {
+    SORTED_INPUT = 0;
+    OUTPUT_DISTINCT = 1;
+    GROUP_BY = 2;
+    JOIN = 3;
+    SORT = 4;
+    BROADCAST = 5;
+    COLUMN_PARTITION = 6;
+  }
+
+  // Identifies which field is filled in.
+  required EnforceType type = 1;
+
+  // One of the following will be filled in.
+  optional SortedInputEnforce sortedInput = 2;
+  optional OutputDistinctEnforce outputDistinct = 3;
+  optional GroupbyEnforce groupby = 4;
+  optional JoinEnforce join = 5;
+  optional SortEnforce sort = 6;
+  optional BroadcastEnforce broadcast = 7;
+  optional ColumnPartitionEnforcer columnPartition = 8;
+}
+
+message SortedInputEnforce {
+  required string tableName = 1;
+  repeated SortSpecProto sortSpecs = 2;
+}
+
+message OutputDistinctEnforce {
+}
+
+message JoinEnforce {
+  enum JoinAlgorithm {
+    NESTED_LOOP_JOIN = 0;
+    BLOCK_NESTED_LOOP_JOIN = 1;
+    IN_MEMORY_HASH_JOIN = 2;
+    HYBRID_HASH_JOIN = 3;
+    MERGE_JOIN = 4;
+  }
+
+  required int32 pid = 1;
+  required JoinAlgorithm algorithm = 2;
+}
+
+message GroupbyEnforce {
+  enum GroupbyAlgorithm {
+    HASH_AGGREGATION = 0;
+    SORT_AGGREGATION = 1;
+  }
+
+  required int32 pid = 1;
+  required GroupbyAlgorithm algorithm = 2;
+  repeated SortSpecProto sortSpecs = 3;
+}
+
+message SortEnforce {
+  enum SortAlgorithm {
+    IN_MEMORY_SORT = 0;
+    MERGE_SORT = 1;
+  }
+
+  required int32 pid = 1;
+  required SortAlgorithm algorithm = 2;
+}
+
+message BroadcastEnforce {
+  required string tableName = 1;
+}
+
+message ColumnPartitionEnforcer {
+  enum ColumnPartitionAlgorithm {
+    HASH_PARTITION = 0;
+    SORT_PARTITION = 1;
+  }
+
+  required int32 pid = 1;
+  required ColumnPartitionAlgorithm algorithm = 2;
+}
+
+message EnforcerProto {
+  repeated EnforceProperty properties = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/catalog-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/catalog-default.xml b/tajo-core/src/main/resources/catalog-default.xml
new file mode 100644
index 0000000..7d37dff
--- /dev/null
+++ b/tajo-core/src/main/resources/catalog-default.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+  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.
+  -->
+
+<configuration>
+  <property>
+    <name>tajo.catalog.store.class</name>
+    <value>org.apache.tajo.catalog.store.DerbyStore</value>
+  </property>
+
+  <property>
+    <name>tajo.catalog.uri</name>
+    <value>jdbc:derby:/tmp/tajo-catalog-${user.name}/db;create=true</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/log4j.properties b/tajo-core/src/main/resources/log4j.properties
new file mode 100644
index 0000000..007c8f5
--- /dev/null
+++ b/tajo-core/src/main/resources/log4j.properties
@@ -0,0 +1,28 @@
+##
+# 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.
+#
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=INFO
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/tajo-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/tajo-default.xml b/tajo-core/src/main/resources/tajo-default.xml
new file mode 100644
index 0000000..c49e8e5
--- /dev/null
+++ b/tajo-core/src/main/resources/tajo-default.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+  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.
+  -->
+
+<configuration>
+  <property>
+    <name>tajo.worker.tmpdir.locations</name>
+    <value>/tmp/tajo-${user.name}/tmpdir</value>
+  </property>
+
+  <property>
+    <name>tajo.worker.mode.querymaster</name>
+    <value>true</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>tajo.worker.mode.taskrunner</name>
+    <value>true</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>tajo.querymaster.task-scheduler</name>
+    <value>org.apache.tajo.master.DefaultTaskScheduler</value>
+  </property>
+
+  <property>
+    <name>tajo.querymaster.lazy-task-scheduler.algorithm</name>
+    <value>org.apache.tajo.master.GreedyFragmentScheduleAlgorithm</value>
+  </property>
+
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/tajo-metrics.properties
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/tajo-metrics.properties b/tajo-core/src/main/resources/tajo-metrics.properties
new file mode 100644
index 0000000..4ae6a6c
--- /dev/null
+++ b/tajo-core/src/main/resources/tajo-metrics.properties
@@ -0,0 +1,75 @@
+##
+# 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.
+#
+
+###############################################################################
+# report definition
+# syntax: reporter.<name>=<class>
+reporter.null=org.apache.tajo.util.metrics.reporter.NullReporter
+reporter.file=org.apache.tajo.util.metrics.reporter.MetricsFileScheduledReporter
+reporter.console=org.apache.tajo.util.metrics.reporter.MetricsConsoleScheduledReporter
+reporter.ganglia=org.apache.tajo.util.metrics.reporter.GangliaReporter
+###############################################################################
+
+###############################################################################
+# syntax: <metrics group name>.reporters=<reporter name1>[,<reporter name2>,...]
+# syntax: <metrics group name>.<reporter name>.<options>=<value>
+###############################################################################
+
+###############################################################################
+# tajo master
+###############################################################################
+tajomaster.reporters=null
+
+#tajomaster.reporters=file,console
+#tajomaster.console.period=60
+#tajomaster.file.filename=/tmp/tajo/tajomaster-metrics.out
+#tajomaster.file.period=60
+#tajomaster.ganglia.server=my.ganglia.com
+#tajomaster.ganglia.port=8649
+#tajomaster.ganglia.period=60
+###############################################################################
+
+###############################################################################
+# tajo master-jvm
+###############################################################################
+tajomaster-jvm.reporters=null
+#tajomaster-jvm.reporters=console
+#tajomaster-jvm.console.period=60
+#tajomaster-jvm.file.filename=/tmp/tajo/tajomaster-jvm-metrics.out
+#tajomaster-jvm.file.period=60
+###############################################################################
+
+###############################################################################
+# worker
+###############################################################################
+worker.reporters=null
+#worker.reporters=file,console
+#worker.console.period=60
+#worker.file.filename=/tmp/tajo/worker-metrics.out
+#worker.file.period=60
+###############################################################################
+
+###############################################################################
+# worker-jvm
+###############################################################################
+worker-jvm.reporters=null
+#worker-jvm.reporters=console
+#worker-jvm.console.period=60
+#worker-jvm.file.filename=/tmp/tajo/worker-jvm-metrics.out
+#worker-jvm.file.period=60
+###############################################################################

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/WEB-INF/jetty-web.xml
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/WEB-INF/jetty-web.xml b/tajo-core/src/main/resources/webapps/admin/WEB-INF/jetty-web.xml
new file mode 100644
index 0000000..7221fbc
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/WEB-INF/jetty-web.xml
@@ -0,0 +1,23 @@
+<?xml version="1.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.
+-->
+<!DOCTYPE Configure PUBLIC "-//Mort Bay Consulting//DTD Configure//EN"
+  "http://jetty.mortbay.org/configure.dtd">
+<Configure id="WebAppContext" class="org.mortbay.jetty.webapp.WebAppContext">
+  <Set name="maxFormContentSize" type="int">2147483647</Set>
+</Configure>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/WEB-INF/web.xml b/tajo-core/src/main/resources/webapps/admin/WEB-INF/web.xml
new file mode 100644
index 0000000..dc7c62c
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/WEB-INF/web.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<web-app xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xmlns="http://java.sun.com/xml/ns/javaee" xmlns:web="http://java.sun.com/xml/ns/javaee/web-app_2_5.xsd"
+	xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_2_5.xsd"
+	version="2.5">
+   <display-name>Tajo Master</display-name>
+    <welcome-file-list>
+        <welcome-file>index.jsp</welcome-file>
+    </welcome-file-list>
+</web-app>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/catalogview.jsp b/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
new file mode 100644
index 0000000..29fd05c
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
@@ -0,0 +1,166 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.catalog.CatalogService" %>
+<%@ page import="org.apache.tajo.catalog.Column" %>
+<%@ page import="org.apache.tajo.catalog.TableDesc" %>
+<%@ page import="org.apache.tajo.master.TajoMaster" %>
+<%@ page import="org.apache.tajo.util.FileUtil" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="java.util.Collection" %>
+<%@ page import="java.util.List" %>
+<%@ page import="java.util.Map" %>
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  CatalogService catalog = master.getCatalog();
+
+  String catalogType = request.getParameter("type");
+  if(catalogType != null && "function".equals(catalogType)) {
+%>
+<script type="text/javascript">
+    document.location.href = 'functions.jsp';
+</script>
+    return;
+<%
+  }
+  String selectedDatabase = request.getParameter("database");
+  if(selectedDatabase == null || selectedDatabase.trim().isEmpty()) {
+    selectedDatabase = "default";
+  }
+
+  TableDesc tableDesc = null;
+  String selectedTable = request.getParameter("table");
+  if(selectedTable != null && !selectedTable.trim().isEmpty()) {
+    tableDesc = catalog.getTableDesc(selectedDatabase, selectedTable);
+  } else {
+    selectedTable = "";
+  }
+
+  //TODO filter with database
+  Collection<String> tableNames = catalog.getAllTableNames(selectedDatabase);
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <h3>Catalog</h3>
+  <div>
+    <div style='float:left; margin-right:10px'><a href='catalogview.jsp'>[Table]</a></div>
+    <div style='float:left; margin-right:10px'><a href='functions.jsp'>[Function]</a></div>
+    <div style='clear:both'></div>
+  </div>
+  <p/>
+  <table width="100%" border='0'>
+    <tr>
+      <!-- left -->
+      <td width="20%" valign="top">
+        <div>
+          <b>Database:</b>
+          <select width="190" style="width: 190px" onchange="document.location.href='catalogview.jsp?database=' + this.value">
+            <%
+              for (String databaseName : catalog.getAllDatabaseNames()) {
+                if (selectedDatabase.equals(databaseName)) { %>
+                  <option value="<%=databaseName%>" selected><%=databaseName%>
+                <%} else {%>
+                <option value="<%=databaseName%>"><%=databaseName%></option>
+                <%}
+              }
+            %>
+          </select>
+        </div>
+        <!-- table list -->
+        <div style='margin-top:5px'>
+<%
+  if(tableNames == null || tableNames.isEmpty()) {
+    out.write("No tables");
+  } else {
+%>
+          <table width="100%" border="1" class="border_table">
+            <tr><th>Table Name</th></tr>
+<%
+    for(String eachTableName: tableNames) {
+      String bold = "";
+      if(eachTableName.equals(selectedTable)) {
+        bold = "font-weight:bold";
+      }
+      String detailLink = "catalogview.jsp?database=" + selectedDatabase + "&table=" + eachTableName;
+      out.write("<tr><td><span style='" + bold + "'><a href='" + detailLink + "'>" + eachTableName + "</a></span></td></tr>");
+    }
+%>
+          </table>
+<%
+  }
+%>
+        </div>
+      </td>
+      <!-- right -->
+      <td width="80%" valign="top">
+        <div style='margin-left: 15px'>
+          <div style='font-weight:bold'>Table name: <%=selectedTable%></div>
+          <div style='margin-top:5px'>
+<%
+    if(tableDesc != null) {
+      List<Column> columns = tableDesc.getSchema().getColumns();
+      out.write("<table border='1' class='border_table'><tr><th>No</th><th>Column name</th><th>Type</th></tr>");
+      int columnIndex = 1;
+      for(Column eachColumn: columns) {
+        out.write("<tr><td width='30' align='right'>" + columnIndex + "</td><td width='320'>" + eachColumn.getSimpleName() + "</td><td width='150'>" + eachColumn.getDataType().getType() + "</td></tr>");
+        columnIndex++;
+      }
+
+      String optionStr = "";
+      String prefix = "";
+      for(Map.Entry<String, String> entry: tableDesc.getMeta().toMap().entrySet()) {
+        optionStr += prefix + "'" + entry.getKey() + "'='" + entry.getValue() + "'";
+        prefix = "<br/>";
+      }
+      out.write("</table>");
+%>
+          </div>
+          <div style='margin-top:10px'>
+            <div style=''>Detail</div>
+            <table border="1" class='border_table'>
+              <tr><td width='100'>Table path</td><td width='410'><%=tableDesc.getPath()%></td></tr>
+              <tr><td>Store type</td><td><%=tableDesc.getMeta().getStoreType()%></td></tr>
+              <tr><td># rows</td><td><%=(tableDesc.hasStats() ? ("" + tableDesc.getStats().getNumRows()) : "-")%></td></tr>
+              <tr><td>Volume</td><td><%=(tableDesc.hasStats() ? FileUtil.humanReadableByteCount(tableDesc.getStats().getNumBytes(),true) : "-")%></td></tr>
+              <tr><td>Options</td><td><%=optionStr%></td></tr>
+            </table>
+          </div>
+        </div>
+<%
+    }
+%>
+      </td>
+    </tr>
+  </table>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/cluster.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/cluster.jsp b/tajo-core/src/main/resources/webapps/admin/cluster.jsp
new file mode 100644
index 0000000..f454c66
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/cluster.jsp
@@ -0,0 +1,216 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="java.util.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.master.*" %>
+<%@ page import="org.apache.tajo.master.rm.*" %>
+<%@ page import="org.apache.tajo.util.JSPUtil" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="java.util.*" %>
+
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  Map<String, Worker> workers = master.getContext().getResourceManager().getWorkers();
+  List<String> wokerKeys = new ArrayList<String>(workers.keySet());
+  Collections.sort(wokerKeys);
+
+  int runningQueryMasterTasks = 0;
+
+  Set<Worker> liveWorkers = new TreeSet<Worker>();
+  Set<Worker> deadWorkers = new TreeSet<Worker>();
+  Set<Worker> decommissionWorkers = new TreeSet<Worker>();
+
+  Set<Worker> liveQueryMasters = new TreeSet<Worker>();
+  Set<Worker> deadQueryMasters = new TreeSet<Worker>();
+
+  for(Worker eachWorker: workers.values()) {
+    if(eachWorker.getResource().isQueryMasterMode()) {
+      liveQueryMasters.add(eachWorker);
+      runningQueryMasterTasks += eachWorker.getResource().getNumQueryMasterTasks();
+    }
+
+    if(eachWorker.getResource().isTaskRunnerMode()) {
+      liveWorkers.add(eachWorker);
+    }
+  }
+
+  for (Worker inactiveWorker : master.getContext().getResourceManager().getInactiveWorkers().values()) {
+    WorkerState state = inactiveWorker.getState();
+
+    if (state == WorkerState.LOST) {
+      if (inactiveWorker.getResource().isQueryMasterMode()) {
+        deadQueryMasters.add(inactiveWorker);
+      }
+      if (inactiveWorker.getResource().isTaskRunnerMode()) {
+        deadWorkers.add(inactiveWorker);
+      }
+    } else if (state == WorkerState.DECOMMISSIONED) {
+      decommissionWorkers.add(inactiveWorker);
+    }
+  }
+
+  String deadWorkersHtml = deadWorkers.isEmpty() ? "0": "<font color='red'>" + deadWorkers.size() + "</font>";
+  String deadQueryMastersHtml = deadQueryMasters.isEmpty() ? "0": "<font color='red'>" + deadQueryMasters.size() + "</font>";
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <h2>Query Master</h2>
+  <div>Live:<%=liveQueryMasters.size()%>, Dead: <%=deadQueryMastersHtml%>, QueryMaster Tasks: <%=runningQueryMasterTasks%></div>
+  <h3>Live QueryMasters</h3>
+<%
+  if(liveQueryMasters.isEmpty()) {
+    out.write("No Live QueryMasters\n");
+  } else {
+%>
+  <table width="100%" class="border_table" border="1">
+    <tr><th>No</th><th>QueryMaster</th><th>Client Port</th><th>Running Query</th><th>Heap(free/total/max)</th><th>Heartbeat</th><th>Status</th></tr>
+
+<%
+    int no = 1;
+    for(Worker queryMaster: liveQueryMasters) {
+      WorkerResource resource = queryMaster.getResource();
+          String queryMasterHttp = "http://" + queryMaster.getHostName() + ":" + queryMaster.getHttpPort() + "/index.jsp";
+%>
+    <tr>
+      <td width='30' align='right'><%=no++%></td>
+      <td><a href='<%=queryMasterHttp%>'><%=queryMaster.getHostName() + ":" + queryMaster.getQueryMasterPort()%></a></td>
+      <td width='100' align='center'><%=queryMaster.getClientPort()%></td>
+      <td width='200' align='right'><%=resource.getNumQueryMasterTasks()%></td>
+      <td width='200' align='center'><%=resource.getFreeHeap()/1024/1024%>/<%=resource.getTotalHeap()/1024/1024%>/<%=resource.getMaxHeap()/1024/1024%> MB</td>
+      <td width='100' align='right'><%=JSPUtil.getElapsedTime(queryMaster.getLastHeartbeatTime(), System.currentTimeMillis())%></td>
+      <td width='100' align='center'><%=queryMaster.getState()%></td>
+    </tr>
+<%
+    } //end fo for
+%>
+  </table>
+<%
+    } //end of if
+%>
+
+  <p/>
+
+<%
+  if(!deadQueryMasters.isEmpty()) {
+%>
+  <hr/>
+  <h3>Dead QueryMaster</h3>
+  <table width="300" class="border_table" border="1">
+    <tr><th>No</th><th>QueryMaster</th>
+<%
+      int no = 1;
+      for(Worker queryMaster: deadQueryMasters) {
+%>
+    <tr>
+      <td width='30' align='right'><%=no++%></td>
+      <td><%=queryMaster.getHostName() + ":" + queryMaster.getQueryMasterPort()%></td>
+    </tr>
+<%
+      } //end fo for
+%>
+  </table>
+  <p/>
+<%
+    } //end of if
+%>
+
+  <hr/>
+  <h2>Worker</h2>
+  <div>Live:<%=liveWorkers.size()%>, Dead: <%=deadWorkersHtml%></div>
+  <hr/>
+  <h3>Live Workers</h3>
+<%
+  if(liveWorkers.isEmpty()) {
+    out.write("No Live Workers\n");
+  } else {
+%>
+  <table width="100%" class="border_table" border="1">
+    <tr><th>No</th><th>Worker</th><th>PullServer<br/>Port</th><th>Running Tasks</th><th>Memory Resource<br/>(used/total)</th><th>Disk Resource<br/>(used/total)</th><th>Heap<br/>(free/total/max)</th><th>Heartbeat</th><th>Status</th></tr>
+<%
+    int no = 1;
+    for(Worker worker: liveWorkers) {
+      WorkerResource resource = worker.getResource();
+          String workerHttp = "http://" + worker.getHostName() + ":" + worker.getHttpPort() + "/index.jsp";
+%>
+    <tr>
+      <td width='30' align='right'><%=no++%></td>
+      <td><a href='<%=workerHttp%>'><%=worker.getHostName() + ":" + worker.getPeerRpcPort()%></a></td>
+      <td width='80' align='center'><%=worker.getPullServerPort()%></td>
+      <td width='100' align='right'><%=resource.getNumRunningTasks()%></td>
+      <td width='150' align='center'><%=resource.getUsedMemoryMB()%>/<%=resource.getMemoryMB()%></td>
+      <td width='100' align='center'><%=resource.getUsedDiskSlots()%>/<%=resource.getDiskSlots()%></td>
+      <td width='200' align='center'><%=resource.getFreeHeap()/1024/1024%>/<%=resource.getTotalHeap()/1024/1024%>/<%=resource.getMaxHeap()/1024/1024%> MB</td>
+      <td width='100' align='right'><%=JSPUtil.getElapsedTime(worker.getLastHeartbeatTime(), System.currentTimeMillis())%></td>
+      <td width='100' align='center'><%=worker.getState()%></td>
+    </tr>
+<%
+    } //end fo for
+%>
+    </table>
+<%
+  } //end of if
+%>
+
+  <p/>
+  <hr/>
+  <p/>
+  <h3>Dead Workers</h3>
+
+<%
+    if(deadWorkers.isEmpty()) {
+%>
+  No Dead Workers
+<%
+  } else {
+%>
+  <table width="300" class="border_table" border="1">
+    <tr><th>No</th><th>Worker</th></tr>
+<%
+      int no = 1;
+      for(Worker worker: deadWorkers) {
+        WorkerResource resource = worker.getResource();
+%>
+    <tr>
+      <td width='30' align='right'><%=no++%></td>
+      <td><%=worker.getHostName() + ":" + worker.getPeerRpcPort()%></td>
+    </tr>
+<%
+      } //end fo for
+%>
+  </table>
+<%
+    } //end of if
+%>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/conf.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/conf.jsp b/tajo-core/src/main/resources/webapps/admin/conf.jsp
new file mode 100644
index 0000000..80073d9
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/conf.jsp
@@ -0,0 +1,56 @@
+<%
+  /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements. See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership. The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License. You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.master.*" %>
+<%@ page import="org.apache.tajo.conf.*" %>
+<%@ page import="java.util.Map" %>
+
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  TajoMaster.MasterContext context = master.getContext();
+  TajoConf tajoConf = context.getConf();
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <table width="100%" border="1" class="border_table">
+<%
+  for(Map.Entry<String,String> entry: tajoConf) {
+%>
+    <tr><td width="200"><%=entry.getKey()%></td><td><%=entry.getValue()%></td>
+<%
+  }
+%>
+  </table>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/env.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/env.jsp b/tajo-core/src/main/resources/webapps/admin/env.jsp
new file mode 100644
index 0000000..34965fc
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/env.jsp
@@ -0,0 +1,67 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.master.*" %>
+<%@ page import="java.util.Map" %>
+
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <h3>System Environment</h3>
+  <table width="100%" class="border_table">
+<%
+  for(Map.Entry<String, String> entry: System.getenv().entrySet()) {
+%>
+    <tr><td width="200"><%=entry.getKey()%></td><td><%=entry.getValue()%></td>
+<%
+  }
+%>
+  </table>
+
+  <h3>Properties</h3>
+  <hr/>
+
+  <table width="100%" class="border_table">
+<%
+  for(Map.Entry<Object, Object> entry: System.getProperties().entrySet()) {
+%>
+    <tr><td width="200"><%=entry.getKey()%></td><td><%=entry.getValue()%></td>
+<%
+  }
+%>
+  </table>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/functions.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/functions.jsp b/tajo-core/src/main/resources/webapps/admin/functions.jsp
new file mode 100644
index 0000000..68e127b
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/functions.jsp
@@ -0,0 +1,77 @@
+<%
+    /*
+    * 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.
+    */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="java.util.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.master.*" %>
+<%@ page import="org.apache.tajo.catalog.*" %>
+<%@ page import="org.apache.hadoop.http.HtmlQuoting" %>
+<%@ page import="org.apache.tajo.util.JSPUtil" %>
+<%
+    TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+    CatalogService catalog = master.getCatalog();
+
+    List<FunctionDesc> functions = new ArrayList<FunctionDesc>(catalog.getFunctions());
+    JSPUtil.sortFunctionDesc(functions);
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+    <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+    <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+    <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+    <h2>Tajo Master: <%=master.getMasterName()%></h2>
+    <hr/>
+    <h3>Catalog</h3>
+    <div>
+        <div style='float:left; margin-right:10px'><a href='catalogview.jsp'>[Table]</a></div>
+        <div style='float:left; margin-right:10px'><a href='functions.jsp'>[Function]</a></div>
+        <div style='clear:both'></div>
+    </div>
+    <p/>
+    <table border="1" class='border_table'>
+        <tr><th width='5%'>Name</th><th width='20%'>Signature</th><th width="5%">Type</th><th width='40%'>Description</th><th>Example</th></tr>
+<%
+    for(FunctionDesc eachFunction: functions) {
+        String fullDecription = eachFunction.getDescription();
+        if(eachFunction.getDetail() != null && !eachFunction.getDetail().isEmpty()) {
+            fullDecription += "\n" + eachFunction.getDetail();
+        }
+%>
+        <tr>
+            <td><%=eachFunction.getSignature()%></td>
+            <td><%=eachFunction.getHelpSignature()%></td>
+            <td><%=eachFunction.getFuncType()%></td>
+            <td><%=HtmlQuoting.quoteHtmlChars(fullDecription).replace("\n", "<br/>")%></td>
+            <td><%=HtmlQuoting.quoteHtmlChars(eachFunction.getExample()).replace("\n", "<br/>")%></td>
+        </tr>
+<%
+    }
+%>
+    </table>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/getCSV.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/getCSV.jsp b/tajo-core/src/main/resources/webapps/admin/getCSV.jsp
new file mode 100644
index 0000000..f398304
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/getCSV.jsp
@@ -0,0 +1,31 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language = "java" contentType = "charset=utf-8" %>
+<%
+  response.setHeader("Content-Type", "application/octet-stream;");
+  response.setHeader("Content-Disposition", "attachment; filename=\"result.csv\";");
+  response.setHeader("Content-Description", "JSP Generated Data");
+  response.setHeader("cache-control", "no-cache");
+  response.setHeader("expires", "0");
+  response.setHeader("pragma", "no-cache");
+  out.print(request.getParameter("csvData"));
+  out.flush();
+%>
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/header.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/header.jsp b/tajo-core/src/main/resources/webapps/admin/header.jsp
new file mode 100644
index 0000000..f98c3d5
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/header.jsp
@@ -0,0 +1,32 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+<div class="menu">
+  <div style='float:left; margin-left:12px; margin-top:6px;'><a href='index.jsp'><img src='/static/img/logo_tajo.gif' border='0'/></a></div>
+  <ul>
+    <li><a class='top_menu_item' style='margin-left:10px;' href='index.jsp'>Home</a></li>
+    <li><a class='top_menu_item' href='cluster.jsp'>Cluster</a></li>
+    <li><a class='top_menu_item' href='query.jsp'>Query</a></li>
+    <li><a class='top_menu_item' href='catalogview.jsp'>Catalog</a></li>
+    <li><a class='top_menu_item' href='query_executor.jsp'>Execute Query</a></li>
+  </ul>
+  <br style='clear:left'/>
+</div>
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp
new file mode 100644
index 0000000..ebd017d
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/index.jsp
@@ -0,0 +1,184 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.hadoop.fs.FileSystem" %>
+<%@ page import="org.apache.tajo.conf.TajoConf" %>
+<%@ page import="org.apache.tajo.ipc.TajoMasterProtocol" %>
+<%@ page import="org.apache.tajo.master.TajoMaster" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryInProgress" %>
+<%@ page import="org.apache.tajo.master.rm.Worker" %>
+<%@ page import="org.apache.tajo.master.rm.WorkerState" %>
+<%@ page import="org.apache.tajo.util.NetUtils" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="java.util.Collection" %>
+<%@ page import="java.util.Date" %>
+<%@ page import="java.util.Map" %>
+
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  Map<String, Worker> workers = master.getContext().getResourceManager().getWorkers();
+  Map<String, Worker> inactiveWorkers = master.getContext().getResourceManager().getInactiveWorkers();
+
+  int numWorkers = 0;
+  int numLiveWorkers = 0;
+  int numDeadWorkers = 0;
+  int numDecommissionWorkers = 0;
+
+  int numQueryMasters = 0;
+  int numLiveQueryMasters = 0;
+  int numDeadQueryMasters = 0;
+  int runningQueryMasterTask = 0;
+
+
+  TajoMasterProtocol.ClusterResourceSummary clusterResourceSummary =
+          master.getContext().getResourceManager().getClusterResourceSummary();
+
+  for(Worker eachWorker: workers.values()) {
+    if(eachWorker.getResource().isQueryMasterMode()) {
+      numQueryMasters++;
+      numLiveQueryMasters++;
+      runningQueryMasterTask += eachWorker.getResource().getNumQueryMasterTasks();
+    }
+    if(eachWorker.getResource().isTaskRunnerMode()) {
+      numWorkers++;
+      numLiveWorkers++;
+    }
+  }
+
+  for (Worker eachWorker : inactiveWorkers.values()) {
+    if (eachWorker.getState() == WorkerState.LOST) {
+      if(eachWorker.getResource().isQueryMasterMode()) {
+        numQueryMasters++;
+        numDeadQueryMasters++;
+      }
+      if(eachWorker.getResource().isTaskRunnerMode()) {
+        numWorkers++;
+        numDeadWorkers++;
+      }
+    } else if(eachWorker.getState() == WorkerState.DECOMMISSIONED) {
+      numDecommissionWorkers++;
+    }
+  }
+
+  String numDeadWorkersHtml = numDeadWorkers == 0 ? "0" : "<font color='red'>" + numDeadWorkers + "</font>";
+  String numDeadQueryMastersHtml = numDeadQueryMasters == 0 ? "0" : "<font color='red'>" + numDeadQueryMasters + "</font>";
+
+  Collection<QueryInProgress> runningQueries = master.getContext().getQueryJobManager().getRunningQueries();
+  Collection<QueryInProgress> finishedQueries = master.getContext().getQueryJobManager().getFinishedQueries();
+
+  int avgQueryTime = 0;
+  int minQueryTime = Integer.MAX_VALUE;
+  int maxQueryTime = 0;
+
+  long totalTime = 0;
+  for(QueryInProgress eachQuery: finishedQueries) {
+    int runTime = (int)(eachQuery.getQueryInfo().getFinishTime() == 0 ? -1 :
+            eachQuery.getQueryInfo().getFinishTime() - eachQuery.getQueryInfo().getStartTime());
+    if(runTime > 0) {
+      totalTime += runTime;
+
+      if(runTime < minQueryTime) {
+        minQueryTime = runTime;
+      }
+
+      if(runTime > maxQueryTime) {
+        maxQueryTime = runTime;
+      }
+    }
+  }
+
+  if(minQueryTime == Integer.MAX_VALUE) {
+    minQueryTime = 0;
+  }
+  if(finishedQueries.size() > 0) {
+    avgQueryTime = (int)(totalTime / (long)finishedQueries.size());
+  }
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <h3>Master Status</h3>
+  <table border='0'>
+    <tr><td width='150'>Version:</td><td><%=master.getVersion()%></td></tr>
+    <tr><td width='150'>Started:</td><td><%=new Date(master.getStartTime())%></td></tr>
+    <tr><td width='150'>File System:</td><td><%=master.getContext().getConf().get(FileSystem.FS_DEFAULT_NAME_KEY)%></td></tr>
+    <tr><td width='150'>Root dir:</td><td><%=TajoConf.getTajoRootDir(master.getContext().getConf())%></td></tr>
+    <tr><td width='150'>System dir:</td><td><%=TajoConf.getSystemDir(master.getContext().getConf())%></td></tr>
+    <tr><td width='150'>Warehouse dir:</td><td><%=TajoConf.getWarehouseDir(master.getContext().getConf())%></td></tr>
+    <tr><td width='150'>Staging dir:</td><td><%=TajoConf.getStagingDir(master.getContext().getConf())%></td></tr>
+    <tr><td width='150'>Client Service:</td><td><%=NetUtils.normalizeInetSocketAddress(master.getTajoMasterClientService().getBindAddress())%></td></tr>
+    <tr><td width='150'>Catalog Service:</td><td><%=master.getCatalogServer().getCatalogServerName()%></td></tr>
+    <tr><td width='150'>Heap(Free/Total/Max): </td><td><%=Runtime.getRuntime().freeMemory()/1024/1024%> MB / <%=Runtime.getRuntime().totalMemory()/1024/1024%> MB / <%=Runtime.getRuntime().maxMemory()/1024/1024%> MB</td>
+    <tr><td width='150'>Configuration:</td><td><a href='conf.jsp'>detail...</a></td></tr>
+    <tr><td width='150'>Environment:</td><td><a href='env.jsp'>detail...</a></td></tr>
+    <tr><td width='150'>Threads:</td><td><a href='thread.jsp'>thread dump...</a></tr>
+  </table>
+  <hr/>
+
+  <h3>Cluster Summary</h3>
+  <table width="100%" class="border_table" border="1">
+    <tr><th>Type</th><th>Total</th><th>Live</th><th>Dead</th><th>Running Master</th><th>Memory Resource<br/>(used/total)</th><th>Disk Resource<br/>(used/total)</th></tr>
+    <tr>
+      <td><a href='cluster.jsp'>Query Master</a></td>
+      <td align='right'><%=numQueryMasters%></td>
+      <td align='right'><%=numLiveQueryMasters%></td>
+      <td align='right'><%=numDeadQueryMastersHtml%></td>
+      <td align='right'><%=runningQueryMasterTask%></td>
+      <td align='center'>-</td>
+      <td align='center'>-</td>
+    </tr>
+    <tr>
+      <td><a href='cluster.jsp'>Worker</a></td>
+      <td align='right'><%=numWorkers%></td>
+      <td align='right'><%=numLiveWorkers%></td>
+      <td align='right'><%=numDeadWorkersHtml%></td>
+      <td align='right'>-</td>
+      <td align='center'><%=clusterResourceSummary.getTotalMemoryMB() - clusterResourceSummary.getTotalAvailableMemoryMB()%>/<%=clusterResourceSummary.getTotalMemoryMB()%></td>
+      <td align='center'><%=clusterResourceSummary.getTotalDiskSlots() - clusterResourceSummary.getTotalAvailableDiskSlots()%>/<%=clusterResourceSummary.getTotalDiskSlots()%></td>
+    </tr>
+  </table>
+  <p/>
+  <hr/>
+
+  <h3>Query Summary</h3>
+  <table width="100%" class="border_table" border="1">
+    <tr><th>Running Queries</th><th>Finished Queries</th><th>Average Execution Time</th><th>Min. Execution Time</th><th>Max. Execution Time</th></tr>
+    <tr>
+      <td align='right'><%=runningQueries.size()%></td>
+      <td align='right'><%=finishedQueries.size()%></td>
+      <td align='left'><%=avgQueryTime/1000%> sec</td>
+      <td align='left'><%=minQueryTime/1000%> sec</td>
+      <td align='left'><%=maxQueryTime/1000%> sec</td>
+    </tr>
+  </table>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/query.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/query.jsp b/tajo-core/src/main/resources/webapps/admin/query.jsp
new file mode 100644
index 0000000..6f15a0e
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/query.jsp
@@ -0,0 +1,137 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.master.TajoMaster" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryInProgress" %>
+<%@ page import="org.apache.tajo.master.rm.Worker" %>
+<%@ page import="org.apache.tajo.util.JSPUtil" %>
+<%@ page import="org.apache.tajo.util.StringUtils" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+<%@ page import="java.util.Collection" %>
+<%@ page import="java.util.HashMap" %>
+<%@ page import="java.util.List" %>
+<%@ page import="java.util.Map" %>
+
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+
+  List<QueryInProgress> runningQueries =
+          JSPUtil.sortQueryInProgress(master.getContext().getQueryJobManager().getRunningQueries(), true);
+
+  List<QueryInProgress> finishedQueries =
+          JSPUtil.sortQueryInProgress(master.getContext().getQueryJobManager().getFinishedQueries(), true);
+
+  SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  Map<String, Worker> workers = master.getContext().getResourceManager().getWorkers();
+  Map<String, Integer> portMap = new HashMap<String, Integer>();
+
+  Collection<String> queryMasters = master.getContext().getResourceManager().getQueryMasters();
+  if (queryMasters == null || queryMasters.isEmpty()) {
+    queryMasters = master.getContext().getResourceManager().getWorkers().keySet();
+  }
+  for(String eachQueryMasterKey: queryMasters) {
+    Worker queryMaster = workers.get(eachQueryMasterKey);
+    if(queryMaster != null) {
+      portMap.put(queryMaster.getHostName(), queryMaster.getHttpPort());
+    }
+  }
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <h3>Running Queries</h3>
+<%
+  if(runningQueries.isEmpty()) {
+    out.write("No running queries");
+  } else {
+%>
+  <table width="100%" border="1" class='border_table'>
+    <tr></tr><th>QueryId</th><th>Query Master</th><th>Started</th><th>Progress</th><th>Time</th><th>Status</th></th><th>sql</th></tr>
+    <%
+      for(QueryInProgress eachQuery: runningQueries) {
+        long time = System.currentTimeMillis() - eachQuery.getQueryInfo().getStartTime();
+        String detailView = "http://" + eachQuery.getQueryInfo().getQueryMasterHost() + ":" + portMap.get(eachQuery.getQueryInfo().getQueryMasterHost()) +
+                "/querydetail.jsp?queryId=" + eachQuery.getQueryId();
+    %>
+    <tr>
+      <td><a href='<%=detailView%>'><%=eachQuery.getQueryId()%></a></td>
+      <td><%=eachQuery.getQueryInfo().getQueryMasterHost()%></td>
+      <td><%=df.format(eachQuery.getQueryInfo().getStartTime())%></td>
+      <td><%=(int)(eachQuery.getQueryInfo().getProgress() * 100.0f)%>%</td>
+      <td><%=StringUtils.formatTime(time)%></td>
+      <td><%=eachQuery.getQueryInfo().getQueryState()%></td>
+      <td><%=eachQuery.getQueryInfo().getSql()%></td>
+    </tr>
+    <%
+      }
+    %>
+  </table>
+<%
+  }
+%>
+  <p/>
+  <hr/>
+  <h3>Finished Queries</h3>
+  <%
+    if(finishedQueries.isEmpty()) {
+      out.write("No finished queries");
+    } else {
+  %>
+  <table width="100%" border="1" class='border_table'>
+    <tr></tr><th>QueryId</th><th>Query Master</th><th>Started</th><th>Finished</th><th>Time</th><th>Status</th><th>sql</th></tr>
+    <%
+      for(QueryInProgress eachQuery: finishedQueries) {
+        long runTime = eachQuery.getQueryInfo().getFinishTime() > 0 ?
+                eachQuery.getQueryInfo().getFinishTime() - eachQuery.getQueryInfo().getStartTime() : -1;
+        String detailView = "http://" + eachQuery.getQueryInfo().getQueryMasterHost() + ":" + portMap.get(eachQuery.getQueryInfo().getQueryMasterHost())  +
+                "/querydetail.jsp?queryId=" + eachQuery.getQueryId();
+    %>
+    <tr>
+      <td><a href='<%=detailView%>'><%=eachQuery.getQueryId()%></a></td>
+      <td><%=eachQuery.getQueryInfo().getQueryMasterHost()%></td>
+      <td><%=df.format(eachQuery.getQueryInfo().getStartTime())%></td>
+      <td><%=eachQuery.getQueryInfo().getFinishTime() > 0 ? df.format(eachQuery.getQueryInfo().getFinishTime()) : "-"%></td>
+      <td><%=runTime == -1 ? "-" : StringUtils.formatTime(runTime) %></td>
+      <td><%=eachQuery.getQueryInfo().getQueryState()%></td>
+      <td><%=eachQuery.getQueryInfo().getSql()%></td>
+    </tr>
+    <%
+      }
+    %>
+  </table>
+<%
+  }
+%>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/query_executor.jsp b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
new file mode 100644
index 0000000..736b202
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
@@ -0,0 +1,312 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+<%@ page import="org.apache.tajo.master.TajoMaster" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+<link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+<meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+<title>Tajo</title>
+<style type="text/css">
+  #progress_bar {
+    border:1px solid #000000;
+    background:#ffffff;
+    width:400px;
+    height:16px;
+    border-radius: 16px;
+  }
+  #progress_status {background:#fbcb46; width:0%; height:16px; border-radius: 10px; }
+</style>
+<script src="/static/js/jquery.js" type="text/javascript"></script>
+<script type="text/javascript">
+var progressInterval = 1000;
+var progressTimer = null;
+var queryRunnerId = null;
+var PRINT_LIMIT = 25;
+var SIZE_LIMIT = 104857600; // Limit size of displayed results.(Bytes)
+var pageNum = 0;
+var pageCount, storedColumns, storedData;
+
+$(document).ready(function() {
+  $('#btnSubmit').click(function() {
+    runQuery();
+  });
+});
+
+function init() {
+  $("#progress_status").css("width", "0%");
+  $("#progress_text").text("0%");
+  $("#queryStatus").html("");
+  $("#queryResult").html("");
+  queryRunnerId = null;
+}
+
+function runQuery() {
+  if(Math.ceil(Number($("#sizeLimit").val())) >= 2048) {
+    SIZE_LIMIT = 2048 * 1024 * 1024 - 1;
+  } else if(Math.ceil(Number($("#sizeLimit").val())) > 0) {
+    SIZE_LIMIT = Number($("#sizeLimit").val()) * 1024 * 1024;
+  }
+  if(Math.ceil(Number($("#printLimit").val())) > 0) {
+    PRINT_LIMIT = Number($("#printLimit").val());
+  }
+  if(progressTimer != null) {
+    alert("Already query running.");
+    return;
+  }
+  init();
+  var query = $("#query").val();
+
+  $.ajax({
+    type: "POST",
+    url: "query_exec",
+    data: { action: "runQuery", query: query, limitSize:SIZE_LIMIT }
+  })
+  .done(function(msg) {
+    var resultJson = $.parseJSON(msg);
+    if(resultJson.success == "false") {
+      clearTimer();
+      alert(resultJson.errorMessage);
+      return;
+    }
+    queryRunnerId = resultJson.queryRunnerId;
+    progressTimer = setInterval(function () {
+      $.ajax({
+        type: "GET",
+        url: "query_exec",
+        data: { action: "getQueryProgress", queryRunnerId: queryRunnerId }
+      })
+      .done(function( msg ) {
+        var resultJson = $.parseJSON(msg);
+        if(resultJson.success == "false") {
+          clearTimer();
+          alert(resultJson.errorMessage);
+          $("#queryStatus").html(getQueryStatusHtml(resultJson));
+          return;
+        }
+        var progress = parseInt(resultJson.progress, 0);
+        $("#progress_status").css("width", progress + "%");
+        $("#progress_text").text(progress + "%");
+        $("#queryStatus").html(getQueryStatusHtml(resultJson));
+        if(progress >= 100) {
+          clearTimer();
+          getResult();
+        }
+      });
+    }, progressInterval);
+  });
+}
+
+function clearTimer() {
+  if(progressTimer != null) {
+    clearInterval(progressTimer);
+  }
+  progressTimer = null;
+}
+
+function getQueryStatusHtml(status) {
+  if(status.success == "false") {
+    return "<div style='color:#ff0000; margin-top: 5px'>" + status.errorMessage + "</div>";
+  } else {
+    var statusHtml = "<div style='margin-top: 5px'>Start: " + status.startTime + "</div>";
+    statusHtml += "<div style='margin-top: 5px'>Finish: " + status.finishTime + "</div>";
+    statusHtml += "<div style='margin-top: 5px'> Running time: " + status.runningTime + "</div>";
+    return statusHtml;
+  }
+}
+
+function getResult() {
+  $.ajax({
+    type: "POST",
+    url: "query_exec",
+    data: { action: "getQueryResult", queryRunnerId: queryRunnerId }
+  })
+  .done(function(msg) {
+    var printedLine = 0;
+    var resultJson = $.parseJSON(msg);
+    if(resultJson.success == "false") {
+      alert(resultJson.errorMessage);
+      $("#queryStatus").html(getQueryStatusHtml(resultJson));
+      return;
+    }
+    $("#queryResult").html("");
+    var resultColumns = resultJson.resultColumns;
+    var resultData = resultJson.resultData;
+	
+    storedColumns = resultColumns;
+    storedData = resultData; 
+    pageCount = Math.ceil((storedData.length / PRINT_LIMIT)) - 1 ;
+	
+    var resultTable = "<table width='100%' class='border_table'><tr>";
+    for(var i = 0; i < resultColumns.length; i++) {
+      resultTable += "<th>" + resultColumns[i] + "</th>";
+    }
+    resultTable += "</tr>";
+    for(var i = 0; i < resultData.length; i++) {
+      resultTable += "<tr>";
+      for(var j = 0; j < resultData[i].length; j++) {
+        resultTable += "<td>" + resultData[i][j] + "</td>";
+      }
+      resultTable += "</tr>";	  
+      if(++printedLine >= PRINT_LIMIT) break;
+    }
+    resultTable += "</table>";
+    $("#queryResult").html(resultTable);
+    $("#queryResultTools").html("");
+    $("#queryResultTools").append("<input type='button' value='Download to CSV' onclick='getCSV();'/> ");
+    $("#queryResultTools").append("<input type='button' value='Prev' onclick='getPrev();'/> ");
+    $("#queryResultTools").append("<input type='button' value='Next' onclick='getNext();'/> ");
+    var selectPage = "<select id='selectPage'>";
+    for(var i = 0; i <= pageCount; i++) {
+      selectPage += "<option value="+i+">"+(i+1)+"</option>";
+    }
+    selectPage += "</select>";
+    $("#queryResultTools").append(selectPage);
+    $("#selectPage").change(getSelectedPage);
+  })
+}
+
+function getCSV() {
+  var csvData = "";
+  var rowCount = storedData.length;
+  var colCount = storedColumns.length;
+  for(var colIndex = 0; colIndex < colCount; colIndex++) {
+    if(colIndex == 0) {
+      csvData += storedColumns[colIndex];  
+    } else {
+      csvData += "," + storedColumns[colIndex];
+    }
+  }
+  csvData += "\n";
+  for(var rowIndex=0; rowIndex < rowCount; rowIndex++) {
+    for(var colIndex = 0; colIndex < colCount; colIndex++){
+      if(colIndex == 0) {
+        csvData += storedData[rowIndex][colIndex];
+      } else {
+        csvData += "," + storedData[rowIndex][colIndex];
+      }
+    }   
+    csvData += "\n";
+  }
+  $("#csvData").val(csvData);
+  $("#dataForm").submit();  
+}
+
+function getNext() {
+	var printedLine = 0;	
+	if(pageCount > pageNum) {
+		pageNum++;
+		document.getElementById("selectPage").options.selectedIndex = pageNum;
+	}else {
+		alert("There's no next page.");
+		return;
+	}
+	getPage();
+}
+
+function getPrev() {
+	if(pageNum > 0  ) {
+		pageNum--;
+		document.getElementById("selectPage").options.selectedIndex = pageNum;
+	} else {
+		alert("There's no previous page.");
+		return;
+	}
+	getPage();
+}
+
+function getSelectedPage() {
+  if(pageNum >= 0 &&  pageNum <= pageCount ) {
+    pageNum = $("#selectPage option:selected").val();
+  } else {
+    alert("Out of range.");
+    return;
+  }
+  getPage();
+}
+
+function getPage() {
+  var printedLine = 0;
+  $("#queryResult").html("");
+  var resultTable = "<table width='100%' class='border_table'><tr>";
+  for(var i = 0; i < storedColumns.length; i++) {
+    resultTable += "<th>" + storedColumns[i] + "</th>";
+  }
+  resultTable += "</tr>";
+  for(var i = pageNum * PRINT_LIMIT; i < storedData.length; i++) {
+    resultTable += "<tr>";
+    for(var j = 0; j < storedData[i].length; j++) {
+      resultTable += "<td>" + storedData[i][j] + "</td>";
+    }
+    resultTable += "</tr>";
+    if(++printedLine >= PRINT_LIMIT) break;
+  }
+  resultTable += "</table>";
+  $("#queryResult").html(resultTable);
+}
+
+</script>
+</head>
+
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <h3>Query</h3>
+  <textarea id="query" style="width:800px; height:250px; font-family:Tahoma; font-size:12px;"></textarea>
+  <p />
+  Limit : <input id="sizeLimit" type="text" value="10" style="width:30px; text-align:center;" /> MB
+  <p />
+  Rows/Page : <input id="printLimit" type="text" value="25" style="width:30px; text-align:center;" />
+  <hr />
+  <input id="btnSubmit" type="submit" value="Submit">
+  <hr/>
+  <div>
+    <div style="float:left; width:60px">Progress:</div>
+    <div style='float:left; margin-left:10px'>
+      <div id='progress_bar'>
+        <div id='progress_status'></div>
+      </div>
+    </div>
+    <div style='float:left; margin-left:10px;'>
+      <div id="progress_text" style='font-family:Tahoma; font-size:14px; color:#000000; font-weight:bold'>0%</div>
+    </div>
+    <div style='clear:both'></div>
+  </div>
+  <div id="queryStatus">
+  </div>
+  <hr/>
+  <h3>Query Result</h3>
+  <div id="queryResult"></div>
+  <hr/>
+  <div id="queryResultTools"></div>
+  <hr/>
+  <div style="dispaly:none;"><form name="dataForm" id="dataForm" method="post" action="getCSV.jsp"><input type="hidden" id="csvData" name="csvData" value="" /></div>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/admin/thread.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/thread.jsp b/tajo-core/src/main/resources/webapps/admin/thread.jsp
new file mode 100644
index 0000000..3eddef2
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/admin/thread.jsp
@@ -0,0 +1,45 @@
+<%
+  /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements. See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership. The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License. You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.master.*" %>
+
+<%
+  TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Master: <%=master.getMasterName()%></h2>
+  <hr/>
+  <h3>Thread Dump</h3>
+  <pre><%master.dumpThread(out);%></pre>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/logo_tajo.gif
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/logo_tajo.gif b/tajo-core/src/main/resources/webapps/static/img/logo_tajo.gif
new file mode 100644
index 0000000..13674e3
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/logo_tajo.gif differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/on.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/on.jpg b/tajo-core/src/main/resources/webapps/static/img/on.jpg
new file mode 100644
index 0000000..dc96459
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/on.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajo_logo.png
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajo_logo.png b/tajo-core/src/main/resources/webapps/static/img/tajo_logo.png
new file mode 100644
index 0000000..cfb10dd
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajo_logo.png differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar.jpg
new file mode 100644
index 0000000..1dec9d0
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_catalog_small.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_catalog_small.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_catalog_small.jpg
new file mode 100644
index 0000000..2a1cb1e
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_catalog_small.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_little.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_little.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_little.jpg
new file mode 100644
index 0000000..f5c958f
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_little.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_middle.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_middle.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_middle.jpg
new file mode 100644
index 0000000..3e39d9d
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_middle.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_queries_small.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_queries_small.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_queries_small.jpg
new file mode 100644
index 0000000..ace30a4
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_queries_small.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_small.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_small.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_small.jpg
new file mode 100644
index 0000000..ae5fda2
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_small.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_title.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_title.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_title.jpg
new file mode 100644
index 0000000..7dcf45f
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_title.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_title_small.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_title_small.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_title_small.jpg
new file mode 100644
index 0000000..f951f49
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_title_small.jpg differ

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/img/tajochar_worker_small.jpg
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/img/tajochar_worker_small.jpg b/tajo-core/src/main/resources/webapps/static/img/tajochar_worker_small.jpg
new file mode 100644
index 0000000..8d883b6
Binary files /dev/null and b/tajo-core/src/main/resources/webapps/static/img/tajochar_worker_small.jpg differ


[43/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/FindInSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/FindInSet.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/FindInSet.java
new file mode 100644
index 0000000..0c3e221
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/FindInSet.java
@@ -0,0 +1,116 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * find_in_set(text,str_array) - Returns the first occurrence of str in str_array where str_array
+ * is a comma-delimited string.
+ *
+ * Returns null if either argument is null.
+ * Returns 0 if the first argument has any commas.
+ *
+ * Example:
+ * SELECT find_in_set('cr','crt,c,cr,c,def') FROM src LIMIT 1;\n"
+ * -> result: 3
+ */
+@Description(
+  functionName = "find_in_set",
+  description = "Returns the first occurrence of str in str_array where str_array is a comma-delimited string",
+  detail = "Returns null if either argument is null.\n"
+      + "Returns 0 if the first argument has any commas.",
+  example = "> SELECT find_in_set('cr','crt,c,cr,c,def');\n"
+          + "3",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
+)
+public class FindInSet extends GeneralFunction {
+  public FindInSet() {
+    super(new Column[]{
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("str_array", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum finding = params.get(0);
+    Datum textArray = params.get(1);
+
+    if (finding instanceof NullDatum || textArray instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    byte[] searchBytes = finding.asByteArray();
+
+    //  Returns 0 if the first argument has any commas.
+    for (int i = 0; i < finding.size(); i++) {
+      if (searchBytes[i] == ',') {
+        return DatumFactory.createInt4(0);
+      }
+    }
+
+    byte[] arrayData = textArray.asByteArray();
+    int findingLength = finding.size();
+
+    int posInTextArray = 0;
+    int curLengthOfCandidate = 0;
+    boolean matching = true;
+
+    for (int i = 0; i < textArray.size(); i++) {
+
+      if (arrayData[i] == ',') {
+        posInTextArray++;
+        if (matching && curLengthOfCandidate == findingLength) {
+          return DatumFactory.createInt4(posInTextArray);
+        } else {
+          matching = true;
+          curLengthOfCandidate = 0;
+        }
+      } else {
+        if (curLengthOfCandidate + 1 <= findingLength) {
+          if (!matching || searchBytes[curLengthOfCandidate] != arrayData[i]) {
+            matching = false;
+          }
+        } else {
+          matching = false;
+        }
+        curLengthOfCandidate++;
+      }
+
+    }
+
+    if (matching && curLengthOfCandidate == findingLength) {
+      posInTextArray++;
+      return DatumFactory.createInt4(posInTextArray);
+    } else {
+      return DatumFactory.createInt4(0);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/HexStringConverter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/HexStringConverter.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/HexStringConverter.java
new file mode 100644
index 0000000..3b33359
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/HexStringConverter.java
@@ -0,0 +1,65 @@
+/**
+ * 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.function.string;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class HexStringConverter {
+  private static HexStringConverter hexStringConverter = null;
+
+  public static HexStringConverter getInstance() {
+    if (hexStringConverter==null)
+      hexStringConverter = new HexStringConverter();
+    return hexStringConverter;
+  }
+
+  private HexStringConverter() {
+  }
+
+  public String encodeHex(String str) {
+    StringBuffer buf = new StringBuffer();
+
+    for(int i=0; i<str.length(); i++) {
+      String tmp = Integer.toHexString(str.charAt(i));
+      if(tmp.length() == 1)
+        buf.append("0x0" + tmp);
+      else
+        buf.append("0x" + tmp);
+    }
+
+    return buf.toString();
+  }
+
+  public String decodeHex(String hexString) {
+    Pattern p = Pattern.compile("(0x([a-fA-F0-9]{2}([a-fA-F0-9]{2})?))");
+    Matcher m = p.matcher(hexString);
+
+    StringBuffer buf = new StringBuffer();
+    int hashCode = 0;
+    while( m.find() ) {
+      hashCode = Integer.decode("0x" + m.group(2));
+      m.appendReplacement(buf, new String( Character.toChars(hashCode)));
+    }
+
+    m.appendTail(buf);
+
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/InitCap.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/InitCap.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/InitCap.java
new file mode 100644
index 0000000..4347dbb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/InitCap.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.string;
+
+import org.apache.commons.lang.WordUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text initcap(string text)
+ */
+@Description(
+  functionName = "initcap",
+  description = "Convert the first letter of each word to upper case "
+          + " and the rest to lower case..",
+  example = "> SELECT initcap('hi THOMAS');\n"
+          + "Hi Thomas",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class InitCap extends GeneralFunction {
+  public InitCap() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(WordUtils.capitalizeFully(datum.asChars()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
new file mode 100644
index 0000000..db6b714
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.eval.FunctionEval;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text ltrim(string text [, characters text])
+ */
+@Description(
+  functionName = "ltrim",
+  description = "Remove the longest string containing only "
+          + "characters from characters (a space by default) "
+          + "from the start of string.",
+  example = "> SELECT ltrim('zzzytrim', 'xyz');\n"
+          + "trim",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,TajoDataTypes.Type.TEXT})}
+)
+public class LTrim extends GeneralFunction {
+  @Expose private boolean hasTrimCharacters;
+
+  public LTrim() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("characters", TajoDataTypes.Type.TEXT),
+    });
+  }
+
+  public void init(FunctionEval.ParamType[] paramTypes) {
+    if (paramTypes.length == 2) {
+      hasTrimCharacters = true;
+    }
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    if (!hasTrimCharacters) {
+      return DatumFactory.createText(StringUtils.stripStart(datum.asChars(), null));
+    } else {
+      return DatumFactory.createText(StringUtils.stripStart(datum.asChars(), params.get(1).asChars()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Left.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Left.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Left.java
new file mode 100644
index 0000000..31469f5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Left.java
@@ -0,0 +1,84 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.datum.TextDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text left(string text, int size)
+ */
+@Description(
+  functionName = "left",
+  description = "First n characters in the string.",
+  example = "> SELECT left('ABC', 2);\n"
+          + "AB",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,
+          TajoDataTypes.Type.INT4})}
+)
+public class Left extends GeneralFunction {
+  public Left() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("size", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  public int getSize(int length, int size) {
+    if (size < 0) {
+        size = length + size;
+        if (size < 0) {
+            size = 0;
+        }
+    }
+
+    return (size < length) ? size : length;
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    Datum sizeDatum = params.get(1);
+    if(sizeDatum instanceof NullDatum) return NullDatum.get();
+
+    String data = datum.asChars();
+    int length = data.length();
+    int size = sizeDatum.asInt4();
+
+    size = getSize(length, size);
+    if (size == 0) {
+        return TextDatum.EMPTY_TEXT;
+    }
+
+    return DatumFactory.createText(data.substring(0, size));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Length.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Length.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Length.java
new file mode 100644
index 0000000..389f358
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Length.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT4 length(string text)
+ */
+@Description(
+  functionName = "length",
+  description = "Number of characters in string.",
+  example = "> SELECT length('Tajo');\n"
+          + "4",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class Length  extends GeneralFunction {
+
+  public Length() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt4(datum.asChars().length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Locate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Locate.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Locate.java
new file mode 100644
index 0000000..67ee389
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Locate.java
@@ -0,0 +1,106 @@
+/**
+ * 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.function.string;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition.
+ * 
+ * INT4 locate(string TEXT, substr TEXT, [, pos INT4])
+ */
+@Description(
+  functionName = "locate",
+  description = "Location of specified substring",
+  example = "> SELECT locate('high', 'ig')\n"
+          + "2",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT, TajoDataTypes.Type.INT4})
+  }
+)
+public class Locate extends GeneralFunction {
+  public Locate() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("substr", TajoDataTypes.Type.TEXT),
+        new Column("pos", TajoDataTypes.Type.INT4)
+    });
+  }
+  
+  /**
+   * Returns the position of the first occurance of substr in string after position pos (using one-based index).
+   * 
+   * if substr is empty string, it always matches except 
+   * pos is greater than string length + 1.(mysql locate() function spec.)
+   * At any not matched case, it returns 0.
+   */
+  private int locate(String str, String substr, int pos) {
+    if (substr.length() == 0) {
+      if (pos <= (str.length() + 1)) {
+        return pos;
+      }
+      else {
+        return 0;
+      }
+    }
+    int idx = StringUtils.indexOf(str, substr, pos - 1);
+    if (idx == -1) {
+      return 0;
+    }
+    return idx + 1;
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum strDatum = params.get(0);
+    if(strDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+    Datum substrDatum = params.get(1);
+    if (substrDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+    
+    int pos = 1;  // one-based index
+    if (params.size() > 2) {
+      pos = params.get(2).asInt4();
+      if (pos < 0) {
+        return DatumFactory.createInt4(0);  // negative value is not acceptable.
+      }
+      if (pos == 0) {
+        pos = 1;  // one-based index
+      }
+    }
+    
+    String str = strDatum.asChars();
+    String substr = substrDatum.asChars();
+    
+    return DatumFactory.createInt4(locate(str, substr, pos));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lower.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lower.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lower.java
new file mode 100644
index 0000000..80c8192
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lower.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text lower(string text)
+ */
+@Description(
+  functionName = "lower",
+  description = "Convert string to lower case",
+  example = "> SELECT lower('ToM');\n"
+          + "tom",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class Lower extends GeneralFunction {
+  public Lower() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(datum.asChars().toLowerCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
new file mode 100644
index 0000000..bc0031e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
@@ -0,0 +1,90 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.eval.FunctionEval;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text lpad(string text, length int [, fill text])
+ */
+@Description(
+  functionName = "lpad",
+  description = "Fill up the string to length length by prepending the characters fill (a space by default)",
+  detail = "If the string is already longer than length then it is truncated (on the right)",
+  example = "> SELECT lpad('hi', 5, 'xy');\n"
+      + "xyxhi",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.INT4}),
+                @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.INT4, TajoDataTypes.Type.TEXT})}
+)
+public class Lpad extends GeneralFunction {
+  @Expose private boolean hasFillCharacters;
+
+  public Lpad() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("length", TajoDataTypes.Type.INT4),
+        new Column("fill_text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  public void init(FunctionEval.ParamType[] paramTypes) {
+    if (paramTypes.length == 3) {
+      hasFillCharacters = true;
+    }
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    Datum lengthDatum = params.get(1);
+
+    if (datum instanceof NullDatum) return NullDatum.get();
+    if (lengthDatum instanceof NullDatum) return NullDatum.get();
+
+    Datum fillText = NullDatum.get();
+
+    if (hasFillCharacters) {
+      fillText = params.get(2);
+    } else {
+      fillText = DatumFactory.createText(" ");
+    }
+
+    int templen = lengthDatum.asInt4() - datum.asChars().length();
+
+    if (templen <= 0) {
+      return DatumFactory.createText(datum.asChars().substring(0,lengthDatum.asInt4()));
+    } else {
+      return DatumFactory.createText(StringUtils.leftPad(datum.asChars(), lengthDatum.asInt4(), fillText.asChars()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Md5.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Md5.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Md5.java
new file mode 100644
index 0000000..fb5f73c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Md5.java
@@ -0,0 +1,65 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import java.security.*;
+import org.apache.commons.codec.binary.Hex;
+
+/**
+ * Function definition
+ *
+ * text md5(string text)
+ */
+@Description(
+  functionName = "md5",
+  description = "Calculates the MD5 hash of string",
+  example = "> SELECT md5('abc');\n"
+          + "900150983cd24fb0 d6963f7d28e17f72",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class Md5 extends GeneralFunction {
+  public Md5() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    try {
+        MessageDigest md = MessageDigest.getInstance("MD5");
+        return DatumFactory.createText(new String(Hex.encodeHex(md.digest(datum.asByteArray()))));
+    } catch (NoSuchAlgorithmException e){
+        return NullDatum.get();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java
new file mode 100644
index 0000000..dc71907
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT4 octet_length(string text)
+ */
+@Description(
+  functionName = "octet_length",
+  description = "Number of bytes in string. ",
+  example = "> SELECT octet_length('jose');\n"
+        + "4",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class OctetLength  extends GeneralFunction {
+
+  public OctetLength() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt4(datum.asByteArray().length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/QuoteIdent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/QuoteIdent.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/QuoteIdent.java
new file mode 100644
index 0000000..d5237d7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/QuoteIdent.java
@@ -0,0 +1,67 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text quote_ident(string text)
+ *
+ * Description:
+ * Return a quoted string.
+ */
+@Description(
+  functionName = "quote_ident",
+  description = "Return the given string suitably quoted to be used as an identifier in an SQL statement string",
+  detail = "Quotes are added only if necessary "
+        + "(i.e., if the string contains non-identifier characters or would be case-folded).\n"
+        + "Embedded quotes are properly doubled.",
+  example = "> SELECT quote_ident('Foo bar');\n"
+      + "\"Foo bar\"",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class QuoteIdent extends GeneralFunction {
+  public QuoteIdent() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+
+    if(datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createText("\"" + datum.asChars() + "\"");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
new file mode 100644
index 0000000..9e3ff8d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
@@ -0,0 +1,76 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.eval.FunctionEval;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text rtrim(string text [, characters text])
+ */
+@Description(
+  functionName = "rtrim",
+  description = "Remove the longest string containing only "
+          + " characters from characters (a space by default) "
+          + " from the end of string.",
+  example = "> SELECT rtrim('trimxxxx', 'x');\n"
+          + "trim",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,TajoDataTypes.Type.TEXT})}
+)
+public class RTrim extends GeneralFunction {
+  @Expose private boolean hasTrimCharacters;
+
+  public RTrim() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+    });
+  }
+
+  public void init(FunctionEval.ParamType[] paramTypes) {
+    if (paramTypes.length == 2) {
+      hasTrimCharacters = true;
+    }
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    if (!hasTrimCharacters) {
+      return DatumFactory.createText(StringUtils.stripEnd(datum.asChars(), null));
+    } else {
+      return DatumFactory.createText(StringUtils.stripEnd(datum.asChars(), params.get(1).asChars()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
new file mode 100644
index 0000000..2b59d34
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
@@ -0,0 +1,115 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.tajo.engine.eval.FunctionEval.ParamType;
+
+/**
+ * This function is defined as:
+ * <pre>
+ * regexp_replace(string text, pattern text, replacement text [, flags text])
+ * </pre>
+ *
+ * flags is not supported yet.
+ */
+@Description(
+  functionName = "regexp_replace",
+  description = " Replace substring(s) matching a POSIX regular expression.",
+  example = "> SELECT regexp_replace('Thomas', '.[mN]a.', 'M');\n"
+          + "ThM",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,
+          TajoDataTypes.Type.TEXT,TajoDataTypes.Type.TEXT})}
+)
+public class RegexpReplace extends GeneralFunction {
+  @Expose protected boolean isPatternConstant;
+
+  // transient variables
+  protected String pattern;
+  private boolean isAlwaysNull = false;
+  private BooleanDatum result;
+  protected Pattern compiled;
+
+  public RegexpReplace() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("pattern", TajoDataTypes.Type.TEXT),
+        new Column("replacement", TajoDataTypes.Type.INT4),
+        new Column("flags", TajoDataTypes.Type.INT4), // it is not supported yet.
+    });
+  }
+
+  public void init(ParamType [] paramTypes) {
+    if (paramTypes[0] == ParamType.NULL || paramTypes[1] == ParamType.NULL || paramTypes[2] == ParamType.NULL) {
+      isAlwaysNull = true;
+    } else if (paramTypes[1] == ParamType.CONSTANT) {
+      isPatternConstant = true;
+    }
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum thisValue = params.get(0);
+    Datum thisPattern = params.get(1);
+    Datum thisReplacement = params.get(2);
+    boolean nullResult = isAlwaysNull
+        || thisValue instanceof NullDatum
+        || thisReplacement instanceof NullDatum
+        || thisPattern instanceof NullDatum;
+
+    Pattern thisCompiled;
+    if (!nullResult) {
+      if (compiled != null) {
+        thisCompiled = compiled;
+      } else {
+        thisCompiled = Pattern.compile(thisPattern.asChars());
+
+        // if a regular expression pattern is a constant,
+        // it will be reused in every call
+        if (isPatternConstant) {
+          compiled = thisCompiled;
+        }
+      }
+
+      Matcher matcher = thisCompiled.matcher(thisValue.asChars());
+      String replacement = thisReplacement.asChars();
+      StringBuffer sb = new StringBuffer();
+      while (matcher.find()) {
+        matcher.appendReplacement(sb, replacement);
+      }
+      matcher.appendTail(sb);
+
+      return DatumFactory.createText(sb.toString());
+    } else {
+      return NullDatum.get();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Repeat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Repeat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Repeat.java
new file mode 100644
index 0000000..4c52a6e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Repeat.java
@@ -0,0 +1,72 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text reverse(string text)
+ */
+@Description(
+  functionName = "repeat",
+  description = "Repeat string the specified number of times.",
+  example = "> SELECT repeat('Pg', 4);\n"
+          + "PgPgPgPg",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,
+          TajoDataTypes.Type.INT4})}
+)
+public class Repeat extends GeneralFunction {
+  public Repeat() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("count", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  private String repeat(String word, int count) {
+    StringBuilder builder = new StringBuilder();
+    for (int i = 0; i < count; i++) {
+        builder.append(word);
+    } 
+
+    return builder.toString();
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    Datum countDatum = params.get(1);
+    if(countDatum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(repeat(datum.asChars(), countDatum.asInt4()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Reverse.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Reverse.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Reverse.java
new file mode 100644
index 0000000..a77ba69
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Reverse.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text reverse(string text)
+ */
+@Description(
+  functionName = "reverse",
+  description = "Reverse str",
+  example = "> SELECT reverse('TAJO');\n"
+          + "OJAT",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class Reverse extends GeneralFunction {
+  public Reverse() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(new StringBuffer(datum.asChars()).reverse().toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Right.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Right.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Right.java
new file mode 100644
index 0000000..aa0dad0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Right.java
@@ -0,0 +1,85 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.datum.TextDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text right(string text, int size)
+ */
+@Description(
+  functionName = "right",
+  description = "Last n characters in the string",
+  example = "> SELECT right('ABC', 2);\n"
+          + "BC",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,
+          TajoDataTypes.Type.INT4})}
+)
+public class Right extends GeneralFunction {
+  public Right() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("size", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  public int getSize(int length, int size) {
+    if (size < 0) {
+        size = length + size;
+        if (size < 0) {
+            size = 0;
+        }
+    }
+
+    return (size < length) ? size : length;
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    Datum sizeDatum = params.get(1);
+    if(sizeDatum instanceof NullDatum) return NullDatum.get();
+
+    String data = datum.asChars();
+    int length = data.length();
+    int size = sizeDatum.asInt4();
+
+    size = getSize(length, size);
+    if (size == 0) {
+        return TextDatum.EMPTY_TEXT;
+    }
+
+    int startIdx = length - size;
+    return DatumFactory.createText(data.substring(startIdx, length));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
new file mode 100644
index 0000000..225777b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
@@ -0,0 +1,90 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.eval.FunctionEval;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+
+/**
+ * Function definition
+ *
+ * text rpad(string text, length int [, fill text])
+ */
+@Description(
+  functionName = "rpad",
+  description = "Fill up the string to length length by appending the characters fill (a space by default)",
+  detail = "If the string is already longer than length then it is truncated.",
+  example = "> SELECT rpad('hi', 5, 'xy');\n"
+      + "hixyx",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.INT4}),
+                @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.INT4, TajoDataTypes.Type.TEXT})}
+)
+public class Rpad extends GeneralFunction {
+  @Expose private boolean hasFillCharacters;
+
+  public Rpad() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("length", TajoDataTypes.Type.INT4),
+        new Column("fill_text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  public void init(FunctionEval.ParamType[] paramTypes) {
+    if (paramTypes.length == 3) {
+      hasFillCharacters = true;
+    }
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    Datum lengthDatum = params.get(1);
+
+    if(datum instanceof NullDatum) return NullDatum.get();
+    if(lengthDatum instanceof NullDatum) return NullDatum.get();
+
+    Datum fillText=NullDatum.get();
+    if(hasFillCharacters) {
+      fillText=params.get(2);
+    }
+    else {
+      fillText=DatumFactory.createText(" ");
+    }
+
+    int templen = lengthDatum.asInt4() - datum.asChars().length();
+
+    if(templen<=0)
+      return DatumFactory.createText(datum.asChars().substring(0,lengthDatum.asInt4()));
+
+    return DatumFactory.createText(StringUtils.rightPad(datum.asChars(), lengthDatum.asInt4(), fillText.asChars()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/SplitPart.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/SplitPart.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/SplitPart.java
new file mode 100644
index 0000000..fd1de55
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/SplitPart.java
@@ -0,0 +1,73 @@
+/**
+ * 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.function.string;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.function.GeneralFunction;
+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.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text split_part(string text, delimiter text, field int)
+ */
+@Description(
+  functionName = "split_part",
+  description = "Split string on delimiter and return the given field",
+  example = "> SELECT split_part('abc~@~def~@~ghi', '~@~', 2);\n"
+          + "def",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,
+          TajoDataTypes.Type.TEXT,TajoDataTypes.Type.INT4})}
+)
+public class SplitPart extends GeneralFunction {
+  public SplitPart() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("delimiter", TajoDataTypes.Type.TEXT),
+        new Column("field", TajoDataTypes.Type.INT4),
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum text = params.get(0);
+    Datum part = params.get(2);
+
+    if (text.isNull() || part.isNull()) {
+      return NullDatum.get();
+    }
+
+    String [] split = StringUtils.splitByWholeSeparatorPreserveAllTokens(text.asChars(), params.get(1).asChars(), -1);
+    int idx = params.get(2).asInt4() - 1;
+    if (split.length > idx) {
+      return DatumFactory.createText(split[idx]);
+    } else {
+      // If part is larger than the number of string portions, it will returns NULL.
+      return NullDatum.get();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPos.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPos.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPos.java
new file mode 100644
index 0000000..d6f88de
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPos.java
@@ -0,0 +1,73 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * int strpos(string text, substring text))
+ */
+@Description(
+  functionName = "strpos",
+  description = "Location of specified substring.",
+  example = "> SELECT strpos('tajo', 'aj');\n"
+          + "2",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,
+          TajoDataTypes.Type.TEXT})}
+)
+public class StrPos extends GeneralFunction {
+  public StrPos() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("substring", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    Datum substringDatum = params.get(1);
+    if(substringDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String value = valueDatum.asChars();
+    String substring = substringDatum.asChars();
+    if(substring.length() == 0) {
+      return DatumFactory.createInt4(1);
+    }
+
+    return DatumFactory.createInt4(value.indexOf(substring) + 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java
new file mode 100644
index 0000000..56fb3a1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java
@@ -0,0 +1,120 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Function definition
+ *
+ * int strposb(string text, substring text))
+ */
+@Description(
+  functionName = "strposb",
+  description = "Binary location of specified substring.",
+  example = "> SELECT strpos('tajo', 'aj');\n"
+      + "2",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,
+          TajoDataTypes.Type.TEXT})}
+)
+public class StrPosb extends GeneralFunction {
+  public StrPosb() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("substring", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    Datum substringDatum = params.get(1);
+    if(substringDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String value = valueDatum.asChars();
+    String substring = substringDatum.asChars();
+    if(substring.length() == 0) {
+      return DatumFactory.createInt4(1);
+    }
+
+    return DatumFactory.createInt4(findText(value, substring) + 1);
+  }
+
+  /**
+   * finds the location of specified substring.
+   * @param value
+   * @param substring
+   * @return
+   */
+  public static int findText(String value, String substring) {
+    //This method is copied from Hive's GenericUDFUtils.findText()
+    int start = 0;
+    byte[] valueBytes = value.getBytes();
+    byte[] substrBytes = substring.getBytes();
+
+    ByteBuffer src = ByteBuffer.wrap(valueBytes, 0, valueBytes.length);
+    ByteBuffer tgt = ByteBuffer.wrap(substrBytes, 0, substrBytes.length);
+    byte b = tgt.get();
+    src.position(start);
+
+    while (src.hasRemaining()) {
+      if (b == src.get()) { // matching first byte
+        src.mark(); // save position in loop
+        tgt.mark(); // save position in target
+        boolean found = true;
+        int pos = src.position() - 1;
+        while (tgt.hasRemaining()) {
+          if (!src.hasRemaining()) { // src expired first
+            tgt.reset();
+            src.reset();
+            found = false;
+            break;
+          }
+          if (!(tgt.get() == src.get())) {
+            tgt.reset();
+            src.reset();
+            found = false;
+            break; // no match
+          }
+        }
+        if (found) {
+          return pos;
+        }
+      }
+    }
+    return -1; // not found
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Substr.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Substr.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Substr.java
new file mode 100644
index 0000000..74492b1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Substr.java
@@ -0,0 +1,94 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text substr(string text, from int4 [, length int4])
+ */
+@Description(
+  functionName = "substr",
+  description = "Extract substring.",
+  example = "> SELECT substr('alphabet', 3, 2);\n"
+          + "ph",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {
+    @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,TajoDataTypes.Type.INT4}),
+    @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,TajoDataTypes.Type.INT4,
+                              TajoDataTypes.Type.INT4})
+  } 
+)
+public class Substr extends GeneralFunction {
+  public Substr() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("from", TajoDataTypes.Type.INT4),
+        new Column("length", TajoDataTypes.Type.INT4)    //optional
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    Datum fromDatum = params.get(1);
+    Datum countDatum = params.size() > 2 ? params.get(2) : null;
+
+    if(valueDatum instanceof NullDatum || fromDatum instanceof NullDatum || countDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String value = valueDatum.asChars();
+    int from = fromDatum.asInt4();
+    int strLength = value.length();
+    int count;
+
+    if (countDatum == null) {
+      count = strLength;
+    } else {
+      count = (countDatum.asInt4() + from) - 1;
+    }
+
+    if (count > strLength) {
+      count = strLength;
+    }
+
+    if (from < 1) {
+      from = 0;
+    } else {
+      from --;
+    }
+
+    if (from >= count) {
+      return DatumFactory.createText("");
+    }
+
+    return DatumFactory.createText(value.substring(from, count));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToBin.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToBin.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToBin.java
new file mode 100644
index 0000000..3048aa8
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToBin.java
@@ -0,0 +1,65 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+
+/**
+ * Function definition
+ * text ToBin(int n) - returns n in binary
+ *   Example:
+ *   SELECT Tobin(22) FROM src LIMIT 1;
+ *   -> result: '10110'
+ */
+@Description(
+  functionName = "to_bin",
+  description = "Returns n in binary.",
+  example = "> SELECT to_bin(22);\n"
+      + "10110",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})}
+)
+public class ToBin extends GeneralFunction {
+
+  public ToBin() {
+    super(new Column[] {
+        new Column("n", TajoDataTypes.Type.INT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return  DatumFactory.createText(Long.toBinaryString(datum.asInt8()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToHex.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToHex.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToHex.java
new file mode 100644
index 0000000..a8624ab
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToHex.java
@@ -0,0 +1,76 @@
+/**
+ * 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.function.string;
+
+import org.apache.commons.codec.binary.Hex;
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text to_hex(text)
+ * text to_hex(int)
+ */
+@Description(
+  functionName = "to_hex",
+  description = "Convert the argument to hexadecimal",
+  example = "SELECT to_hex(15);\n"
+          + "F",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8})}
+)
+public class ToHex extends GeneralFunction {
+
+  public ToHex() {
+    super(new Column[] {
+        new Column("n", TajoDataTypes.Type.INT8)
+    });
+  }
+
+  public String trimZero(String hexString) {
+    int len = hexString.length();
+    for (int i = 0; i < len; i++) {
+        if (hexString.charAt(i) != '0') {
+            return hexString.substring(i);
+        }
+    }
+
+    return hexString;
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String ret = new String(Hex.encodeHex(datum.asByteArray()));
+    return DatumFactory.createText(trimZero(ret));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Upper.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Upper.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Upper.java
new file mode 100644
index 0000000..47d38ba
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Upper.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text upper(string text)
+ */
+@Description(
+  functionName = "upper",
+  description = "Convert string to upper case.",
+  example = "> SELECT upper('tajo');\n"
+          + "TAJO",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class Upper extends GeneralFunction {
+  public Upper() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(datum.asChars().toUpperCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
new file mode 100644
index 0000000..4dfb314
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
@@ -0,0 +1,90 @@
+/**
+ * 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.json;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.function.Function;
+import org.apache.tajo.catalog.json.FunctionAdapter;
+import org.apache.tajo.catalog.json.TableMetaAdapter;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.json.*;
+import org.apache.tajo.util.TUtil;
+
+import java.lang.reflect.Type;
+import java.util.Map;
+
+public class CoreGsonHelper {
+  private static Gson gson;
+  private static Gson gsonPretty;
+
+  private CoreGsonHelper() {
+  }
+	
+	private static Map<Type, GsonSerDerAdapter> registerAdapters() {
+    Map<Type, GsonSerDerAdapter> adapters = TUtil.newHashMap();
+    adapters.put(Path.class, new PathSerializer());
+    adapters.put(Class.class, new ClassNameSerializer());
+    adapters.put(LogicalNode.class, new LogicalNodeAdapter());
+    adapters.put(EvalNode.class, new EvalNodeAdapter());
+    adapters.put(TableMeta.class, new TableMetaAdapter());
+    adapters.put(Function.class, new FunctionAdapter());
+    adapters.put(GeneralFunction.class, new FunctionAdapter());
+    adapters.put(AggFunction.class, new FunctionAdapter());
+    adapters.put(Datum.class, new DatumAdapter());
+    adapters.put(DataType.class, new DataTypeAdapter());
+
+    return adapters;
+	}
+
+	public static Gson getInstance() {
+	  if (gson == null ) {
+      GsonHelper helper = new GsonHelper(registerAdapters());
+      gson = helper.getGson();
+	  }
+	  return gson;
+	}
+
+  public static Gson getPrettyInstance() {
+    if (gsonPretty == null) {
+      GsonBuilder prettyBuilder = new GsonBuilder()
+          .setPrettyPrinting()
+          .excludeFieldsWithoutExposeAnnotation();
+      GsonHelper.registerAdapters(prettyBuilder, registerAdapters());
+      gsonPretty = prettyBuilder.create();
+    }
+
+    return gsonPretty;
+  }
+
+  public static String toJson(GsonObject object, Class<? extends GsonObject> clazz) {
+    return getInstance().toJson(object, clazz);
+  }
+
+  public static <T extends GsonObject> T fromJson(String json, Class<T> clazz) {
+    return getInstance().fromJson(json, clazz);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/json/EvalNodeAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/json/EvalNodeAdapter.java b/tajo-core/src/main/java/org/apache/tajo/engine/json/EvalNodeAdapter.java
new file mode 100644
index 0000000..56600a6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/json/EvalNodeAdapter.java
@@ -0,0 +1,50 @@
+/**
+ * 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.json;
+
+import com.google.gson.*;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.eval.EvalType;
+import org.apache.tajo.json.GsonSerDerAdapter;
+
+import java.lang.reflect.Type;
+
+public class EvalNodeAdapter implements GsonSerDerAdapter<EvalNode> {
+
+  @Override
+  public EvalNode deserialize(JsonElement json, Type type,
+                              JsonDeserializationContext ctx) throws JsonParseException {
+    JsonObject jsonObject = json.getAsJsonObject();
+    String nodeName = jsonObject.get("type").getAsString();
+    Class clazz = EvalType.valueOf(nodeName).getBaseClass();
+    return ctx.deserialize(jsonObject.get("body"), clazz);
+  }
+
+  @Override
+  public JsonElement serialize(EvalNode evalNode, Type type,
+                               JsonSerializationContext ctx) {
+    JsonObject json = new JsonObject();
+    json.addProperty("type", evalNode.getType().name());
+    json.add("body", ctx.serialize(evalNode));
+    return json;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/json/LogicalNodeAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/json/LogicalNodeAdapter.java b/tajo-core/src/main/java/org/apache/tajo/engine/json/LogicalNodeAdapter.java
new file mode 100644
index 0000000..e2becef
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/json/LogicalNodeAdapter.java
@@ -0,0 +1,50 @@
+/**
+ * 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.json;
+
+import com.google.gson.*;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.json.GsonSerDerAdapter;
+
+import java.lang.reflect.Type;
+
+public class LogicalNodeAdapter implements GsonSerDerAdapter<LogicalNode> {
+
+  @Override
+  public LogicalNode deserialize(JsonElement src, Type type,
+                                 JsonDeserializationContext ctx) throws JsonParseException {
+    JsonObject jsonObject = src.getAsJsonObject();
+    String nodeName = jsonObject.get("type").getAsString();
+    Class clazz = NodeType.valueOf(nodeName).getBaseClass();
+    return ctx.deserialize(jsonObject.get("body"), clazz);
+  }
+
+  @Override
+  public JsonElement serialize(LogicalNode src, Type typeOfSrc,
+                               JsonSerializationContext context) {
+    JsonObject json = new JsonObject();
+    json.addProperty("type", src.getType().name());
+    json.add("body", context.serialize(src));
+    return json;
+  }
+}


[19/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsReporter.java
new file mode 100644
index 0000000..a32a913
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsReporter.java
@@ -0,0 +1,232 @@
+/**
+ * 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.util.metrics.reporter;
+
+import com.codahale.metrics.*;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+
+public abstract class TajoMetricsReporter {
+  public abstract void report(SortedMap<String, Gauge> gauges,
+                              SortedMap<String, Counter> counters,
+                              SortedMap<String, Histogram> histograms,
+                              SortedMap<String, Meter> meters,
+                              SortedMap<String, Timer> timers);
+
+  public <T> Map<String, Map<String, T>> findMetricsItemGroup(SortedMap<String, T> metricsMap) {
+    Map<String, Map<String, T>> metricsGroup = new HashMap<String, Map<String, T>>();
+
+    String previousGroup = null;
+    Map<String, T> groupItems = new HashMap<String, T>();
+
+    for (Map.Entry<String, T> entry : metricsMap.entrySet()) {
+      String key = entry.getKey();
+      String[] keyTokens = key.split("\\.");
+
+      String groupName = null;
+      String itemName = null;
+
+      if (keyTokens.length > 2) {
+        groupName = keyTokens[0] + "." + keyTokens[1];
+        itemName = "";
+        String prefix = "";
+        for (int i = 2; i < keyTokens.length; i++) {
+          itemName += prefix + keyTokens[i];
+          prefix = ".";
+        }
+      } else {
+        groupName = "";
+        itemName = key;
+        if(!metricsGroup.containsKey(groupName)) {
+          metricsGroup.put(groupName, new HashMap<String, T>());
+        }
+        metricsGroup.get(groupName).put(itemName, entry.getValue());
+        continue;
+      }
+
+      if (previousGroup != null && !previousGroup.equals(groupName)) {
+        metricsGroup.put(previousGroup, groupItems);
+        groupItems = new HashMap<String, T>();
+      }
+      groupItems.put(itemName, entry.getValue());
+      previousGroup = groupName;
+    }
+
+    if(groupItems != null && !groupItems.isEmpty()) {
+      metricsGroup.put(previousGroup, groupItems);
+    }
+
+    return metricsGroup;
+  }
+
+  protected String meterGroupToString(String dateTime, String hostAndPort, double rateFactor,
+                                      String groupName, Map<String, Meter> meters) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(dateTime).append(" ");
+    if(hostAndPort != null && !hostAndPort.isEmpty()) {
+      sb.append(hostAndPort).append(" ");
+    }
+    sb.append("meter").append(" ");
+
+    if(!groupName.isEmpty()) {
+      sb.append(groupName).append(" ");
+    }
+    String prefix = "";
+    for(Map.Entry<String, Meter> eachMeter: meters.entrySet()) {
+      String key = eachMeter.getKey();
+      Meter meter = eachMeter.getValue();
+      sb.append(prefix);
+      sb.append(key).append(".count=").append(meter.getCount()).append("|");
+      sb.append(key).append(".mean=").append(String.format("%2.2f",
+          convertRate(meter.getMeanRate(), rateFactor))).append("|");
+      sb.append(key).append(".1minute=").append(String.format("%2.2f",
+          convertRate(meter.getOneMinuteRate(), rateFactor))).append("|");
+      sb.append(key).append(".5minute=").append(String.format("%2.2f",
+          convertRate(meter.getFiveMinuteRate(), rateFactor))).append("|");
+      sb.append(key).append(".15minute=").append(String.format("%2.2f",
+          convertRate(meter.getFifteenMinuteRate(), rateFactor)));
+      prefix = ",";
+    }
+
+    return sb.toString();
+  }
+
+  protected String counterGroupToString(String dateTime, String hostAndPort, double rateFactor,
+                                        String groupName, Map<String, Counter> counters) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(dateTime).append(" ");
+    if(hostAndPort != null && !hostAndPort.isEmpty()) {
+      sb.append(hostAndPort).append(" ");
+    }
+    sb.append("counter").append(" ");
+
+    if(!groupName.isEmpty()) {
+      sb.append(groupName).append(" ");
+    }
+    String prefix = "";
+    for(Map.Entry<String, Counter> eachCounter: counters.entrySet()) {
+      sb.append(prefix);
+      sb.append(eachCounter.getKey()).append("=").append(eachCounter.getValue().getCount());
+      prefix = ",";
+
+    }
+    return sb.toString();
+  }
+
+  protected String gaugeGroupToString(String dateTime, String hostAndPort, double rateFactor,
+                                      String groupName, Map<String, Gauge> gauges) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(dateTime).append(" ");
+    if(hostAndPort != null && !hostAndPort.isEmpty()) {
+      sb.append(hostAndPort).append(" ");
+    }
+    sb.append("guage").append(" ");
+
+    if(!groupName.isEmpty()) {
+      sb.append(groupName).append(" ");
+    }
+    String prefix = "";
+    for(Map.Entry<String, Gauge> eachGauge: gauges.entrySet()) {
+      sb.append(prefix).append(eachGauge.getKey()).append("=").append(eachGauge.getValue().getValue());
+      prefix = ",";
+    }
+    return sb.toString();
+  }
+
+  protected String histogramGroupToString(String dateTime, String hostAndPort, double rateFactor,
+                                          String groupName, Map<String, Histogram> histograms) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(dateTime).append(" ");
+    if(hostAndPort != null && !hostAndPort.isEmpty()) {
+      sb.append(hostAndPort).append(" ");
+    }
+    sb.append("histo").append(" ");
+
+    String prefix = "";
+    for(Map.Entry<String, Histogram> eachHistogram: histograms.entrySet()) {
+      String key = eachHistogram.getKey();
+      Histogram histogram = eachHistogram.getValue();
+      sb.append(prefix);
+      sb.append(key).append(".count=").append(histogram.getCount()).append("|");
+
+      Snapshot snapshot = histogram.getSnapshot();
+
+      sb.append(key).append(".min=").append(snapshot.getMin()).append("|");
+      sb.append(key).append(".max=").append(snapshot.getMax()).append("|");
+      sb.append(key).append(".mean=").append(String.format("%2.2f", snapshot.getMean())).append("|");
+      sb.append(key).append(".stddev=").append(String.format("%2.2f", snapshot.getStdDev())).append("|");
+      sb.append(key).append(".median=").append(String.format("%2.2f", snapshot.getMedian())).append("|");
+      sb.append(key).append(".75%=").append(String.format("%2.2f", snapshot.get75thPercentile())).append("|");
+      sb.append(key).append(".95%=").append(String.format("%2.2f", snapshot.get95thPercentile())).append("|");
+      sb.append(key).append(".98%=").append(String.format("%2.2f", snapshot.get98thPercentile())).append("|");
+      sb.append(key).append(".99%=").append(String.format("%2.2f", snapshot.get99thPercentile())).append("|");
+      sb.append(key).append(".999%=").append(String.format("%2.2f", snapshot.get999thPercentile()));
+      prefix = ",";
+    }
+    return sb.toString();
+  }
+
+  protected String timerGroupToString(String dateTime, String hostAndPort, double rateFactor,
+                                 String groupName, Map<String, Timer> timers) {
+    StringBuilder sb = new StringBuilder();
+
+    sb.append(dateTime).append(" ");
+    if(hostAndPort != null && !hostAndPort.isEmpty()) {
+      sb.append(hostAndPort).append(" ");
+    }
+    sb.append("timer").append(" ");
+
+    if(!groupName.isEmpty()) {
+      sb.append(groupName).append(" ");
+    }
+    String prefix = "";
+    for(Map.Entry<String, Timer> eachTimer: timers.entrySet()) {
+      String key = eachTimer.getKey();
+      Timer timer = eachTimer.getValue();
+      Snapshot snapshot = timer.getSnapshot();
+
+      sb.append(prefix);
+      sb.append(key).append(".count=").append(timer.getCount()).append("|");
+      sb.append(key).append(".meanrate=").append(String.format("%2.2f", convertRate(timer.getMeanRate(), rateFactor))).append("|");
+      sb.append(key).append(".1minuterate=").append(String.format("%2.2f", convertRate(timer.getOneMinuteRate(), rateFactor))).append("|");
+      sb.append(key).append(".5minuterate=").append(String.format("%2.2f", convertRate(timer.getFiveMinuteRate(), rateFactor))).append("|");
+      sb.append(key).append(".15minuterate=").append(String.format("%2.2f", convertRate(timer.getFifteenMinuteRate(),rateFactor))).append("|");
+      sb.append(key).append(".min=").append(String.format("%2.2f", convertRate(snapshot.getMin(), rateFactor))).append("|");
+      sb.append(key).append(".max=").append(String.format("%2.2f", convertRate(snapshot.getMax(),rateFactor))).append("|");
+      sb.append(key).append(".mean=").append(String.format("%2.2f", convertRate(snapshot.getMean(), rateFactor))).append("|");
+      sb.append(key).append(".stddev=").append(String.format("%2.2f", convertRate(snapshot.getStdDev(),rateFactor))).append("|");
+      sb.append(key).append(".median=").append(String.format("%2.2f", convertRate(snapshot.getMedian(), rateFactor))).append("|");
+      sb.append(key).append(".75%=").append(String.format("%2.2f", convertRate(snapshot.get75thPercentile(), rateFactor))).append("|");
+      sb.append(key).append(".95%=").append(String.format("%2.2f", convertRate(snapshot.get95thPercentile(), rateFactor))).append("|");
+      sb.append(key).append(".98%=").append(String.format("%2.2f", convertRate(snapshot.get98thPercentile(), rateFactor))).append("|");
+      sb.append(key).append(".99%=").append(String.format("%2.2f", convertRate(snapshot.get99thPercentile(), rateFactor))).append("|");
+      sb.append(key).append(".999%=").append(String.format("%2.2f", convertRate(snapshot.get999thPercentile(),rateFactor)));
+      prefix = ",";
+    }
+
+    return sb.toString();
+  }
+
+  protected double convertRate(double rate, double rateFactor) {
+    return rate * rateFactor;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsScheduledReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsScheduledReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsScheduledReporter.java
new file mode 100644
index 0000000..f11d520
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/TajoMetricsScheduledReporter.java
@@ -0,0 +1,206 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.util.metrics.reporter;
+
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.util.metrics.GroupNameMetricsFilter;
+import org.apache.tajo.util.metrics.MetricsFilterList;
+import org.apache.tajo.util.metrics.RegexpMetricsFilter;
+
+import java.io.Closeable;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public abstract class TajoMetricsScheduledReporter extends TajoMetricsReporter implements Closeable {
+  private static final Log LOG = LogFactory.getLog(TajoMetricsScheduledReporter.class);
+
+  public static final String PERIOD_KEY = "period";
+
+  protected MetricRegistry registry;
+  protected ScheduledExecutorService executor;
+  protected MetricFilter filter;
+  protected double durationFactor;
+  protected String durationUnit;
+  protected double rateFactor;
+  protected String rateUnit;
+  protected Map<String, String> metricsProperties;
+  protected String metricsName;
+  protected String metricsPropertyKey;
+  protected String hostAndPort;
+  protected long period;
+
+  protected abstract String getReporterName();
+  protected abstract void afterInit();
+
+  private static class NamedThreadFactory implements ThreadFactory {
+    private final ThreadGroup group;
+    private final AtomicInteger threadNumber = new AtomicInteger(1);
+    private final String namePrefix;
+
+    private NamedThreadFactory(String name) {
+      final SecurityManager s = System.getSecurityManager();
+      this.group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup();
+      this.namePrefix = "metrics-" + name + "-thread-";
+    }
+
+    @Override
+    public Thread newThread(Runnable r) {
+      final Thread t = new Thread(group, r, namePrefix + threadNumber.getAndIncrement(), 0);
+      t.setDaemon(true);
+      if (t.getPriority() != Thread.NORM_PRIORITY) {
+        t.setPriority(Thread.NORM_PRIORITY);
+      }
+      return t;
+    }
+  }
+
+  public long getPeriod() {
+    return period;
+  }
+
+  public void init(MetricRegistry registry,
+                   String metricsName,
+                   String hostAndPort,
+                   Map<String, String> metricsProperties) {
+    this.registry = registry;
+    this.metricsName = metricsName;
+    this.executor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory(metricsName));
+    this.rateFactor = TimeUnit.SECONDS.toSeconds(1);
+    this.rateUnit = calculateRateUnit(TimeUnit.MILLISECONDS);
+    this.durationFactor = 1.0 / TimeUnit.MILLISECONDS.toNanos(1);
+    this.durationUnit = TimeUnit.MILLISECONDS.toString().toLowerCase(Locale.US);
+    this.metricsProperties = metricsProperties;
+    this.metricsPropertyKey = metricsName + "." + getReporterName() + ".";
+    this.hostAndPort = hostAndPort;
+
+    MetricsFilterList filterList = new MetricsFilterList();
+    filterList.addMetricFilter(new GroupNameMetricsFilter(metricsName));
+
+    String regexpFilterKey = metricsPropertyKey + "regexp.";
+    Set<String> regexpExpressions = new HashSet<String>();
+
+    for(Map.Entry<String, String> entry: metricsProperties.entrySet()) {
+      String key = entry.getKey();
+      if(key.indexOf(regexpFilterKey) == 0) {
+        regexpExpressions.add(entry.getValue());
+      }
+    }
+
+    if(!regexpExpressions.isEmpty()) {
+      filterList.addMetricFilter(new RegexpMetricsFilter(regexpExpressions));
+    }
+    this.filter = filterList;
+
+    this.period = 60;
+    if(metricsProperties.get(metricsPropertyKey + PERIOD_KEY) != null) {
+      this.period = Integer.parseInt(metricsProperties.get(metricsPropertyKey + PERIOD_KEY));
+    }
+    afterInit();
+  }
+
+  public void start() {
+    start(period, TimeUnit.SECONDS);
+  }
+
+  /**
+   * Starts the reporter polling at the given period.
+   *
+   * @param period the amount of time between polls
+   * @param unit   the unit for {@code period}
+   */
+  public void start(long period, TimeUnit unit) {
+    this.period = period;
+    executor.scheduleAtFixedRate(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          report();
+        } catch (Exception e) {
+          if(LOG.isDebugEnabled()) {
+            LOG.warn("Metric report error:" + e.getMessage(), e);
+          } else {
+            LOG.warn("Metric report error:" + e.getMessage(), e);
+          }
+        }
+      }
+    }, period, period, unit);
+  }
+
+  /**
+   * Stops the reporter and shuts down its thread of execution.
+   */
+  public void stop() {
+    executor.shutdown();
+    try {
+      executor.awaitTermination(1, TimeUnit.SECONDS);
+    } catch (InterruptedException ignored) {
+      // do nothing
+    }
+  }
+
+  /**
+   * Stops the reporter and shuts down its thread of execution.
+   */
+  @Override
+  public void close() {
+    stop();
+  }
+
+  /**
+   * Report the current values of all metrics in the registry.
+   */
+  public void report() {
+    report(registry.getGauges(filter),
+        registry.getCounters(filter),
+        registry.getHistograms(filter),
+        registry.getMeters(filter),
+        registry.getTimers(filter));
+  }
+
+  protected String getRateUnit() {
+    return rateUnit;
+  }
+
+  protected String getDurationUnit() {
+    return durationUnit;
+  }
+
+  protected double convertDuration(double duration) {
+    return duration * durationFactor;
+  }
+
+  protected double convertRate(double rate) {
+    return rate * rateFactor;
+  }
+
+  private String calculateRateUnit(TimeUnit unit) {
+    final String s = unit.toString().toLowerCase(Locale.US);
+    return s.substring(0, s.length() - 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/webapp/HttpServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/webapp/HttpServer.java b/tajo-core/src/main/java/org/apache/tajo/webapp/HttpServer.java
new file mode 100644
index 0000000..60faef2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/webapp/HttpServer.java
@@ -0,0 +1,447 @@
+/**
+ * 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.webapp;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.Handler;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.SessionIdManager;
+import org.mortbay.jetty.handler.ContextHandlerCollection;
+import org.mortbay.jetty.nio.SelectChannelConnector;
+import org.mortbay.jetty.servlet.*;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.mortbay.thread.QueuedThreadPool;
+import org.mortbay.util.MultiException;
+
+import javax.servlet.http.HttpServlet;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.BindException;
+import java.net.URL;
+import java.util.*;
+
+/**
+ * This class is borrowed from Hadoop and is simplified to our objective.
+ */
+public class HttpServer {
+  private static final Log LOG = LogFactory.getLog(HttpServer.class);
+
+  protected final Server webServer;
+  protected final Connector listener;
+  protected final WebAppContext webAppContext;
+  protected final boolean findPort;
+  protected final Map<Context, Boolean> defaultContexts = 
+      new HashMap<Context, Boolean>();
+  protected final List<String> filterNames = new ArrayList<String>();
+  private static final int MAX_RETRIES = 10;
+  private final boolean listenerStartedExternally;
+  static final String STATE_DESCRIPTION_ALIVE = " - alive";
+  static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
+
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Connector connector, Configuration conf,
+      String[] pathSpecs) throws IOException {
+    this.webServer = new Server();
+    this.findPort = findPort;
+
+    if (connector == null) {
+      listenerStartedExternally = false;
+      listener = createBaseListener(conf);
+      listener.setHost(bindAddress);
+      listener.setPort(port);
+
+    } else {
+      listenerStartedExternally = true;
+      listener = connector;
+    }
+    webServer.addConnector(listener);
+
+    SessionIdManager sessionIdManager = new HashSessionIdManager(new Random(System.currentTimeMillis()));
+    webServer.setSessionIdManager(sessionIdManager);
+
+    int maxThreads = conf.getInt("tajo.http.maxthreads", -1);
+    // If HTTP_MAX_THREADS is not configured, QueueThreadPool() will use the
+    // default value (currently 250).
+    QueuedThreadPool threadPool = maxThreads == -1 ? new QueuedThreadPool()
+        : new QueuedThreadPool(maxThreads);
+    webServer.setThreadPool(threadPool);
+
+    final String appDir = getWebAppsPath(name);
+    ContextHandlerCollection contexts = new ContextHandlerCollection();
+
+    webAppContext = new WebAppContext();
+    webAppContext.setDisplayName(name);
+    webAppContext.setContextPath("/");
+    webAppContext.setResourceBase(appDir + "/" + name);
+    webAppContext.setDescriptor(appDir + "/" + name + "/WEB-INF/web.xml");
+
+    contexts.addHandler(webAppContext);
+    webServer.setHandler(contexts);
+
+    addDefaultApps(contexts, appDir, conf);
+  }
+
+  /**
+   * Create a required listener for the Jetty instance listening on the port
+   * provided. This wrapper and all subclasses must create at least one
+   * listener.
+   */
+  public Connector createBaseListener(Configuration conf) throws IOException {
+    return HttpServer.createDefaultChannelConnector();
+  }
+
+  static Connector createDefaultChannelConnector() {
+    SelectChannelConnector ret = new SelectChannelConnector();
+    ret.setLowResourceMaxIdleTime(10000);
+    ret.setAcceptQueueSize(128);
+    ret.setResolveNames(false);
+    ret.setUseDirectBuffers(false);
+    return ret;
+  }
+
+  /**
+   * Add default apps.
+   * 
+   * @param appDir
+   *          The application directory
+   * @throws IOException
+   */
+  protected void addDefaultApps(ContextHandlerCollection parent,
+      final String appDir, Configuration conf) throws IOException {
+    // set up the context for "/logs/" if "hadoop.log.dir" property is defined.
+    String logDir = System.getProperty("tajo.log.dir");
+    if (logDir != null) {
+      Context logContext = new Context(parent, "/logs");
+      logContext.setResourceBase(logDir);
+      //logContext.addServlet(AdminAuthorizedServlet.class, "/*");
+      logContext.setDisplayName("logs");
+      defaultContexts.put(logContext, true);
+    }
+    // set up the context for "/static/*"
+    Context staticContext = new Context(parent, "/static");
+    staticContext.setResourceBase(appDir + "/static");
+    staticContext.addServlet(DefaultServlet.class, "/*");
+    staticContext.setDisplayName("static");
+    defaultContexts.put(staticContext, true);
+  }
+  
+  public void addContext(Context ctxt, boolean isFiltered)
+      throws IOException {
+    webServer.addHandler(ctxt);
+    defaultContexts.put(ctxt, isFiltered);
+  }
+  
+  /**
+   * Add a context 
+   * @param pathSpec The path spec for the context
+   * @param dir The directory containing the context
+   * @param isFiltered if true, the servlet is added to the filter path mapping 
+   * @throws IOException
+   */
+  protected void addContext(String pathSpec, String dir, boolean isFiltered) throws IOException {
+    if (0 == webServer.getHandlers().length) {
+      throw new RuntimeException("Couldn't find handler");
+    }
+    WebAppContext webAppCtx = new WebAppContext();
+    webAppCtx.setContextPath(pathSpec);
+    webAppCtx.setWar(dir);
+    addContext(webAppCtx, true);
+  }
+  
+  /**
+   * Set a value in the webapp context. These values are available to the jsp
+   * pages as "application.getAttribute(name)".
+   * @param name The name of the attribute
+   * @param value The value of the attribute
+   */
+  public void setAttribute(String name, Object value) {
+    webAppContext.setAttribute(name, value);
+  }
+  
+  /**
+   * Add a servlet in the server.
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param clazz The servlet class
+   */
+  public void addServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
+    addInternalServlet(name, pathSpec, clazz, false);
+    addFilterPathMapping(pathSpec, webAppContext);
+  }
+  
+  /**
+   * Add an internal servlet in the server, specifying whether or not to
+   * protect with Kerberos authentication. 
+   * Note: This method is to be used for adding servlets that facilitate
+   * internal communication and not for user facing functionality. For
+   * servlets added using this method, filters (except internal Kerberized
+   * filters) are not enabled. 
+   * 
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param clazz The servlet class
+   */
+  public void addInternalServlet(String name, String pathSpec, 
+      Class<? extends HttpServlet> clazz, boolean requireAuth) {
+    ServletHolder holder = new ServletHolder(clazz);
+    if (name != null) {
+      holder.setName(name);
+    }
+    webAppContext.addServlet(holder, pathSpec);
+    
+    if(requireAuth && UserGroupInformation.isSecurityEnabled()) {
+       LOG.info("Adding Kerberos filter to " + name);
+       ServletHandler handler = webAppContext.getServletHandler();
+       FilterMapping fmap = new FilterMapping();
+       fmap.setPathSpec(pathSpec);
+       fmap.setFilterName("krb5Filter");
+       fmap.setDispatches(Handler.ALL);
+       handler.addFilterMapping(fmap);
+    }
+  }
+  
+  /**
+   * Add the path spec to the filter path mapping.
+   * @param pathSpec The path spec
+   * @param webAppCtx The WebApplicationContext to add to
+   */
+  protected void addFilterPathMapping(String pathSpec,
+      Context webAppCtx) {
+    ServletHandler handler = webAppCtx.getServletHandler();
+    for(String name : filterNames) {
+      FilterMapping fmap = new FilterMapping();
+      fmap.setPathSpec(pathSpec);
+      fmap.setFilterName(name);
+      fmap.setDispatches(Handler.ALL);
+      handler.addFilterMapping(fmap);
+    }
+  }
+  
+  protected String getWebAppsPath(String name) throws FileNotFoundException {
+    URL url = getClass().getClassLoader().getResource("webapps/" + name);
+    if (url == null) {
+      throw new FileNotFoundException("webapps/" + name + " not found in CLASSPATH");
+    }
+    String urlString = url.toString();
+    return urlString.substring(0, urlString.lastIndexOf('/'));
+  }
+  
+  /**
+   * Get the value in the webapp context.
+   * @param name The name of the attribute
+   * @return The value of the attribute
+   */
+  public Object getAttribute(String name) {
+    return webAppContext.getAttribute(name);
+  }  
+  
+  /**
+   * Get the port that the server is on
+   * @return the port
+   */
+  public int getPort() {
+    return webServer.getConnectors()[0].getLocalPort();
+  }
+
+  /**
+   * Set the min, max number of worker threads (simultaneous connections).
+   */
+  public void setThreads(int min, int max) {
+    QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool() ;
+    pool.setMinThreads(min);
+    pool.setMaxThreads(max);
+  }
+
+  /**
+   * Start the server. Does not wait for the server to start.
+   */
+  public void start() throws IOException {
+    try {
+      if (listenerStartedExternally) { // Expect that listener was started
+                                       // securely
+        if (listener.getLocalPort() == -1) // ... and verify
+          throw new Exception("Exepected webserver's listener to be started "
+              + "previously but wasn't");
+        // And skip all the port rolling issues.
+        webServer.start();
+      } else {
+        int port;
+        int oriPort = listener.getPort(); // The original requested port
+        while (true) {
+          try {
+            port = webServer.getConnectors()[0].getLocalPort();
+            LOG.debug("Port returned by webServer.getConnectors()[0]."
+                + "getLocalPort() before open() is " + port
+                + ". Opening the listener on " + oriPort);
+            listener.open();
+            port = listener.getLocalPort();
+            LOG.debug("listener.getLocalPort() returned "
+                + listener.getLocalPort()
+                + " webServer.getConnectors()[0].getLocalPort() returned "
+                + webServer.getConnectors()[0].getLocalPort());
+            // Workaround to handle the problem reported in HADOOP-4744
+            if (port < 0) {
+              Thread.sleep(100);
+              int numRetries = 1;
+              while (port < 0) {
+                LOG.warn("listener.getLocalPort returned " + port);
+                if (numRetries++ > MAX_RETRIES) {
+                  throw new Exception(" listener.getLocalPort is returning "
+                      + "less than 0 even after " + numRetries + " resets");
+                }
+                for (int i = 0; i < 2; i++) {
+                  LOG.info("Retrying listener.getLocalPort()");
+                  port = listener.getLocalPort();
+                  if (port > 0) {
+                    break;
+                  }
+                  Thread.sleep(200);
+                }
+                if (port > 0) {
+                  break;
+                }
+                LOG.info("Bouncing the listener");
+                listener.close();
+                Thread.sleep(1000);
+                listener.setPort(oriPort == 0 ? 0 : (oriPort += 1));
+                listener.open();
+                Thread.sleep(100);
+                port = listener.getLocalPort();
+              }
+            } // Workaround end
+            LOG.info("Jetty bound to port " + port);
+            webServer.start();
+            break;
+          } catch (IOException ex) {
+            // if this is a bind exception,
+            // then try the next port number.
+            if (ex instanceof BindException) {
+              if (!findPort) {
+                BindException be = new BindException("Port in use: "
+                    + listener.getHost() + ":" + listener.getPort());
+                be.initCause(ex);
+                throw be;
+              }
+            } else {
+              LOG.info("HttpServer.start() threw a non Bind IOException");
+              throw ex;
+            }
+          } catch (MultiException ex) {
+            LOG.info("HttpServer.start() threw a MultiException");
+            throw ex;
+          }
+          listener.setPort((oriPort += 1));
+        }
+      }
+      // Make sure there is no handler failures.
+      Handler[] handlers = webServer.getHandlers();
+      for (int i = 0; i < handlers.length; i++) {
+        if (handlers[i].isFailed()) {
+          throw new IOException(
+              "Problem in starting http server. Server handlers failed");
+        }
+      }
+    } catch (IOException e) {
+      throw e;
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException(
+          "Interrupted while starting HTTP server").initCause(e);
+    } catch (Exception e) {
+      throw new IOException("Problem starting http server", e);
+    }
+  }
+
+  /**
+   * stop the server
+   */
+  public void stop() throws Exception {
+    MultiException exception = null;
+    try {
+      listener.close();
+    } catch (Exception e) {
+      LOG.error(
+          "Error while stopping listener for webapp"
+              + webAppContext.getDisplayName(), e);
+      exception = addMultiException(exception, e);
+    }
+
+    try {
+      // clear & stop webAppContext attributes to avoid memory leaks.
+      webAppContext.clearAttributes();
+      webAppContext.stop();
+    } catch (Exception e) {
+      LOG.error("Error while stopping web app context for webapp "
+          + webAppContext.getDisplayName(), e);
+      exception = addMultiException(exception, e);
+    }
+    try {
+      webServer.stop();
+    } catch (Exception e) {
+      LOG.error(
+          "Error while stopping web server for webapp "
+              + webAppContext.getDisplayName(), e);
+      exception = addMultiException(exception, e);
+    }
+
+    if (exception != null) {
+      exception.ifExceptionThrow();
+    }
+
+  }
+
+  private MultiException addMultiException(MultiException exception, Exception e) {
+    if (exception == null) {
+      exception = new MultiException();
+    }
+    exception.add(e);
+    return exception;
+  }
+
+  public void join() throws InterruptedException {
+    webServer.join();
+  }
+
+  /**
+   * Test for the availability of the web server
+   * 
+   * @return true if the web server is started, false otherwise
+   */
+  public boolean isAlive() {
+    return webServer != null && webServer.isStarted();
+  }
+
+  /**
+   * Return the host and port of the HttpServer, if live
+   * 
+   * @return the classname and any HTTP URL
+   */
+  @Override
+  public String toString() {
+    return listener != null ? ("HttpServer at http://" + listener.getHost()
+        + ":" + listener.getLocalPort() + "/" + (isAlive() ? STATE_DESCRIPTION_ALIVE
+        : STATE_DESCRIPTION_NOT_LIVE))
+        : "Inactive HttpServer";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
new file mode 100644
index 0000000..faeadaf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
@@ -0,0 +1,376 @@
+package org.apache.tajo.webapp;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.client.QueryStatus;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.jdbc.TajoResultSet;
+import org.apache.tajo.util.JSPUtil;
+import org.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import javax.servlet.ServletConfig;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * 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.
+ */
+
+public class QueryExecutorServlet extends HttpServlet {
+  private static final Log LOG = LogFactory.getLog(QueryExecutorServlet.class);
+
+  ObjectMapper om = new ObjectMapper();
+
+  //queryRunnerId -> QueryRunner
+  private final Map<String, QueryRunner> queryRunners = new HashMap<String, QueryRunner>();
+
+  private TajoClient tajoClient;
+
+  private ExecutorService queryRunnerExecutor = Executors.newFixedThreadPool(5);
+
+  private QueryRunnerCleaner queryRunnerCleaner;
+  @Override
+  public void init(ServletConfig config) throws ServletException {
+    om.getDeserializationConfig().disable(
+        DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES);
+
+    try {
+      tajoClient = new TajoClient(new TajoConf());
+
+      queryRunnerCleaner = new QueryRunnerCleaner();
+      queryRunnerCleaner.start();
+    } catch (IOException e) {
+      LOG.error(e.getMessage());
+    }
+  }
+
+  @Override
+  public void service(HttpServletRequest request,
+                      HttpServletResponse response) throws ServletException, IOException {
+    String action = request.getParameter("action");
+    Map<String, Object> returnValue = new HashMap<String, Object>();
+    try {
+      if(tajoClient == null) {
+        errorResponse(response, "TajoClient not initialized");
+        return;
+      }
+      if(action == null || action.trim().isEmpty()) {
+        errorResponse(response, "no action parameter.");
+        return;
+      }
+
+      if("runQuery".equals(action)) {
+        String query = request.getParameter("query");
+        if(query == null || query.trim().isEmpty()) {
+          errorResponse(response, "No query parameter");
+          return;
+        }
+        String queryRunnerId = null;
+        while(true) {
+          synchronized(queryRunners) {
+            queryRunnerId = "" + System.currentTimeMillis();
+            if(!queryRunners.containsKey(queryRunnerId)) {
+              break;
+            }
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException e) {
+            }
+          }
+        }
+        QueryRunner queryRunner = new QueryRunner(queryRunnerId, query);
+        try {
+          queryRunner.sizeLimit = Integer.parseInt(request.getParameter("limitSize"));
+        } catch (java.lang.NumberFormatException nfe) {
+          queryRunner.sizeLimit = 1048576;
+        }
+        synchronized(queryRunners) {
+          queryRunners.put(queryRunnerId, queryRunner);
+        }
+        queryRunnerExecutor.submit(queryRunner);
+        returnValue.put("queryRunnerId", queryRunnerId);
+      } else if("getQueryProgress".equals(action)) {
+        synchronized(queryRunners) {
+          String queryRunnerId = request.getParameter("queryRunnerId");
+          QueryRunner queryRunner = queryRunners.get(queryRunnerId);
+          if(queryRunner == null) {
+            errorResponse(response, "No query info:" + queryRunnerId);
+            return;
+          }
+          if(queryRunner.error != null) {
+            errorResponse(response, queryRunner.error);
+            return;
+          }
+          SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+          returnValue.put("progress", queryRunner.progress);
+          returnValue.put("startTime", df.format(queryRunner.startTime));
+          returnValue.put("finishTime", queryRunner.finishTime == 0 ? "-" : df.format(queryRunner.startTime));
+          returnValue.put("runningTime", JSPUtil.getElapsedTime(queryRunner.startTime, queryRunner.finishTime));
+        }
+      } else if("getQueryResult".equals(action)) {
+        synchronized(queryRunners) {
+          String queryRunnerId = request.getParameter("queryRunnerId");
+          QueryRunner queryRunner = queryRunners.get(queryRunnerId);
+          if(queryRunner == null) {
+            errorResponse(response, "No query info:" + queryRunnerId);
+            return;
+          }
+          if(queryRunner.error != null) {
+            errorResponse(response, queryRunner.error);
+            return;
+          }
+          returnValue.put("numOfRows", queryRunner.numOfRows);
+          returnValue.put("resultSize", queryRunner.resultSize);
+          returnValue.put("resultData", queryRunner.queryResult);
+          returnValue.put("resultColumns", queryRunner.columnNames);
+          returnValue.put("runningTime", JSPUtil.getElapsedTime(queryRunner.startTime, queryRunner.finishTime));
+        }
+      } else if("clearAllQueryRunner".equals(action)) {
+        synchronized(queryRunners) {
+          for(QueryRunner eachQueryRunner: queryRunners.values()) {
+            eachQueryRunner.setStop();
+          }
+          queryRunners.clear();
+        }
+      }
+      returnValue.put("success", "true");
+      writeHttpResponse(response, returnValue);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      errorResponse(response, e);
+    }
+  }
+
+  private void errorResponse(HttpServletResponse response, Exception e) throws IOException {
+    errorResponse(response, e.getMessage() + "\n" + StringUtils.stringifyException(e));
+  }
+
+  private void errorResponse(HttpServletResponse response, String message) throws IOException {
+    Map<String, Object> errorMessage = new HashMap<String, Object>();
+    errorMessage.put("success", "false");
+    errorMessage.put("errorMessage", message);
+    writeHttpResponse(response, errorMessage);
+  }
+
+  private void writeHttpResponse(HttpServletResponse response, Map<String, Object> outputMessage) throws IOException {
+    response.setContentType("text/html");
+
+    OutputStream out = response.getOutputStream();
+    out.write(om.writeValueAsBytes(outputMessage));
+
+    out.flush();
+    out.close();
+  }
+
+  class QueryRunnerCleaner extends Thread {
+    public void run() {
+      List<QueryRunner> queryRunnerList;
+      synchronized(queryRunners) {
+        queryRunnerList = new ArrayList<QueryRunner>(queryRunners.values());
+        for(QueryRunner eachQueryRunner: queryRunnerList) {
+          if(!eachQueryRunner.running.get() &&
+              (System.currentTimeMillis() - eachQueryRunner.finishTime > 180 * 1000)) {
+            queryRunners.remove(eachQueryRunner.queryRunnerId);
+          }
+        }
+      }
+    }
+  }
+
+  class QueryRunner extends Thread {
+    long startTime;
+    long finishTime;
+
+    String queryRunnerId;
+
+    ClientProtos.SubmitQueryResponse queryRespons;
+    AtomicBoolean running = new AtomicBoolean(true);
+    AtomicBoolean stop = new AtomicBoolean(false);
+    QueryId queryId;
+    String query;
+    long resultSize;
+    int sizeLimit;
+    long numOfRows;
+    Exception error;
+
+    AtomicInteger progress = new AtomicInteger(0);
+
+    List<String> columnNames = new ArrayList<String>();
+
+    List<List<Object>> queryResult;
+
+    public QueryRunner(String queryRunnerId, String query) {
+      this.queryRunnerId = queryRunnerId;
+      this.query = query;
+    }
+
+    public void setStop() {
+      this.stop.set(true);
+      this.interrupt();
+    }
+
+    public void run() {
+      startTime = System.currentTimeMillis();
+      try {
+        queryRespons = tajoClient.executeQuery(query);
+        if (queryRespons.getResultCode() == ClientProtos.ResultCode.OK) {
+          QueryId queryId = null;
+          try {
+            queryId = new QueryId(queryRespons.getQueryId());
+            getQueryResult(queryId);
+          } finally {
+            if (queryId != null) {
+              tajoClient.closeQuery(queryId);
+            }
+          }
+        } else {
+          LOG.error("queryRespons.getResultCode() not OK:" + queryRespons.getResultCode());
+          error = new Exception("queryRespons.getResultCode() not OK:" + queryRespons.getResultCode());
+        }
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        error = e;
+      } finally {
+        running.set(false);
+        finishTime = System.currentTimeMillis();
+      }
+    }
+
+    private void getQueryResult(QueryId tajoQueryId) {
+      // query execute
+      try {
+        QueryStatus status = null;
+
+        while (!stop.get()) {
+          try {
+            Thread.sleep(1000);
+          } catch(InterruptedException e) {
+            break;
+          }
+          status = tajoClient.getQueryStatus(tajoQueryId);
+          if (status.getState() == TajoProtos.QueryState.QUERY_MASTER_INIT
+              || status.getState() == TajoProtos.QueryState.QUERY_MASTER_LAUNCHED) {
+            continue;
+          }
+
+          if (status.getState() == TajoProtos.QueryState.QUERY_RUNNING
+              || status.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+            int progressValue = (int) (status.getProgress() * 100.0f);
+            if(progressValue == 100)  {
+              progressValue = 99;
+            }
+            progress.set(progressValue);
+          }
+          if (status.getState() != TajoProtos.QueryState.QUERY_RUNNING
+              && status.getState() != TajoProtos.QueryState.QUERY_NOT_ASSIGNED) {
+            break;
+          }
+        }
+
+        if(status == null) {
+          LOG.error("Query Status is null");
+          error = new Exception("Query Status is null");
+          return;
+        }
+        if (status.getState() == TajoProtos.QueryState.QUERY_ERROR ||
+            status.getState() == TajoProtos.QueryState.QUERY_FAILED) {
+          error = new Exception(status.getErrorMessage());
+        } else if (status.getState() == TajoProtos.QueryState.QUERY_KILLED) {
+          LOG.info(queryId + " is killed.");
+          error = new Exception(queryId + " is killed.");
+        } else {
+          if (status.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+            if (status.hasResult()) {
+              ResultSet res = null;
+              try {
+                ClientProtos.GetQueryResultResponse response = tajoClient.getResultResponse(tajoQueryId);
+                TableDesc desc = CatalogUtil.newTableDesc(response.getTableDesc());
+                tajoClient.getConf().setVar(TajoConf.ConfVars.USERNAME, response.getTajoUserName());
+                res = new TajoResultSet(tajoClient, queryId, tajoClient.getConf(), desc);
+
+                ResultSetMetaData rsmd = res.getMetaData();
+                resultSize = desc.getStats().getNumBytes();
+                LOG.info("Tajo Query Result: " + desc.getPath() + "\n");
+
+                int numOfColumns = rsmd.getColumnCount();
+                for(int i = 0; i < numOfColumns; i++) {
+                  columnNames.add(rsmd.getColumnName(i + 1));
+                }
+                queryResult = new ArrayList<List<Object>>();
+
+                if(sizeLimit < resultSize) {
+                    numOfRows = (long)((float)(desc.getStats().getNumRows()) * ((float)sizeLimit / (float)resultSize));
+                } else {
+                    numOfRows = desc.getStats().getNumRows();
+                }
+                int rowCount = 0;
+                boolean hasMoreData = false;
+                while (res.next()) {
+                  if(rowCount > numOfRows) {
+                    hasMoreData = true;
+                    break;
+                  }
+                  List<Object> row = new ArrayList<Object>();
+                  for(int i = 0; i < numOfColumns; i++) {
+                    row.add(res.getObject(i + 1).toString());
+                  }
+                  queryResult.add(row);
+                  rowCount++;
+
+                }
+              } finally {
+                if (res != null) {
+                  res.close();
+                }
+                progress.set(100);
+              }
+            } else {
+              error = new Exception(queryId + " no result");
+            }
+          }
+        }
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        error = e;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java b/tajo-core/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
new file mode 100644
index 0000000..09426e0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
@@ -0,0 +1,69 @@
+/**
+ * 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.webapp;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.worker.TajoWorker;
+import org.mortbay.jetty.Connector;
+
+import java.io.IOException;
+import java.net.Inet4Address;
+
+public class StaticHttpServer extends HttpServer {
+  private static StaticHttpServer instance = null;
+
+  private StaticHttpServer(Object containerObject , String name, String bindAddress, int port,
+      boolean findPort, Connector connector, Configuration conf,
+      String[] pathSpecs) throws IOException {
+    super( name, bindAddress, port, findPort, connector, conf, pathSpecs);
+  }
+  public static StaticHttpServer getInstance() {
+    return instance;
+  }
+  public static StaticHttpServer getInstance(Object containerObject, String name,
+      String bindAddress, int port, boolean findPort, Connector connector,
+      TajoConf conf,
+      String[] pathSpecs) throws IOException {
+    String addr = bindAddress;
+    if(instance == null) {
+      if(bindAddress == null || bindAddress.compareTo("") == 0) {
+        if (containerObject instanceof TajoMaster) {
+          addr = conf.getVar(ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS).split(":")[0];
+        } else if (containerObject instanceof TajoWorker) {
+          addr = Inet4Address.getLocalHost().getHostName();
+        }
+      }
+      
+      instance = new StaticHttpServer(containerObject, name, addr, port,
+          findPort, connector, conf, pathSpecs);
+      instance.setAttribute("tajo.info.server.object", containerObject);
+      instance.setAttribute("tajo.info.server.addr", addr);
+      instance.setAttribute("tajo.info.server.conf", conf);
+      instance.setAttribute("tajo.info.server.starttime", System.currentTimeMillis());
+    }
+    return instance;
+  }
+
+  public void set(String name, Object object) {
+    instance.setAttribute(name, object);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
new file mode 100644
index 0000000..55aa8c4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
@@ -0,0 +1,54 @@
+/**
+ * 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.worker;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tajo.master.ContainerProxy;
+
+import java.util.Map;
+
+public abstract class AbstractResourceAllocator extends CompositeService implements ResourceAllocator {
+  private Map<ContainerId, ContainerProxy> containers = Maps.newConcurrentMap();
+
+  public AbstractResourceAllocator() {
+    super(AbstractResourceAllocator.class.getName());
+  }
+
+  public void addContainer(ContainerId cId, ContainerProxy container) {
+    containers.put(cId, container);
+  }
+
+  public void removeContainer(ContainerId cId) {
+    containers.remove(cId);
+  }
+
+  public boolean containsContainer(ContainerId cId) {
+    return containers.containsKey(cId);
+  }
+
+  public ContainerProxy getContainer(ContainerId cId) {
+    return containers.get(cId);
+  }
+
+  public Map<ContainerId, ContainerProxy> getContainers() {
+    return containers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/DeletionService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/DeletionService.java b/tajo-core/src/main/java/org/apache/tajo/worker/DeletionService.java
new file mode 100644
index 0000000..42ea71f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/DeletionService.java
@@ -0,0 +1,115 @@
+/**
+ * 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.worker;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+
+import java.io.IOException;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+public class DeletionService {
+  static final Log LOG = LogFactory.getLog(DeletionService.class);
+
+  private int debugDelay;
+  private ScheduledThreadPoolExecutor sched;
+  private final FileContext lfs = getLfs();
+
+  static final FileContext getLfs() {
+    try {
+      return FileContext.getLocalFSFileContext();
+    } catch (UnsupportedFileSystemException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public DeletionService(int defaultThreads, int debugDelay) {
+    ThreadFactory tf = new ThreadFactoryBuilder().setNameFormat("DeletionService #%d").build();
+
+    sched = new ScheduledThreadPoolExecutor(defaultThreads, tf);
+    sched.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
+    sched.setKeepAliveTime(60L, TimeUnit.SECONDS);
+    this.debugDelay = debugDelay;
+  }
+
+
+  /**
+   * /**
+   * Delete the path(s) as this user.
+   *
+   * @param subDir   the sub directory name
+   * @param baseDirs the base directories which contains the subDir's
+   */
+  public void delete(Path subDir, Path... baseDirs) {
+    if (debugDelay != -1) {
+      sched.schedule(new FileDeletion(subDir, baseDirs), debugDelay, TimeUnit.SECONDS);
+    }
+  }
+
+  public void stop() {
+    sched.shutdown();
+    boolean terminated = false;
+    try {
+      terminated = sched.awaitTermination(10, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+    }
+    if (!terminated) {
+      sched.shutdownNow();
+    }
+  }
+
+  private class FileDeletion implements Runnable {
+    final Path subDir;
+    final Path[] baseDirs;
+
+    FileDeletion(Path subDir, Path[] baseDirs) {
+      this.subDir = subDir;
+      this.baseDirs = baseDirs;
+    }
+
+    @Override
+    public void run() {
+
+      if (baseDirs == null || baseDirs.length == 0) {
+        LOG.debug("Worker deleting absolute path : " + subDir);
+        try {
+          lfs.delete(subDir, true);
+        } catch (IOException e) {
+          LOG.warn("Failed to delete " + subDir);
+        }
+        return;
+      }
+      for (Path baseDir : baseDirs) {
+        Path del = subDir == null ? baseDir : new Path(baseDir, subDir);
+        LOG.debug("Worker deleting path : " + del);
+        try {
+          lfs.delete(del, true);
+        } catch (IOException e) {
+          LOG.warn("Failed to delete " + subDir);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java
new file mode 100644
index 0000000..bb136f7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java
@@ -0,0 +1,256 @@
+/**
+ * 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.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.IOUtils;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.handler.codec.http.*;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.channels.FileChannel;
+
+import static org.jboss.netty.channel.Channels.pipeline;
+
+/**
+ * Fetcher fetches data from a given uri via HTTP protocol and stores them into
+ * a specific file. It aims at asynchronous and efficient data transmit.
+ */
+public class Fetcher {
+  private final static Log LOG = LogFactory.getLog(Fetcher.class);
+
+  private final URI uri;
+  private final File file;
+
+  private final String host;
+  private int port;
+
+  private long startTime;
+  private long finishTime;
+  private long fileLen;
+  private int messageReceiveCount;
+
+  private ClientBootstrap bootstrap;
+
+  public Fetcher(URI uri, File file, ClientSocketChannelFactory factory) {
+    this.uri = uri;
+    this.file = file;
+
+    String scheme = uri.getScheme() == null ? "http" : uri.getScheme();
+    this.host = uri.getHost() == null ? "localhost" : uri.getHost();
+    this.port = uri.getPort();
+    if (port == -1) {
+      if (scheme.equalsIgnoreCase("http")) {
+        this.port = 80;
+      } else if (scheme.equalsIgnoreCase("https")) {
+        this.port = 443;
+      }
+    }
+
+    bootstrap = new ClientBootstrap(factory);
+    bootstrap.setOption("connectTimeoutMillis", 5000L); // set 5 sec
+    bootstrap.setOption("receiveBufferSize", 1048576); // set 1M
+    bootstrap.setOption("tcpNoDelay", true);
+
+    ChannelPipelineFactory pipelineFactory = new HttpClientPipelineFactory(file);
+    bootstrap.setPipelineFactory(pipelineFactory);
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  public long getFileLen() {
+    return fileLen;
+  }
+
+  public int getMessageReceiveCount() {
+    return messageReceiveCount;
+  }
+
+  public String getStatus() {
+    if(startTime == 0) {
+      return "READY";
+    }
+
+    if(startTime > 0 && finishTime == 0) {
+      return "FETCHING";
+    } else {
+      return "FINISH";
+    }
+  }
+
+  public File get() throws IOException {
+    startTime = System.currentTimeMillis();
+
+    ChannelFuture future = bootstrap.connect(new InetSocketAddress(host, port));
+
+    // Wait until the connection attempt succeeds or fails.
+    Channel channel = future.awaitUninterruptibly().getChannel();
+    if (!future.isSuccess()) {
+      future.getChannel().close();
+      throw new IOException(future.getCause());
+    }
+
+    String query = uri.getPath()
+        + (uri.getRawQuery() != null ? "?" + uri.getRawQuery() : "");
+    // Prepare the HTTP request.
+    HttpRequest request = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, query);
+    request.setHeader(HttpHeaders.Names.HOST, host);
+    LOG.info("Fetch: " + uri);
+    request.setHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE);
+    request.setHeader(HttpHeaders.Names.ACCEPT_ENCODING, HttpHeaders.Values.GZIP);
+
+
+    // Send the HTTP request.
+    ChannelFuture channelFuture = channel.write(request);
+
+    // Wait for the server to close the connection.
+    channel.getCloseFuture().awaitUninterruptibly();
+
+    channelFuture.addListener(ChannelFutureListener.CLOSE);
+
+    // Close the channel to exit.
+    future.getChannel().close();
+    finishTime = System.currentTimeMillis();
+    return file;
+  }
+
+  public URI getURI() {
+    return this.uri;
+  }
+
+  class HttpClientHandler extends SimpleChannelUpstreamHandler {
+    private volatile boolean readingChunks;
+    private final File file;
+    private RandomAccessFile raf;
+    private FileChannel fc;
+    private long length = -1;
+
+    public HttpClientHandler(File file) throws FileNotFoundException {
+      this.file = file;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+        throws Exception {
+      messageReceiveCount++;
+      try {
+        if (!readingChunks) {
+          HttpResponse response = (HttpResponse) e.getMessage();
+
+          StringBuilder sb = new StringBuilder();
+          if (LOG.isDebugEnabled()) {
+            sb.append("STATUS: ").append(response.getStatus())
+                .append(", VERSION: ").append(response.getProtocolVersion())
+                .append(", HEADER: ");
+          }
+          if (!response.getHeaderNames().isEmpty()) {
+            for (String name : response.getHeaderNames()) {
+              for (String value : response.getHeaders(name)) {
+                if (LOG.isDebugEnabled()) {
+                  sb.append(name).append(" = ").append(value);
+                }
+                if (this.length == -1 && name.equals("Content-Length")) {
+                  this.length = Long.valueOf(value);
+                }
+              }
+            }
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(sb.toString());
+          }
+
+          if (response.getStatus() == HttpResponseStatus.NO_CONTENT) {
+            LOG.info("There are no data corresponding to the request");
+            return;
+          }
+
+          this.raf = new RandomAccessFile(file, "rw");
+          this.fc = raf.getChannel();
+
+          if (response.isChunked()) {
+            readingChunks = true;
+          } else {
+            ChannelBuffer content = response.getContent();
+            if (content.readable()) {
+              fc.write(content.toByteBuffer());
+            }
+          }
+        } else {
+          HttpChunk chunk = (HttpChunk) e.getMessage();
+          if (chunk.isLast()) {
+            readingChunks = false;
+            long fileLength = file.length();
+            if (fileLength == length) {
+              LOG.info("Data fetch is done (total received bytes: " + fileLength
+                  + ")");
+            } else {
+              LOG.info("Data fetch is done, but cannot get all data "
+                  + "(received/total: " + fileLength + "/" + length + ")");
+            }
+          } else {
+            fc.write(chunk.getContent().toByteBuffer());
+          }
+        }
+      } finally {
+        if(raf != null) {
+          fileLen = file.length();
+        }
+
+        if(fileLen >= length){
+          IOUtils.cleanup(LOG, fc, raf);
+
+        }
+      }
+    }
+  }
+
+  class HttpClientPipelineFactory implements
+      ChannelPipelineFactory {
+    private final File file;
+
+    public HttpClientPipelineFactory(File file) {
+      this.file = file;
+    }
+
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+      ChannelPipeline pipeline = pipeline();
+
+      pipeline.addLast("codec", new HttpClientCodec());
+      pipeline.addLast("inflater", new HttpContentDecompressor());
+      pipeline.addLast("handler", new HttpClientHandler(file));
+      return pipeline;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/InterDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/InterDataRetriever.java b/tajo-core/src/main/java/org/apache/tajo/worker/InterDataRetriever.java
new file mode 100644
index 0000000..42ad875
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/InterDataRetriever.java
@@ -0,0 +1,113 @@
+/**
+ * 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.worker;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.apache.tajo.QueryUnitId;
+import org.apache.tajo.worker.dataserver.FileAccessForbiddenException;
+import org.apache.tajo.worker.dataserver.retriever.DataRetriever;
+import org.apache.tajo.worker.dataserver.retriever.FileChunk;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+@Deprecated
+public class InterDataRetriever implements DataRetriever {
+  private final Log LOG = LogFactory.getLog(InterDataRetriever.class);
+  private final Set<QueryUnitId> registered = Sets.newHashSet();
+  private final Map<String, String> map = Maps.newConcurrentMap();
+
+  public InterDataRetriever() {
+  }
+  
+  public void register(QueryUnitId id, String baseURI) {
+    synchronized (registered) {
+      if (!registered.contains(id)) {      
+        map.put(id.toString(), baseURI);
+        registered.add(id);      
+      }
+    } 
+  }
+  
+  public void unregister(QueryUnitId id) {
+    synchronized (registered) {
+      if (registered.contains(id)) {
+        map.remove(id.toString());
+        registered.remove(id);
+      }
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.tajo.worker.dataserver.retriever.DataRetriever#handle(org.jboss.netty.channel.ChannelHandlerContext, org.jboss.netty.handler.codec.http.HttpRequest)
+   */
+  @Override
+  public FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
+      throws IOException {
+       
+    int start = request.getUri().indexOf('?');
+    if (start < 0) {
+      throw new IllegalArgumentException("Wrong request: " + request.getUri());
+    }
+    
+    String queryStr = request.getUri().substring(start + 1);
+    LOG.info("QUERY: " + queryStr);
+    String [] queries = queryStr.split("&");
+    
+    String qid = null;
+    String fn = null;
+    String [] kv;
+    for (String query : queries) {
+      kv = query.split("=");
+      if (kv[0].equals("qid")) {
+        qid = kv[1];
+      } else if (kv[0].equals("fn")) {
+        fn = kv[1];
+      }
+    }
+    
+    String baseDir = map.get(qid);
+    if (baseDir == null) {
+      throw new FileNotFoundException("No such qid: " + qid);
+    }
+
+    File file = new File(baseDir + "/" + fn);
+    if (file.isHidden() || !file.exists()) {
+      throw new FileNotFoundException("No such file: " + baseDir + "/" 
+          + file.getName());
+    }
+    if (!file.isFile()) {
+      throw new FileAccessForbiddenException("No such file: " 
+          + baseDir + "/" + file.getName()); 
+    }
+    
+    return new FileChunk[] {new FileChunk(file, 0, file.length())};
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/PartitionRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/PartitionRetrieverHandler.java b/tajo-core/src/main/java/org/apache/tajo/worker/PartitionRetrieverHandler.java
new file mode 100644
index 0000000..36e7353
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/PartitionRetrieverHandler.java
@@ -0,0 +1,44 @@
+/**
+ * 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.worker;
+
+import org.apache.tajo.worker.dataserver.retriever.FileChunk;
+import org.apache.tajo.worker.dataserver.retriever.RetrieverHandler;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+public class PartitionRetrieverHandler implements RetrieverHandler {
+  private final String baseDir;
+
+  public PartitionRetrieverHandler(String baseDir) {
+    this.baseDir = baseDir;
+  }
+
+  @Override
+  public FileChunk get(Map<String, List<String>> kvs) throws IOException {
+    // nothing to verify the file because AdvancedDataRetriever checks
+    // its validity of the file.
+    File file = new File(baseDir + "/" + kvs.get("fn").get(0));
+
+    return new FileChunk(file, 0, file.length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/RangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/RangeRetrieverHandler.java b/tajo-core/src/main/java/org/apache/tajo/worker/RangeRetrieverHandler.java
new file mode 100644
index 0000000..be33a12
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/RangeRetrieverHandler.java
@@ -0,0 +1,163 @@
+/**
+ * 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.worker;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.worker.dataserver.retriever.FileChunk;
+import org.apache.tajo.worker.dataserver.retriever.RetrieverHandler;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ * It retrieves the file chunk ranged between start and end keys.
+ * The start key is inclusive, but the end key is exclusive.
+ *
+ * Internally, there are four cases:
+ * <ul>
+ *   <li>out of scope: the index range does not overlapped with the query range.</li>
+ *   <li>overlapped: the index range is partially overlapped with the query range. </li>
+ *   <li>included: the index range is included in the start and end keys</li>
+ *   <li>covered: the index range covers the query range (i.e., start and end keys).</li>
+ * </ul>
+ */
+public class RangeRetrieverHandler implements RetrieverHandler {
+  private static final Log LOG = LogFactory.getLog(RangeRetrieverHandler.class);
+  private final File file;
+  private final BSTIndex.BSTIndexReader idxReader;
+  private final Schema schema;
+  private final TupleComparator comp;
+  private final RowStoreDecoder decoder;
+
+  public RangeRetrieverHandler(File outDir, Schema schema, TupleComparator comp) throws IOException {
+    this.file = outDir;
+    BSTIndex index = new BSTIndex(new TajoConf());
+    this.schema = schema;
+    this.comp = comp;
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    Path indexPath = fs.makeQualified(new Path(outDir.getCanonicalPath(), "index"));
+    this.idxReader =
+        index.getIndexReader(indexPath, this.schema, this.comp);
+    this.idxReader.open();
+    LOG.info("BSTIndex is loaded from disk (" + idxReader.getFirstKey() + ", "
+        + idxReader.getLastKey());
+    this.decoder = RowStoreUtil.createDecoder(schema);
+  }
+
+  @Override
+  public FileChunk get(Map<String, List<String>> kvs) throws IOException {
+    // nothing to verify the file because AdvancedDataRetriever checks
+    // its validity of the file.
+    File data = new File(this.file, "data/data");
+    byte [] startBytes = Base64.decodeBase64(kvs.get("start").get(0));
+    Tuple start = decoder.toTuple(startBytes);
+    byte [] endBytes;
+    Tuple end;
+    endBytes = Base64.decodeBase64(kvs.get("end").get(0));
+    end = decoder.toTuple(endBytes);
+    boolean last = kvs.containsKey("final");
+
+    if(!comp.isAscendingFirstKey()) {
+      Tuple tmpKey = start;
+      start = end;
+      end = tmpKey;
+    }
+
+    LOG.info("GET Request for " + data.getAbsolutePath() + " (start="+start+", end="+ end +
+        (last ? ", last=true" : "") + ")");
+
+    if (idxReader.getFirstKey() == null && idxReader.getLastKey() == null) { // if # of rows is zero
+      LOG.info("There is no contents");
+      return null;
+    }
+
+    if (comp.compare(end, idxReader.getFirstKey()) < 0 ||
+        comp.compare(idxReader.getLastKey(), start) < 0) {
+      LOG.info("Out of Scope (indexed data [" + idxReader.getFirstKey() + ", " + idxReader.getLastKey() +
+          "], but request start:" + start + ", end: " + end);
+      return null;
+    }
+
+    long startOffset;
+    long endOffset;
+    try {
+      startOffset = idxReader.find(start);
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + "[" + start + ", " + end+")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+    try {
+      endOffset = idxReader.find(end);
+      if (endOffset == -1) {
+        endOffset = idxReader.find(end, true);
+      }
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + "[" + start + ", " + end+")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+
+    // if startOffset == -1 then case 2-1 or case 3
+    if (startOffset == -1) { // this is a hack
+      // if case 2-1 or case 3
+      try {
+        startOffset = idxReader.find(start, true);
+      } catch (IOException ioe) {
+        LOG.error("State Dump (the requested range: "
+            + "[" + start + ", " + end+")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+            + idxReader.getLastKey());
+        throw ioe;
+      }
+    }
+
+    if (startOffset == -1) {
+      throw new IllegalStateException("startOffset " + startOffset + " is negative \n" +
+          "State Dump (the requested range: "
+          + "[" + start + ", " + end+")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+    }
+
+    // if greater than indexed values
+    if (last || (endOffset == -1 && comp.compare(idxReader.getLastKey(), end) < 0)) {
+      endOffset = data.length();
+    }
+
+    FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset);
+    LOG.info("Retrieve File Chunk: " + chunk);
+    return chunk;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
new file mode 100644
index 0000000..8b9219c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
@@ -0,0 +1,29 @@
+/**
+ * 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.worker;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+
+public interface ResourceAllocator {
+  public void allocateTaskWorker();
+  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId);
+  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext,
+                                           int numTasks, int memoryMBPerTask);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
new file mode 100644
index 0000000..1731854
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
@@ -0,0 +1,49 @@
+/**
+ * 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.worker;
+
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.PhysicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.physical.PhysicalExec;
+import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.StorageManagerFactory;
+
+import java.io.IOException;
+
+public class TajoQueryEngine {
+
+  private final AbstractStorageManager storageManager;
+  private final PhysicalPlanner phyPlanner;
+
+  public TajoQueryEngine(TajoConf conf) throws IOException {
+    this.storageManager = StorageManagerFactory.getStorageManager(conf);
+    this.phyPlanner = new PhysicalPlannerImpl(conf, storageManager);
+  }
+  
+  public PhysicalExec createPlan(TaskAttemptContext ctx, LogicalNode plan)
+      throws InternalException {
+    return phyPlanner.createPlan(ctx, plan);
+  }
+  
+  public void stop() throws IOException {
+  }
+}


[44/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Acos.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Acos.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Acos.java
new file mode 100644
index 0000000..f1f2804
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Acos.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 acos(value FLOAT8)
+ */
+@Description(
+  functionName = "acos",
+  description = "Inverse cosine.",
+  detail = "",
+  example = "> SELECT acos(x);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Acos extends GeneralFunction {
+  public Acos() {
+    super(new Column[] {
+        new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.acos(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Asin.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Asin.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Asin.java
new file mode 100644
index 0000000..6f6c905
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Asin.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 asin(value FLOAT8)
+ */
+@Description(
+  functionName = "asin",
+  description = "Inverse sine.",
+  example = "> SELECT asin(x);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Asin extends GeneralFunction {
+  public Asin() {
+    super(new Column[] {
+        new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.asin(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan.java
new file mode 100644
index 0000000..9e70ae3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 atan(value FLOAT8)
+ */
+@Description(
+  functionName = "atan",
+  description = "Inverse tangent.",
+  example = "> SELECT atan(x);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Atan extends GeneralFunction {
+  public Atan() {
+    super(new Column[] {
+        new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.atan(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan2.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan2.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan2.java
new file mode 100644
index 0000000..bc2764c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Atan2.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 atan2(value FLOAT8, value FLOAT8)
+ */
+@Description(
+  functionName = "atan2",
+  description = "Inverse tangent of y/x.",
+  example = "> SELECT atan(y,x);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4, TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8, TajoDataTypes.Type.FLOAT8})}
+)
+public class Atan2 extends GeneralFunction {
+  public Atan2() {
+    super(new Column[] {
+        new Column("y", TajoDataTypes.Type.FLOAT8),
+        new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatumx = params.get(0);
+    Datum valueDatumy = params.get(1);
+    if(valueDatumx instanceof NullDatum || valueDatumy instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.atan2(valueDatumx.asFloat8(), valueDatumy.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cbrt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cbrt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cbrt.java
new file mode 100644
index 0000000..b85fe58
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cbrt.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 cbrt(value FLOAT8)
+ */
+@Description(
+  functionName = "cbrt",
+  description = "Cube root",
+  example = "> SELECT cbrt(27.0);\n"
+      + "3",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Cbrt extends GeneralFunction {
+  public Cbrt() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.cbrt(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Ceil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Ceil.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Ceil.java
new file mode 100644
index 0000000..cca7f60
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Ceil.java
@@ -0,0 +1,63 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 ceil(value FLOAT4/FLOAT8)
+ */
+@Description(
+  functionName = "ceil",
+  synonyms = {"ceiling"},
+  description = "Smallest integer not less than argument.",
+  example = "> SELECT ceil(-42.8);\n"
+          + "-42",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Ceil extends GeneralFunction {
+  public Ceil() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8((long)Math.ceil(valueDatum.asFloat8()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cos.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cos.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cos.java
new file mode 100644
index 0000000..997784a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Cos.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 cos(value FLOAT8)
+ */
+@Description(
+  functionName = "cos",
+  description = "Cosine.",
+  example = "> SELECT cos(x);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Cos extends GeneralFunction {
+  public Cos() {
+    super(new Column[] {
+        new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.cos(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Degrees.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Degrees.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Degrees.java
new file mode 100644
index 0000000..cd5fd2f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Degrees.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 degrees(value FLOAT8)
+ */
+@Description(
+  functionName = "degrees",
+  description = "Radians to degrees",
+  example = "> SELECT degrees(0.5);\n"
+      + "28.6478897565412",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Degrees extends GeneralFunction {
+  public Degrees() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.toDegrees(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Div.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Div.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Div.java
new file mode 100644
index 0000000..d904a2c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Div.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 div(value INT8, value INT8)
+ */
+@Description(
+  functionName = "div",
+  description = "Division(integer division truncates results)",
+  example = "> SELECT div(4, 2);\n"
+    + "2",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT4})
+  }
+)
+public class Div extends GeneralFunction {
+  public Div() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.INT8),
+      new Column("y", TajoDataTypes.Type.INT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum value1Datum = params.get(0);
+    if(value1Datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    Datum value2Datum = params.get(1);
+    if(value2Datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    long value1 = value1Datum.asInt8();
+    long value2 = value2Datum.asInt8();
+
+    if (value2 == 0) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8(value1/value2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Exp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Exp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Exp.java
new file mode 100644
index 0000000..3c3b07c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Exp.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 exp(value FLOAT8)
+ */
+@Description(
+  functionName = "exp",
+  description = "Exponential",
+  example = "> SELECT exp(1.0);\n"
+      + "2.71828182845905",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Exp extends GeneralFunction {
+  public Exp() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.exp(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Floor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Floor.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Floor.java
new file mode 100644
index 0000000..7e4cee0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Floor.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 floor(value FLOAT8)
+ */
+@Description(
+  functionName = "floor",
+  description = " Largest integer not greater than argument.",
+  example = "> SELECT floor(-42.8);\n" +
+      "-43",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Floor extends GeneralFunction {
+  public Floor() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8((long)Math.floor(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Mod.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Mod.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Mod.java
new file mode 100644
index 0000000..c748a2d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Mod.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 mod(value INT8, value INT8)
+ */
+@Description(
+  functionName = "mod",
+  description = "Remainder of y/x",
+  example = "> SELECT mod(9, 4);\n" +
+            "1",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT4})
+  }
+)
+public class Mod extends GeneralFunction {
+  public Mod() {
+    super(new Column[] {
+      new Column("y", TajoDataTypes.Type.INT8),
+      new Column("x", TajoDataTypes.Type.INT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum value1Datum = params.get(0);
+    if(value1Datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    Datum value2Datum = params.get(1);
+    if(value2Datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    long value1 = value1Datum.asInt8();
+    long value2 = value2Datum.asInt8();
+
+    if (value2 == 0) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8(value1%value2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pi.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pi.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pi.java
new file mode 100644
index 0000000..abc06b5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pi.java
@@ -0,0 +1,51 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 ceil(value FLOAT8)
+ */
+@Description(
+  functionName = "pi",
+  description = "\"π\" constant",
+  example = "> SELECT pi();\n"
+          + "3.14159265358979",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {})}
+)
+public class Pi extends GeneralFunction {
+  public Pi() {
+    super(NoArgs);
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    return DatumFactory.createFloat8(Math.PI);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pow.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pow.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pow.java
new file mode 100644
index 0000000..101e508
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Pow.java
@@ -0,0 +1,78 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 pow(value FLOAT8, value FLOAT8)
+ */
+@Description(
+  functionName = "pow",
+  description = "x raised to the power of y",
+  example = "> SELECT pow(9.0, 3.0)\n"
+           + "729",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4, TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4, TajoDataTypes.Type.FLOAT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8, TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8, TajoDataTypes.Type.FLOAT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4, TajoDataTypes.Type.FLOAT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.FLOAT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4, TajoDataTypes.Type.INT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4, TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8, TajoDataTypes.Type.INT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8, TajoDataTypes.Type.INT8})
+  }
+)
+public class Pow extends GeneralFunction {
+  public Pow() {
+    super(new Column[] {
+        new Column("x", TajoDataTypes.Type.FLOAT8),
+        new Column("y", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum value1Datum = params.get(0);
+    Datum value2Datum = params.get(1);
+    if(value1Datum instanceof NullDatum || value2Datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.pow(value1Datum.asFloat8(), value2Datum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Radians.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Radians.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Radians.java
new file mode 100644
index 0000000..a8bf363
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Radians.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 radians(value FLOAT8)
+ */
+@Description(
+  functionName = "radians",
+  description = "Degrees to radians",
+  example = "> SELECT radians(45.0)\n"
+      + "0.785398163397448",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4})
+  }
+)
+public class Radians extends GeneralFunction {
+  public Radians() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.toRadians(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Round.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Round.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Round.java
new file mode 100644
index 0000000..9740888
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Round.java
@@ -0,0 +1,71 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 round(value FLOAT8)
+ */
+@Description(
+  functionName = "round",
+  description = "Round to nearest integer.",
+  example = "> SELECT round(42.4)\n"
+          + "42",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Round extends GeneralFunction {
+  public Round() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    double value = valueDatum.asFloat8();
+
+    // Note: there are various round up/down approaches (https://en.wikipedia.org/wiki/Rounding#Tie-breaking).
+    //       Math.round uses an approach different from other programming languages, so the results of round function
+    //       can be different from other DBMSs. For example, Math.round(-5.5) returns -5. In contrast,
+    //       round function in MySQL and PostgreSQL returns -6. The below code is a workaround code for this.
+    if (value < 0) {
+      return DatumFactory.createInt8((long) Math.ceil(value - 0.5d));
+    } else {
+      return DatumFactory.createInt8((long) Math.floor(value + 0.5d));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sign.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sign.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sign.java
new file mode 100644
index 0000000..f9c49bf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sign.java
@@ -0,0 +1,64 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 sign(value FLOAT8)
+ */
+@Description(
+  functionName = "sign",
+  description = "sign of the argument (-1, 0, +1)",
+  example = "> SELECT sign(-8.4);\n"
+          + "-1",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})
+  }
+)
+public class Sign extends GeneralFunction {
+  public Sign() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.signum(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sin.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sin.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sin.java
new file mode 100644
index 0000000..31d56b4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sin.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 sin(value FLOAT8)
+ */
+@Description(
+  functionName = "sin",
+  description = "Sine.",
+  example = "> SELECT sin(x);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Sin extends GeneralFunction {
+  public Sin() {
+    super(new Column[] {
+        new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.sin(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sqrt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sqrt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sqrt.java
new file mode 100644
index 0000000..aba33f4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Sqrt.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 sqrt(value FLOAT8)
+ */
+@Description(
+  functionName = "sqrt",
+  description = "Square root",
+  example = "> SELECT sqrt(2.0);\n"
+          + "1.4142135623731",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4})
+  }
+)
+public class Sqrt extends GeneralFunction {
+  public Sqrt() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.sqrt(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Tan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Tan.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Tan.java
new file mode 100644
index 0000000..d551d8b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/Tan.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 tan(value FLOAT8)
+ */
+@Description(
+  functionName = "tan",
+  description = "Tangent.",
+  example = "> SELECT tan(x);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class Tan extends GeneralFunction {
+  public Tan() {
+    super(new Column[] {
+        new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.tan(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Ascii.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Ascii.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Ascii.java
new file mode 100644
index 0000000..ecb32a4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Ascii.java
@@ -0,0 +1,67 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * int4 ascii(string text)
+ */
+@Description(
+  functionName = "ascii",
+  description = "ASCII code of the first character of the argument.",
+  detail = "For UTF8 returns the Unicode code point of the character.\n"
+        + "For other multibyte encodings, the argument must be an ASCII character.",
+  example = "> SELECT ascii('x');\n"
+          + "120",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class Ascii extends GeneralFunction {
+
+
+  public Ascii() {
+    super(new Column[]{
+            new Column("char", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if (datum instanceof NullDatum) return NullDatum.get();
+
+    byte[] bytes = datum.asByteArray();
+    if (bytes.length > 0) {
+      return DatumFactory.createInt4(bytes[0]);
+    } else {
+      return DatumFactory.createInt4(0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
new file mode 100644
index 0000000..b7c5611
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.eval.FunctionEval;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text btrim(string text [, characters text])
+ */
+@Description(
+  functionName = "trim",
+  synonyms = {"btrim"},
+  description = " Remove the longest string consisting only of "
+          + " characters in characters (a space by default) "
+          + "from the start and end of string.",
+  example = "> SELECT trim('xyxtrimyyx', 'xy');\n"
+          + "trim",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT}),
+          @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT,TajoDataTypes.Type.TEXT})}
+)
+public class BTrim extends GeneralFunction {
+  @Expose private boolean hasTrimCharacters;
+
+  public BTrim() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+    });
+  }
+
+  public void init(FunctionEval.ParamType[] paramTypes) {
+    if (paramTypes.length == 2) {
+      hasTrimCharacters = true;
+    }
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    if (!hasTrimCharacters) {
+      return DatumFactory.createText(StringUtils.strip(datum.asChars(), null));
+    } else {
+      return DatumFactory.createText(StringUtils.strip(datum.asChars(), params.get(1).asChars()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BitLength.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BitLength.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BitLength.java
new file mode 100644
index 0000000..3a5c216
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BitLength.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT4 bit_length(string text)
+ */
+@Description(
+  functionName = "bit_length",
+  description = "Number of bits in string",
+  example = "> SELECT bit_length('jose');\n"
+          + "32",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class BitLength extends GeneralFunction {
+  public BitLength() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createInt4(datum.asByteArray().length*8);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/CharLength.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/CharLength.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/CharLength.java
new file mode 100644
index 0000000..76d3b55
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/CharLength.java
@@ -0,0 +1,59 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT4 char_length(string text) or INT4 character_length(string text)
+ */
+@Description(
+  functionName = "char_length",
+  synonyms = {"character_length"},
+  description = "Number of characters in string",
+  example = "> SELECT char_length('jose');\n"
+          + "4",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class CharLength extends GeneralFunction {
+  public CharLength() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createInt4(datum.asChars().length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Chr.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Chr.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Chr.java
new file mode 100644
index 0000000..fac4b27
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Chr.java
@@ -0,0 +1,65 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * char chr(value int4)
+ */
+@Description(
+  functionName = "chr",
+  description = "Character with the given code.",
+  detail = "For UTF8 the argument is treated as a Unicode code point. "
+    + "For other multibyte encodings the argument must designate an ASCII character.",
+  example = "> SELECT chr(65);\n"
+          + "A",
+  returnType = TajoDataTypes.Type.CHAR,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})}
+)
+public class Chr extends GeneralFunction {
+  public Chr() {
+    super(new Column[]{
+            new Column("n", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if (datum instanceof NullDatum) return NullDatum.get();
+
+    int value = datum.asInt4();
+    if (value <= 0 || value > 65525) {
+        return NullDatum.get();
+    } else {
+        return DatumFactory.createText(String.valueOf((char)value));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat.java
new file mode 100644
index 0000000..56b07eb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat.java
@@ -0,0 +1,73 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+
+/**
+ * Function definition
+ *
+ * text concat(str "any" [, str "any" [, ...] ])
+ */
+@Description(
+    functionName = "concat",
+    description = "Concatenate all arguments.",
+    detail = "Concatenate all arguments. NULL arguments are ignored.",
+    example = "> SELECT concat('abcde', '2');\n"
+        + "abcde2",
+    returnType = TajoDataTypes.Type.TEXT,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})
+        }
+)
+public class Concat extends GeneralFunction {
+  @Expose private boolean hasMoreCharacters;
+
+  public Concat() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    StringBuilder result = new StringBuilder(datum.asChars());
+
+    for(int i = 1 ; i < params.size() ; i++) {
+      Datum tmpDatum = params.get(i);
+      if(tmpDatum instanceof NullDatum)
+        continue;
+      result.append(tmpDatum.asChars());
+    }
+    return DatumFactory.createText(result.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat_ws.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat_ws.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat_ws.java
new file mode 100644
index 0000000..dd48151
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Concat_ws.java
@@ -0,0 +1,78 @@
+/**
+ * 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.function.string;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text concat(str "any" [, str "any" [, ...] ])
+ */
+@Description(
+    functionName = "concat_ws",
+    description = "Concatenate all but first arguments with separators.",
+    detail = "The first parameter is used as a separator. NULL arguments are ignored.",
+    example = "> concat_ws(',', 'abcde', 2);\n"
+        + "abcde,2",
+    returnType = TajoDataTypes.Type.TEXT,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
+)
+public class Concat_ws extends GeneralFunction {
+  @Expose private boolean hasMoreCharacters;
+
+  public Concat_ws() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("text", TajoDataTypes.Type.TEXT),
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum sepDatum = params.get(0);
+    Datum datum = params.get(1);
+
+
+    if(datum instanceof NullDatum) return NullDatum.get();
+    if(sepDatum instanceof NullDatum) return NullDatum.get();
+
+    StringBuilder result = new StringBuilder(datum.asChars());
+
+
+    for(int i = 2 ; i < params.size() ; i++) {
+      Datum tmpDatum = params.get(i);
+      if(tmpDatum instanceof NullDatum)
+        continue;
+      result.append(sepDatum.asChars());
+      result.append(tmpDatum.asChars());
+    }
+    return DatumFactory.createText(result.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Decode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Decode.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Decode.java
new file mode 100644
index 0000000..13f0749
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Decode.java
@@ -0,0 +1,91 @@
+/**
+ * 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.function.string;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+
+/**
+ * Function definition
+ *
+ * bytearray decode(string text, format text)
+ */
+@Description(
+  functionName = "decode",
+  description = "Decode binary data from textual representation in string. "
+          + "Options for format are same as in encode.",
+  detail = "format is one of 'base64', 'hex'",
+  example = "> SELECT decode('MTIzAAE=', 'base64');\n"
+          + "\\x3132330001",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
+)
+public class Decode extends GeneralFunction {
+  public Decode() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("format", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    Datum formatType = params.get(1);
+    String decodedBase64Text="";
+    String decodedHexString="";
+
+    if(datum instanceof NullDatum) return NullDatum.get();
+    if(formatType instanceof NullDatum) return NullDatum.get();
+
+    if(formatType.asChars().toLowerCase().equals("base64")) {
+      try {
+        // Base64
+        decodedBase64Text = new String(Base64.decodeBase64(datum.asChars().getBytes()));
+      }
+      catch (Exception e) {
+        return NullDatum.get();
+      }
+
+      return DatumFactory.createText(StringEscapeUtils.escapeJava(decodedBase64Text));
+    }
+    else if(formatType.asChars().toLowerCase().equals("hex")) {
+      try {
+        // Hex
+        decodedHexString = HexStringConverter.getInstance().decodeHex(datum.asChars());
+      }
+      catch (Exception e) {
+        return NullDatum.get();
+      }
+      return DatumFactory.createText(StringEscapeUtils.escapeJava(decodedHexString));
+    }
+    else
+      return NullDatum.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Digest.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Digest.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Digest.java
new file mode 100644
index 0000000..163c6fe
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Digest.java
@@ -0,0 +1,87 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import java.security.*;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+
+/**
+ * Function definition
+ *
+ * text digest(string text, type text)
+ */
+@Description(
+  functionName = "digest",
+  description = "Calculates the Digest hash of string",
+  example = "> SELECT digest('tajo', 'sha1');\n"
+          + "02b0e20540b89f0b735092bbac8093eb2e3804cf",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
+)
+
+public class Digest extends GeneralFunction {
+  public Digest() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  String digest(byte [] data, String type) throws NoSuchAlgorithmException{
+    if ("SHA1".equalsIgnoreCase(type) == true) {
+      MessageDigest md = MessageDigest.getInstance("SHA-1");
+      return new String(Hex.encodeHex(md.digest(data)));
+    } else if ("SHA256".equalsIgnoreCase(type) == true) {
+      return DigestUtils.sha256Hex(data);
+    } else if ("SHA384".equalsIgnoreCase(type) == true) {
+      return DigestUtils.sha384Hex(data);
+    } else if ("SHA512".equalsIgnoreCase(type) == true) {
+      return DigestUtils.sha512Hex(data);
+    } else if ("MD5".equalsIgnoreCase(type) == true) {
+      return DigestUtils.md5Hex(data);
+    } else if ("MD2".equalsIgnoreCase(type) == true) {
+      MessageDigest md = MessageDigest.getInstance("MD2");
+      return new String(Hex.encodeHex(md.digest(data)));
+    }
+
+    throw new NoSuchAlgorithmException("Not supported DigestType");
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    Datum typeDatum = params.get(1);
+    if(datum instanceof NullDatum || typeDatum instanceof NullDatum) return NullDatum.get();
+
+    try {
+        return DatumFactory.createText(digest(datum.asByteArray(), typeDatum.asChars()));
+    } catch (NoSuchAlgorithmException e){
+        return NullDatum.get();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Encode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Encode.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Encode.java
new file mode 100644
index 0000000..0023ae6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Encode.java
@@ -0,0 +1,93 @@
+/**
+ * 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.function.string;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+
+/**
+ * Function definition
+ *
+ * bytearray encode(data bytea, format text)
+ */
+@Description(
+  functionName = "encode",
+  description = "Encode binary data into a textual representation.",
+  detail = "Supported formats are: base64, hex, escape. escape converts zero bytes and "
+        + "high-bit-set bytes to octal sequences (\\nnn) and doubles backslashes.",
+  example = "> SELECT encode(E'123\\\\000\\\\001', 'base64');\n"
+          + "MTIzAAE=",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
+)
+public class Encode extends GeneralFunction {
+  public Encode() {
+    super(new Column[] {
+        new Column("string", TajoDataTypes.Type.TEXT),
+        new Column("format", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    Datum formatType = params.get(1);
+    String encodedBase64Text="";
+    String encodedHexString="";
+
+    if(datum instanceof NullDatum) return NullDatum.get();
+    if(formatType instanceof NullDatum) return NullDatum.get();
+
+
+    if(formatType.asChars().toLowerCase().equals("base64")) {
+      try {
+        // Base64
+        encodedBase64Text = new String(Base64.encodeBase64(StringEscapeUtils.unescapeJava(datum.asChars()).getBytes()));
+      }
+      catch (Exception e) {
+        return NullDatum.get();
+      }
+
+      return DatumFactory.createText(encodedBase64Text);
+    }
+    else if(formatType.asChars().toLowerCase().equals("hex")) {
+      try {
+        // Hex
+        encodedHexString = HexStringConverter.getInstance().encodeHex(StringEscapeUtils.unescapeJava(datum.asChars()));
+      }
+      catch (Exception e) {
+        return NullDatum.get();
+      }
+
+      return DatumFactory.createText(encodedHexString);
+    }
+    else
+      return NullDatum.get();
+  }
+}


[29/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/QueryRewriteEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/QueryRewriteEngine.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/QueryRewriteEngine.java
new file mode 100644
index 0000000..cb66582
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/QueryRewriteEngine.java
@@ -0,0 +1,32 @@
+/**
+ * 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.planner.rewrite;
+
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.PlanningException;
+
+public interface QueryRewriteEngine {
+  /**
+   * Rewrite a logical plan with all query rewrite rules added to this engine.
+   *
+   * @param plan The plan to be rewritten with all query rewrite rule.
+   * @return The rewritten plan.
+   */
+  LogicalPlan rewrite(LogicalPlan plan) throws PlanningException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/RewriteRule.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/RewriteRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/RewriteRule.java
new file mode 100644
index 0000000..89854df
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/RewriteRule.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.rewrite;
+
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.PlanningException;
+
+/**
+ * An interface for a rewrite rule.
+ */
+public interface RewriteRule {
+
+  /**
+   * It returns the rewrite rule name. It will be used for debugging and
+   * building a optimization history.
+   *
+   * @return The rewrite rule name
+   */
+  String getName();
+
+  /**
+   * This method checks if this rewrite rule can be applied to a given query plan.
+   * For example, the selection push down can not be applied to the query plan without any filter.
+   * In such case, it will return false.
+   *
+   * @param plan The plan to be checked
+   * @return True if this rule can be applied to a given plan. Otherwise, false.
+   */
+  boolean isEligible(LogicalPlan plan);
+
+  /**
+   * Updates a logical plan and returns an updated logical plan rewritten by this rule.
+   * It must be guaranteed that the input logical plan is not modified even after rewrite.
+   * In other words, the rewrite has to modify an plan copied from the input plan.
+   *
+   * @param plan Input logical plan. It will not be modified.
+   * @return The rewritten logical plan.
+   */
+  LogicalPlan rewrite(LogicalPlan plan) throws PlanningException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
new file mode 100644
index 0000000..44c4ddc
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
@@ -0,0 +1,191 @@
+/**
+ * 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.query;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.logical.NodeType;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.KeyValueSetProto;
+
+public class QueryContext extends Options {
+  public static final String COMMAND_TYPE = "tajo.query.command";
+
+  public static final String STAGING_DIR = "tajo.query.staging_dir";
+
+  public static final String USER_NAME = "tajo.query.username";
+
+  public static final String OUTPUT_TABLE_NAME = "tajo.query.output.table";
+  public static final String OUTPUT_TABLE_PATH = "tajo.query.output.path";
+  public static final String OUTPUT_PARTITIONS = "tajo.query.output.partitions";
+  public static final String OUTPUT_OVERWRITE = "tajo.query.output.overwrite";
+  public static final String OUTPUT_AS_DIRECTORY = "tajo.query.output.asdirectory";
+
+  public static final String TRUE_VALUE = "1";
+  public static final String FALSE_VALUE = "0";
+
+  public QueryContext() {}
+
+  public QueryContext(KeyValueSetProto proto) {
+    super(proto);
+  }
+
+  public void put(TajoConf.ConfVars key, String value) {
+    put(key.varname, value);
+  }
+
+  public String get(TajoConf.ConfVars key) {
+    return get(key.varname);
+  }
+
+  public String get(String key) {
+    return super.get(key);
+  }
+
+  public void setBool(String key, boolean val) {
+    put(key, val ? TRUE_VALUE : FALSE_VALUE);
+  }
+
+  public boolean getBool(String key) {
+    String strVal = get(key);
+    return strVal != null ? strVal.equals(TRUE_VALUE) : false;
+  }
+
+  public void setUser(String username) {
+    put(USER_NAME, username);
+  }
+
+  public String getUser() {
+    return get(USER_NAME);
+  }
+
+  public void setStagingDir(Path path) {
+    put(STAGING_DIR, path.toUri().toString());
+  }
+
+  public Path getStagingDir() {
+    String strVal = get(STAGING_DIR);
+    return strVal != null ? new Path(strVal) : null;
+  }
+
+  /**
+   * The fact that QueryContext has an output table means this query has a target table.
+   * In other words, this query is 'CREATE TABLE' or 'INSERT (OVERWRITE) INTO <table name>' statement.
+   * This config is not set if a query has INSERT (OVERWRITE) INTO LOCATION '/path/..'.
+   */
+  public boolean hasOutputTable() {
+    return get(OUTPUT_TABLE_NAME) != null;
+  }
+
+  /**
+   * Set a target table name
+   *
+   * @param tableName The target table name
+   */
+  public void setOutputTable(String tableName) {
+    put(OUTPUT_TABLE_NAME, tableName);
+  }
+
+  public String getOutputTable() {
+    String strVal = get(OUTPUT_TABLE_NAME);
+    return strVal != null ? strVal : null;
+  }
+
+  /**
+   * The fact that QueryContext has an output path means this query will write the output to a specific directory.
+   * In other words, this query is 'CREATE TABLE' or 'INSERT (OVERWRITE) INTO (<table name>|LOCATION)' statement.
+   *
+   * @return
+   */
+  public boolean hasOutputPath() {
+    return get(OUTPUT_TABLE_PATH) != null;
+  }
+
+  public void setOutputPath(Path path) {
+    put(OUTPUT_TABLE_PATH, path.toUri().toString());
+  }
+
+  public Path getOutputPath() {
+    String strVal = get(OUTPUT_TABLE_PATH);
+    return strVal != null ? new Path(strVal) : null;
+  }
+
+  public boolean hasPartition() {
+    return get(OUTPUT_PARTITIONS) != null;
+  }
+
+  public void setPartitionMethod(PartitionMethodDesc partitionMethodDesc) {
+    put(OUTPUT_PARTITIONS, partitionMethodDesc != null ? partitionMethodDesc.toJson() : null);
+  }
+
+  public PartitionMethodDesc getPartitionMethod() {
+    return PartitionMethodDesc.fromJson(get(OUTPUT_PARTITIONS));
+  }
+
+  public void setOutputOverwrite() {
+    setBool(OUTPUT_OVERWRITE, true);
+  }
+
+  public boolean isOutputOverwrite() {
+    return getBool(OUTPUT_OVERWRITE);
+  }
+
+  public void setFileOutput() {
+    setBool(OUTPUT_AS_DIRECTORY, true);
+  }
+
+  public boolean isFileOutput() {
+    return getBool(OUTPUT_AS_DIRECTORY);
+  }
+
+  public void setCommandType(NodeType nodeType) {
+    put(COMMAND_TYPE, nodeType.name());
+  }
+
+  public NodeType getCommandType() {
+    String strVal = get(COMMAND_TYPE);
+    return strVal != null ? NodeType.valueOf(strVal) : null;
+  }
+
+  public void setCreateTable() {
+    setCommandType(NodeType.CREATE_TABLE);
+  }
+
+  public boolean isCreateTable() {
+    return getCommandType() == NodeType.CREATE_TABLE;
+  }
+
+  public void setInsert() {
+    setCommandType(NodeType.INSERT);
+  }
+
+  public boolean isInsert() {
+    return getCommandType() == NodeType.INSERT;
+  }
+
+  public void setHiveQueryMode() {
+    setBool("hive.query.mode", true);
+  }
+
+  public boolean isHiveQueryMode() {
+    return getBool("hive.query.mode");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequest.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequest.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequest.java
new file mode 100644
index 0000000..383a787
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequest.java
@@ -0,0 +1,50 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+
+import java.net.URI;
+import java.util.List;
+
+public interface QueryUnitRequest extends ProtoObject<TajoWorkerProtocol.QueryUnitRequestProto> {
+
+	public QueryUnitAttemptId getId();
+	public List<CatalogProtos.FragmentProto> getFragments();
+	public String getOutputTableId();
+	public boolean isClusteredOutput();
+	public String getSerializedData();
+	public boolean isInterQuery();
+	public void setInterQuery();
+	public void addFetch(String name, URI uri);
+	public List<TajoWorkerProtocol.Fetch> getFetches();
+  public boolean shouldDie();
+  public void setShouldDie();
+  public QueryContext getQueryContext();
+  public DataChannel getDataChannel();
+  public Enforcer getEnforcer();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
new file mode 100644
index 0000000..d4006e0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
@@ -0,0 +1,328 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.ipc.TajoWorkerProtocol.Fetch;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProtoOrBuilder;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public class QueryUnitRequestImpl implements QueryUnitRequest {
+	
+  private QueryUnitAttemptId id;
+  private List<FragmentProto> fragments;
+  private String outputTable;
+	private boolean isUpdated;
+	private boolean clusteredOutput;
+	private String serializedData;     // logical node
+	private Boolean interQuery;
+	private List<Fetch> fetches;
+  private Boolean shouldDie;
+  private QueryContext queryContext;
+  private DataChannel dataChannel;
+  private Enforcer enforcer;
+	
+	private QueryUnitRequestProto proto = QueryUnitRequestProto.getDefaultInstance();
+	private QueryUnitRequestProto.Builder builder = null;
+	private boolean viaProto = false;
+	
+	public QueryUnitRequestImpl() {
+		builder = QueryUnitRequestProto.newBuilder();
+		this.id = null;
+		this.isUpdated = false;
+	}
+	
+	public QueryUnitRequestImpl(QueryUnitAttemptId id, List<FragmentProto> fragments,
+			String outputTable, boolean clusteredOutput,
+			String serializedData, QueryContext queryContext, DataChannel channel, Enforcer enforcer) {
+		this();
+		this.set(id, fragments, outputTable, clusteredOutput, serializedData, queryContext, channel, enforcer);
+	}
+	
+	public QueryUnitRequestImpl(QueryUnitRequestProto proto) {
+		this.proto = proto;
+		viaProto = true;
+		id = null;
+		isUpdated = false;
+	}
+	
+	public void set(QueryUnitAttemptId id, List<FragmentProto> fragments,
+			String outputTable, boolean clusteredOutput,
+			String serializedData, QueryContext queryContext, DataChannel dataChannel, Enforcer enforcer) {
+		this.id = id;
+		this.fragments = fragments;
+		this.outputTable = outputTable;
+		this.clusteredOutput = clusteredOutput;
+		this.serializedData = serializedData;
+		this.isUpdated = true;
+    this.queryContext = queryContext;
+    this.queryContext = queryContext;
+    this.dataChannel = dataChannel;
+    this.enforcer = enforcer;
+	}
+
+	@Override
+	public QueryUnitRequestProto getProto() {
+		mergeLocalToProto();
+		proto = viaProto ? proto : builder.build();
+		viaProto = true;
+		return proto;
+	}
+
+	@Override
+	public QueryUnitAttemptId getId() {
+		QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+		if (id != null) {
+			return this.id;
+		}
+		if (!p.hasId()) {
+			return null;
+		}
+		this.id = new QueryUnitAttemptId(p.getId());
+		return this.id;
+	}
+
+	@Override
+	public List<FragmentProto> getFragments() {
+		QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+		if (fragments != null) {
+			return fragments;
+		}
+		if (fragments == null) {
+			fragments = new ArrayList<FragmentProto>();
+		}
+		for (int i = 0; i < p.getFragmentsCount(); i++) {
+			fragments.add(p.getFragments(i));
+		}
+		return this.fragments;
+	}
+
+	@Override
+	public String getOutputTableId() {
+		QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+		if (outputTable != null) {
+			return this.outputTable;
+		}
+		if (!p.hasOutputTable()) {
+			return null;
+		}
+		this.outputTable = p.getOutputTable();
+		return this.outputTable;
+	}
+
+	@Override
+	public boolean isClusteredOutput() {
+		QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+		if (isUpdated) {
+			return this.clusteredOutput;
+		}
+		if (!p.hasClusteredOutput()) {
+			return false;
+		}
+		this.clusteredOutput = p.getClusteredOutput();
+		this.isUpdated = true;
+		return this.clusteredOutput;
+	}
+
+	@Override
+	public String getSerializedData() {
+		QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+		if (this.serializedData != null) {
+			return this.serializedData;
+		}
+		if (!p.hasSerializedData()) {
+			return null;
+		}
+		this.serializedData = p.getSerializedData();
+		return this.serializedData;
+	}
+	
+	public boolean isInterQuery() {
+	  QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (interQuery != null) {
+      return interQuery;
+    }
+    if (!p.hasInterQuery()) {
+      return false;
+    }
+    this.interQuery = p.getInterQuery();
+    return this.interQuery;
+	}
+	
+	public void setInterQuery() {
+	  maybeInitBuilder();
+	  this.interQuery = true;
+	}
+	
+	public void addFetch(String name, URI uri) {
+	  maybeInitBuilder();
+	  initFetches();
+	  fetches.add(
+	  Fetch.newBuilder()
+	    .setName(name)
+	    .setUrls(uri.toString()).build());
+	  
+	}
+
+  public QueryContext getQueryContext() {
+    QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (queryContext != null) {
+      return queryContext;
+    }
+    if (!p.hasQueryContext()) {
+      return null;
+    }
+    this.queryContext = new QueryContext(p.getQueryContext());
+    return this.queryContext;
+  }
+
+  public void setQueryContext(QueryContext queryContext) {
+    maybeInitBuilder();
+    this.queryContext = queryContext;
+  }
+
+  public void setDataChannel(DataChannel dataChannel) {
+    maybeInitBuilder();
+    this.dataChannel = dataChannel;
+  }
+
+  @Override
+  public DataChannel getDataChannel() {
+    QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (dataChannel != null) {
+      return dataChannel;
+    }
+    if (!p.hasDataChannel()) {
+      return null;
+    }
+    this.dataChannel = new DataChannel(p.getDataChannel());
+    return this.dataChannel;
+  }
+
+  @Override
+  public Enforcer getEnforcer() {
+    QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (enforcer != null) {
+      return enforcer;
+    }
+    if (!p.hasEnforcer()) {
+      return null;
+    }
+    this.enforcer = new Enforcer(p.getEnforcer());
+    return this.enforcer;
+  }
+
+  public List<Fetch> getFetches() {
+	  initFetches();    
+
+    return this.fetches;
+	}
+	
+	private void initFetches() {
+	  if (this.fetches != null) {
+      return;
+    }
+    QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+    this.fetches = new ArrayList<Fetch>();
+    for(Fetch fetch : p.getFetchesList()) {
+      fetches.add(fetch);
+    }
+	}
+
+  @Override
+  public boolean shouldDie() {
+    QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (shouldDie != null) {
+      return shouldDie;
+    }
+    if (!p.hasShouldDie()) {
+      return false;
+    }
+    this.shouldDie = p.getShouldDie();
+    return this.shouldDie;
+  }
+
+  @Override
+  public void setShouldDie() {
+    maybeInitBuilder();
+    shouldDie = true;
+  }
+
+  private void maybeInitBuilder() {
+		if (viaProto || builder == null) {
+			builder = QueryUnitRequestProto.newBuilder(proto);
+		}
+		viaProto = true;
+	}
+	
+	private void mergeLocalToBuilder() {
+		if (id != null) {
+			builder.setId(this.id.getProto());
+		}
+		if (fragments != null) {
+			for (int i = 0; i < fragments.size(); i++) {
+				builder.addFragments(fragments.get(i));
+			}
+		}
+		if (this.outputTable != null) {
+			builder.setOutputTable(this.outputTable);
+		}
+		if (this.isUpdated) {
+			builder.setClusteredOutput(this.clusteredOutput);
+		}
+		if (this.serializedData != null) {
+			builder.setSerializedData(this.serializedData);
+		}
+		if (this.interQuery != null) {
+		  builder.setInterQuery(this.interQuery);
+		}
+		if (this.fetches != null) {
+		  builder.addAllFetches(this.fetches);
+		}
+    if (this.shouldDie != null) {
+      builder.setShouldDie(this.shouldDie);
+    }
+    if (this.queryContext != null) {
+      builder.setQueryContext(queryContext.getProto());
+    }
+    if (this.dataChannel != null) {
+      builder.setDataChannel(dataChannel.getProto());
+    }
+    if (this.enforcer != null) {
+      builder.setEnforcer(enforcer.getProto());
+    }
+	}
+
+	private void mergeLocalToProto() {
+		if(viaProto) {
+			maybeInitBuilder();
+		}
+		mergeLocalToBuilder();
+		proto = builder.build();
+		viaProto = true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/utils/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/SchemaUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/SchemaUtil.java
new file mode 100644
index 0000000..c882607
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/SchemaUtil.java
@@ -0,0 +1,72 @@
+/**
+ * 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.utils;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+
+public class SchemaUtil {
+  public static Schema merge(Schema left, Schema right) {
+    Schema merged = new Schema();
+    for(Column col : left.getColumns()) {
+      if (!merged.containsByQualifiedName(col.getQualifiedName())) {
+        merged.addColumn(col);
+      }
+    }
+    for(Column col : right.getColumns()) {
+      if (!merged.containsByQualifiedName(col.getQualifiedName())) {
+        merged.addColumn(col);
+      }
+    }
+    
+    return merged;
+  }
+
+  /**
+   * Get common columns to be used as join keys of natural joins.
+   */
+  public static Schema getNaturalJoinColumns(Schema left, Schema right) {
+    Schema common = new Schema();
+    for (Column outer : left.getColumns()) {
+      if (!common.containsByName(outer.getSimpleName()) && right.containsByName(outer.getSimpleName())) {
+        common.addColumn(new Column(outer.getSimpleName(), outer.getDataType()));
+      }
+    }
+    
+    return common;
+  }
+
+  public static Schema getQualifiedLogicalSchema(TableDesc tableDesc, String tableName) {
+    Schema logicalSchema = new Schema(tableDesc.getLogicalSchema());
+    if (tableName != null) {
+      logicalSchema.setQualifier(tableName);
+    }
+    return logicalSchema;
+  }
+
+  public static <T extends Schema> T clone(Schema schema) {
+    try {
+      T copy = (T) schema.clone();
+      return copy;
+    } catch (CloneNotSupportedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/utils/ThreadUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/ThreadUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/ThreadUtil.java
new file mode 100644
index 0000000..23b1e5d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/ThreadUtil.java
@@ -0,0 +1,149 @@
+/**
+ * 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.utils;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import java.io.PrintWriter;
+import java.lang.Thread.UncaughtExceptionHandler;
+
+public class ThreadUtil {
+	protected static final Log LOG = LogFactory.getLog(ThreadUtil.class);
+
+	  /**
+	   * Utility method that sets name, daemon status and starts passed thread.
+	   * @param t thread to run
+	   * @return Returns the passed Thread <code>t</code>.
+	   */
+	  public static Thread setDaemonThreadRunning(final Thread t) {
+	    return setDaemonThreadRunning(t, t.getName());
+	  }
+
+	  /**
+	   * Utility method that sets name, daemon status and starts passed thread.
+	   * @param t thread to frob
+	   * @param name new name
+	   * @return Returns the passed Thread <code>t</code>.
+	   */
+	  public static Thread setDaemonThreadRunning(final Thread t,
+	    final String name) {
+	    return setDaemonThreadRunning(t, name, null);
+	  }
+
+	  /**
+	   * Utility method that sets name, daemon status and starts passed thread.
+	   * @param t thread to frob
+	   * @param name new name
+	   * @param handler A handler to set on the thread.  Pass null if want to
+	   * use default handler.
+	   * @return Returns the passed Thread <code>t</code>.
+	   */
+	  public static Thread setDaemonThreadRunning(final Thread t,
+	    final String name, final UncaughtExceptionHandler handler) {
+	    t.setName(name);
+	    if (handler != null) {
+	      t.setUncaughtExceptionHandler(handler);
+	    }
+	    t.setDaemon(true);
+	    t.start();
+	    return t;
+	  }
+
+	  /**
+	   * Shutdown passed thread using isAlive and join.
+	   * @param t Thread to shutdown
+	   */
+	  public static void shutdown(final Thread t) {
+	    shutdown(t, 0);
+	  }
+
+	  /**
+	   * Shutdown passed thread using isAlive and join.
+	   * @param joinwait Pass 0 if we're to wait forever.
+	   * @param t Thread to shutdown
+	   */
+	  public static void shutdown(final Thread t, final long joinwait) {
+	    if (t == null) return;
+	    while (t.isAlive()) {
+	      try {
+	        t.join(joinwait);
+	      } catch (InterruptedException e) {
+	        LOG.warn(t.getName() + "; joinwait=" + joinwait, e);
+	      }
+	    }
+	  }
+
+
+	  /**
+	   * @param t Waits on the passed thread to die dumping a threaddump every
+	   * minute while its up.
+	   * @throws InterruptedException
+	   */
+	  public static void threadDumpingIsAlive(final Thread t)
+	  throws InterruptedException {
+	    if (t == null) {
+	      return;
+	    }
+
+	    while (t.isAlive()) {
+	      t.join(60 * 1000);
+	      if (t.isAlive()) {
+	        ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
+	            "Automatic Stack Trace every 60 seconds waiting on " +
+	            t.getName());
+	      }
+	    }
+	  }
+
+	  /**
+	   * @param millis How long to sleep for in milliseconds.
+	   */
+	  public static void sleep(int millis) {
+	    try {
+	      Thread.sleep(millis);
+	    } catch (InterruptedException e) {
+	      e.printStackTrace();
+	    }
+	  }
+
+	  /**
+	   * Sleeps for the given amount of time even if interrupted. Preserves
+	   * the interrupt status.
+	   * @param msToWait the amount of time to sleep in milliseconds
+	   */
+	  public static void sleepWithoutInterrupt(final long msToWait) {
+	    long timeMillis = System.currentTimeMillis();
+	    long endTime = timeMillis + msToWait;
+	    boolean interrupted = false;
+	    while (timeMillis < endTime) {
+	      try {
+	        Thread.sleep(endTime - timeMillis);
+	      } catch (InterruptedException ex) {
+	        interrupted = true;
+	      }
+	      timeMillis = System.currentTimeMillis();
+	    }
+
+	    if (interrupted) {
+	      Thread.currentThread().interrupt();
+	    }
+	  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java
new file mode 100644
index 0000000..00647b5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java
@@ -0,0 +1,122 @@
+/**
+ * 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.utils;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TupleCache {
+  private static TupleCache instance;
+
+  private Map<TupleCacheKey, List<Tuple>> broadcastTupleCacheData
+      = new HashMap<TupleCacheKey, List<Tuple>>();
+  private Map<TupleCacheKey, TupleCacheStatus> broadcastTupleCacheStatus
+      = new HashMap<TupleCacheKey, TupleCacheStatus>();
+
+  private Object lockMonitor = new Object();
+
+  public static enum TupleCacheStatus {
+    STARTED,
+    ENDED
+  };
+
+  private TupleCache() {
+  }
+
+  public static synchronized TupleCache getInstance() {
+    if (instance == null) {
+      instance = new TupleCache();
+    }
+    return instance;
+  }
+
+  public Object getLockMonitor() {
+    return lockMonitor;
+  }
+
+  public void removeBroadcastCache(ExecutionBlockId ebId) {
+    if (ebId == null) {
+      return;
+    }
+    synchronized (lockMonitor) {
+      TupleCacheKey matchedKey = null;
+      for (TupleCacheKey eachKey: broadcastTupleCacheStatus.keySet()) {
+        if (eachKey.ebId.equals(ebId.toString())) {
+          matchedKey = eachKey;
+          break;
+        }
+      }
+      if (matchedKey != null) {
+        broadcastTupleCacheStatus.remove(matchedKey);
+        broadcastTupleCacheData.remove(matchedKey);
+      }
+    }
+  }
+
+  public void addBroadcastCache(TupleCacheKey cacheKey, List<Tuple> cacheData) {
+    synchronized (lockMonitor) {
+      if (broadcastTupleCacheStatus.containsKey(cacheKey) &&
+          broadcastTupleCacheStatus.get(cacheKey) == TupleCacheStatus.ENDED) {
+        return;
+      }
+      broadcastTupleCacheData.put(cacheKey, cacheData);
+      broadcastTupleCacheStatus.put(cacheKey, TupleCacheStatus.ENDED);
+      lockMonitor.notifyAll();
+    }
+  }
+
+  public boolean lockBroadcastScan(TupleCacheKey cacheKey) {
+    synchronized (lockMonitor) {
+      if (broadcastTupleCacheStatus.containsKey(cacheKey)) {
+        return false;
+      } else {
+        broadcastTupleCacheStatus.put(cacheKey, TupleCacheStatus.STARTED);
+        return true;
+      }
+    }
+  }
+
+  public boolean isBroadcastCacheReady(TupleCacheKey cacheKey) {
+    synchronized (lockMonitor) {
+      if (!broadcastTupleCacheStatus.containsKey(cacheKey)) {
+        return false;
+      }
+      return broadcastTupleCacheStatus.get(cacheKey) == TupleCacheStatus.ENDED;
+    }
+  }
+
+  public TupleCacheScanner openCacheScanner(TupleCacheKey cacheKey, Schema schema) throws IOException {
+    synchronized (lockMonitor) {
+      List<Tuple> cacheData = broadcastTupleCacheData.get(cacheKey);
+      if (cacheData != null) {
+        TupleCacheScanner scanner = new TupleCacheScanner(cacheData, schema);
+        scanner.init();
+        return scanner;
+      } else {
+        return null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java
new file mode 100644
index 0000000..ad9204f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java
@@ -0,0 +1,60 @@
+/**
+ * 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.utils;
+
+public class TupleCacheKey {
+  String ebId;
+  String tableName;
+
+  public TupleCacheKey(String ebId, String tableName) {
+    this.ebId = ebId;
+    this.tableName = tableName;
+  }
+
+  public String getEbId() {
+    return ebId;
+  }
+
+  public void setEbId(String ebId) {
+    this.ebId = ebId;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return toString().equals(o.toString());
+  }
+
+  @Override
+  public String toString() {
+    return ebId + "," + tableName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheScanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheScanner.java
new file mode 100644
index 0000000..3b91f94
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheScanner.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.utils;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+public class TupleCacheScanner implements Scanner {
+  List<Tuple> cacheData;
+  Schema schema;
+  Iterator<Tuple> it;
+  int count;
+  TableStats inputStats = new TableStats();
+
+  public TupleCacheScanner(List<Tuple> cacheData, Schema schema) {
+    this.cacheData = cacheData;
+    this.schema = schema;
+  }
+  @Override
+  public void init() throws IOException {
+    inputStats.setNumRows(cacheData.size());
+    inputStats.setReadBytes(0);
+    it = cacheData.iterator();
+    count = 0;
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (it.hasNext()) {
+      count++;
+      return it.next();
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    init();
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  @Override
+  public void setTarget(Column[] targets) {
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return true;
+  }
+
+  @Override
+  public void setSearchCondition(Object expr) {
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return false;
+  }
+
+  @Override
+  public float getProgress() {
+    if (cacheData.size() == 0) {
+      return 1.0f;
+    }
+    return count / cacheData.size();
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    return inputStats;
+  }
+
+  @Override
+  public Schema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
new file mode 100644
index 0000000..86f4935
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
@@ -0,0 +1,231 @@
+/**
+ * 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.utils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.storage.VTuple;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public class TupleUtil {
+
+  public static String rangeToQuery(Schema schema, TupleRange range, boolean last)
+      throws UnsupportedEncodingException {
+    return rangeToQuery(range, last, RowStoreUtil.createEncoder(schema));
+  }
+
+  public static String rangeToQuery(TupleRange range, boolean last, RowStoreEncoder encoder)
+      throws UnsupportedEncodingException {
+    StringBuilder sb = new StringBuilder();
+    byte [] firstKeyBytes = encoder.toBytes(range.getStart());
+    byte [] endKeyBytes = encoder.toBytes(range.getEnd());
+
+    String firstKeyBase64 = new String(Base64.encodeBase64(firstKeyBytes));
+    String lastKeyBase64 = new String(Base64.encodeBase64(endKeyBytes));
+
+    sb.append("start=")
+        .append(URLEncoder.encode(firstKeyBase64, "utf-8"))
+        .append("&")
+        .append("end=")
+        .append(URLEncoder.encode(lastKeyBase64, "utf-8"));
+
+    if (last) {
+      sb.append("&final=true");
+    }
+
+    return sb.toString();
+  }
+
+  public static TupleRange columnStatToRange(SortSpec [] sortSpecs, Schema target, List<ColumnStats> colStats) {
+
+    Map<Column, ColumnStats> statSet = Maps.newHashMap();
+    for (ColumnStats stat : colStats) {
+      statSet.put(stat.getColumn(), stat);
+    }
+
+    for (Column col : target.getColumns()) {
+      Preconditions.checkState(statSet.containsKey(col),
+          "ERROR: Invalid Column Stats (column stats: " + colStats + ", there exists not target " + col);
+    }
+
+    Tuple startTuple = new VTuple(target.size());
+    Tuple endTuple = new VTuple(target.size());
+    int i = 0;
+
+    // In outer join, empty table could be searched.
+    // As a result, min value and max value would be null.
+    // So, we should put NullDatum for this case.
+    for (Column col : target.getColumns()) {
+      if (sortSpecs[i].isAscending()) {
+        if (statSet.get(col).getMinValue() != null)
+          startTuple.put(i, statSet.get(col).getMinValue());
+        else
+          startTuple.put(i, DatumFactory.createNullDatum());
+
+        if (statSet.get(col).getMaxValue() != null)
+          endTuple.put(i, statSet.get(col).getMaxValue());
+        else
+          endTuple.put(i, DatumFactory.createNullDatum());
+      } else {
+        if (statSet.get(col).getMaxValue() != null)
+          startTuple.put(i, statSet.get(col).getMaxValue());
+        else
+          startTuple.put(i, DatumFactory.createNullDatum());
+
+        if (statSet.get(col).getMinValue() != null)
+          endTuple.put(i, statSet.get(col).getMinValue());
+        else
+          endTuple.put(i, DatumFactory.createNullDatum());
+      }
+      i++;
+    }
+    return new TupleRange(sortSpecs, startTuple, endTuple);
+  }
+
+  /**
+   * It creates a tuple of a given size filled with NULL values in all fields
+   * It is usually used in outer join algorithms.
+   *
+   * @param size The number of columns of a creating tuple
+   * @return The created tuple filled with NULL values
+   */
+  public static Tuple createNullPaddedTuple(int size){
+    VTuple aTuple = new VTuple(size);
+    int i;
+    for(i = 0; i < size; i++){
+      aTuple.put(i, DatumFactory.createNullDatum());
+    }
+    return aTuple;
+  }
+
+  @SuppressWarnings("unused")
+  public static Collection<Tuple> filterTuple(Schema schema, Collection<Tuple> tupleBlock, EvalNode filterCondition) {
+    TupleBlockFilterScanner filter = new TupleBlockFilterScanner(schema, tupleBlock, filterCondition);
+    return filter.nextBlock();
+  }
+
+  private static class TupleBlockFilterScanner {
+    private EvalNode qual;
+    private Iterator<Tuple> iterator;
+    private Schema schema;
+
+    public TupleBlockFilterScanner(Schema schema, Collection<Tuple> tuples, EvalNode qual) {
+      this.schema = schema;
+      this.qual = qual;
+      this.iterator = tuples.iterator();
+    }
+
+    public List<Tuple> nextBlock() {
+      List<Tuple> results = Lists.newArrayList();
+
+      Tuple tuple;
+      while (iterator.hasNext()) {
+        tuple = iterator.next();
+        if (qual.eval(schema, tuple).isTrue()) {
+          results.add(tuple);
+        }
+      }
+      return results;
+    }
+  }
+
+  /**
+   * Take a look at a column partition path. A partition path consists
+   * of a table path part and column values part. This method transforms
+   * a partition path into a tuple with a given partition column schema.
+   *
+   * hdfs://192.168.0.1/tajo/warehouse/table1/col1=abc/col2=def/col3=ghi
+   *                   ^^^^^^^^^^^^^^^^^^^^^  ^^^^^^^^^^^^^^^^^^^^^^^^^^
+   *                      table path part        column values part
+   *
+   * When a file path is given, it can perform two ways depending on beNullIfFile flag.
+   * If it is true, it returns NULL when a given path is a file.
+   * Otherwise, it returns a built tuple regardless of file or directory.
+   *
+   * @param partitionColumnSchema The partition column schema
+   * @param partitionPath The partition path
+   * @param beNullIfFile If true, this method returns NULL when a given path is a file.
+   * @return The tuple transformed from a column values part.
+   */
+  public static Tuple buildTupleFromPartitionPath(Schema partitionColumnSchema, Path partitionPath,
+                                                  boolean beNullIfFile) {
+    int startIdx = partitionPath.toString().indexOf(getColumnPartitionPathPrefix(partitionColumnSchema));
+
+    if (startIdx == -1) { // if there is no partition column in the patch
+      return null;
+    }
+    String columnValuesPart = partitionPath.toString().substring(startIdx);
+
+    String [] columnValues = columnValuesPart.split("/");
+
+    // true means this is a file.
+    if (beNullIfFile && partitionColumnSchema.size() < columnValues.length) {
+      return null;
+    }
+
+    Tuple tuple = new VTuple(partitionColumnSchema.size());
+    int i = 0;
+    for (; i < columnValues.length && i < partitionColumnSchema.size(); i++) {
+      String [] parts = columnValues[i].split("=");
+      if (parts.length != 2) {
+        return null;
+      }
+      int columnId = partitionColumnSchema.getColumnIdByName(parts[0]);
+      Column keyColumn = partitionColumnSchema.getColumn(columnId);
+      tuple.put(columnId, DatumFactory.createFromString(keyColumn.getDataType(), parts[1]));
+    }
+    for (; i < partitionColumnSchema.size(); i++) {
+      tuple.put(i, NullDatum.get());
+    }
+    return tuple;
+  }
+
+  /**
+   * Get a prefix of column partition path. For example, consider a column partition (col1, col2).
+   * Then, you will get a string 'col1='.
+   *
+   * @param partitionColumn the schema of column partition
+   * @return The first part string of column partition path.
+   */
+  private static String getColumnPartitionPathPrefix(Schema partitionColumn) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(partitionColumn.getColumn(0).getSimpleName()).append("=");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java
new file mode 100644
index 0000000..320a5aa
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.master.event.TaskRequestEvent;
+import org.apache.tajo.master.event.TaskSchedulerEvent;
+
+
+public abstract class AbstractTaskScheduler extends AbstractService implements EventHandler<TaskSchedulerEvent> {
+
+  protected int hostLocalAssigned;
+  protected int rackLocalAssigned;
+  protected int totalAssigned;
+
+  /**
+   * Construct the service.
+   *
+   * @param name service name
+   */
+  public AbstractTaskScheduler(String name) {
+    super(name);
+  }
+
+  public int getHostLocalAssigned() {
+    return hostLocalAssigned;
+  }
+
+  public int getRackLocalAssigned() {
+    return rackLocalAssigned;
+  }
+
+  public int getTotalAssigned() {
+    return totalAssigned;
+  }
+
+  public abstract void handleTaskRequestEvent(TaskRequestEvent event);
+  public abstract int remainingScheduledObjectNum();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
new file mode 100644
index 0000000..59b071a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
@@ -0,0 +1,81 @@
+/**
+ * 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.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+
+public abstract class ContainerProxy {
+  protected static final Log LOG = LogFactory.getLog(ContainerProxy.class);
+
+  final public static FsPermission QUERYCONF_FILE_PERMISSION =
+          FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+
+  protected static enum ContainerState {
+    PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
+  }
+
+  protected final ExecutionBlockId executionBlockId;
+  protected Configuration conf;
+  protected QueryMasterTask.QueryMasterTaskContext context;
+
+  protected ContainerState state;
+  // store enough information to be able to cleanup the container
+  protected Container container;
+  protected ContainerId containerID;
+  protected String hostName;
+  protected int port = -1;
+
+  public abstract void launch(ContainerLaunchContext containerLaunchContext);
+  public abstract void stopContainer();
+
+  public ContainerProxy(QueryMasterTask.QueryMasterTaskContext context, Configuration conf,
+                        ExecutionBlockId executionBlockId, Container container) {
+    this.context = context;
+    this.conf = conf;
+    this.state = ContainerState.PREP;
+    this.container = container;
+    this.executionBlockId = executionBlockId;
+    this.containerID = container.getId();
+  }
+
+  public synchronized boolean isCompletelyDone() {
+    return state == ContainerState.DONE || state == ContainerState.FAILED;
+  }
+
+  public String getTaskHostName() {
+    return this.hostName;
+  }
+
+  public int getTaskPort() {
+    return this.port;
+  }
+
+  public String getId() {
+    return executionBlockId.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java
new file mode 100644
index 0000000..e4b98d4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java
@@ -0,0 +1,247 @@
+/*
+ * 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.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.util.NetUtils;
+
+import java.util.*;
+import java.util.Map.Entry;
+
+/**
+ * DefaultFragmentScheduleAlgorithm selects a fragment randomly for the given argument.
+ * For example, when getHostLocalFragment(host, disk) is called, this algorithm randomly selects a fragment among
+ * the fragments which are stored at the disk of the host specified by the arguments.
+ */
+public class DefaultFragmentScheduleAlgorithm implements FragmentScheduleAlgorithm {
+  private final static Log LOG = LogFactory.getLog(DefaultFragmentScheduleAlgorithm.class);
+  private Map<String, Map<Integer, FragmentsPerDisk>> fragmentHostMapping =
+      new HashMap<String, Map<Integer, FragmentsPerDisk>>();
+  private Map<String, Set<FragmentPair>> rackFragmentMapping =
+      new HashMap<String, Set<FragmentPair>>();
+  private int fragmentNum = 0;
+  private Random random = new Random(System.currentTimeMillis());
+
+  public static class FragmentsPerDisk {
+    private Integer diskId;
+    private Set<FragmentPair> fragmentPairSet;
+
+    public FragmentsPerDisk(Integer diskId) {
+      this.diskId = diskId;
+      this.fragmentPairSet = Collections.newSetFromMap(new HashMap<FragmentPair, Boolean>());
+    }
+
+    public Integer getDiskId() {
+      return diskId;
+    }
+
+    public Set<FragmentPair> getFragmentPairSet() {
+      return fragmentPairSet;
+    }
+
+    public void addFragmentPair(FragmentPair fragmentPair) {
+      fragmentPairSet.add(fragmentPair);
+    }
+
+    public boolean removeFragmentPair(FragmentPair fragmentPair) {
+      return fragmentPairSet.remove(fragmentPair);
+    }
+
+    public int size() {
+      return fragmentPairSet.size();
+    }
+
+    public Iterator<FragmentPair> getFragmentPairIterator() {
+      return fragmentPairSet.iterator();
+    }
+
+    public boolean isEmpty() {
+      return fragmentPairSet.isEmpty();
+    }
+  }
+
+  @Override
+  public void addFragment(FragmentPair fragmentPair) {
+    String[] hosts = fragmentPair.getLeftFragment().getHosts();
+    int[] diskIds = fragmentPair.getLeftFragment().getDiskIds();
+    for (int i = 0; i < hosts.length; i++) {
+      addFragment(hosts[i], diskIds[i], fragmentPair);
+    }
+    fragmentNum++;
+  }
+
+  private void addFragment(String host, Integer diskId, FragmentPair fragmentPair) {
+    // update the fragment maps per host
+    String normalizeHost = NetUtils.normalizeHost(host);
+    Map<Integer, FragmentsPerDisk> diskFragmentMap;
+    if (fragmentHostMapping.containsKey(normalizeHost)) {
+      diskFragmentMap = fragmentHostMapping.get(normalizeHost);
+    } else {
+      diskFragmentMap = new HashMap<Integer, FragmentsPerDisk>();
+      fragmentHostMapping.put(normalizeHost, diskFragmentMap);
+    }
+    FragmentsPerDisk fragmentsPerDisk;
+    if (diskFragmentMap.containsKey(diskId)) {
+      fragmentsPerDisk = diskFragmentMap.get(diskId);
+    } else {
+      fragmentsPerDisk = new FragmentsPerDisk(diskId);
+      diskFragmentMap.put(diskId, fragmentsPerDisk);
+    }
+    fragmentsPerDisk.addFragmentPair(fragmentPair);
+
+    // update the fragment maps per rack
+    String rack = RackResolver.resolve(normalizeHost).getNetworkLocation();
+    Set<FragmentPair> fragmentPairList;
+    if (rackFragmentMapping.containsKey(rack)) {
+      fragmentPairList = rackFragmentMapping.get(rack);
+    } else {
+      fragmentPairList = Collections.newSetFromMap(new HashMap<FragmentPair, Boolean>());
+      rackFragmentMapping.put(rack, fragmentPairList);
+    }
+    fragmentPairList.add(fragmentPair);
+  }
+
+  @Override
+  public void removeFragment(FragmentPair fragmentPair) {
+    boolean removed = false;
+    for (String eachHost : fragmentPair.getLeftFragment().getHosts()) {
+      String normalizedHost = NetUtils.normalizeHost(eachHost);
+      Map<Integer, FragmentsPerDisk> diskFragmentMap = fragmentHostMapping.get(normalizedHost);
+      for (Entry<Integer, FragmentsPerDisk> entry : diskFragmentMap.entrySet()) {
+        FragmentsPerDisk fragmentsPerDisk = entry.getValue();
+        removed = fragmentsPerDisk.removeFragmentPair(fragmentPair);
+        if (removed) {
+          if (fragmentsPerDisk.size() == 0) {
+            diskFragmentMap.remove(entry.getKey());
+          }
+          if (diskFragmentMap.size() == 0) {
+            fragmentHostMapping.remove(normalizedHost);
+          }
+          break;
+        }
+      }
+      String rack = RackResolver.resolve(normalizedHost).getNetworkLocation();
+      if (rackFragmentMapping.containsKey(rack)) {
+        Set<FragmentPair> fragmentPairs = rackFragmentMapping.get(rack);
+        fragmentPairs.remove(fragmentPair);
+        if (fragmentPairs.size() == 0) {
+          rackFragmentMapping.remove(rack);
+        }
+      }
+    }
+    if (removed) {
+      fragmentNum--;
+    }
+  }
+
+  /**
+   * Randomly select a fragment among the fragments stored on the host.
+   * @param host
+   * @return a randomly selected fragment
+   */
+  @Override
+  public FragmentPair getHostLocalFragment(String host) {
+    String normalizedHost = NetUtils.normalizeHost(host);
+    if (fragmentHostMapping.containsKey(normalizedHost)) {
+      Collection<FragmentsPerDisk> disks = fragmentHostMapping.get(normalizedHost).values();
+      Iterator<FragmentsPerDisk> diskIterator = disks.iterator();
+      int randomIndex = random.nextInt(disks.size());
+      FragmentsPerDisk fragmentsPerDisk = null;
+      for (int i = 0; i < randomIndex; i++) {
+        fragmentsPerDisk = diskIterator.next();
+      }
+
+      if (fragmentsPerDisk != null) {
+        Iterator<FragmentPair> fragmentIterator = fragmentsPerDisk.getFragmentPairIterator();
+        if (fragmentIterator.hasNext()) {
+          return fragmentIterator.next();
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Randomly select a fragment among the fragments stored at the disk of the host.
+   * @param host
+   * @param diskId
+   * @return a randomly selected fragment
+   */
+  @Override
+  public FragmentPair getHostLocalFragment(String host, Integer diskId) {
+    String normalizedHost = NetUtils.normalizeHost(host);
+    if (fragmentHostMapping.containsKey(normalizedHost)) {
+      Map<Integer, FragmentsPerDisk> fragmentsPerDiskMap = fragmentHostMapping.get(normalizedHost);
+      if (fragmentsPerDiskMap.containsKey(diskId)) {
+        FragmentsPerDisk fragmentsPerDisk = fragmentsPerDiskMap.get(diskId);
+        if (!fragmentsPerDisk.isEmpty()) {
+          return fragmentsPerDisk.getFragmentPairIterator().next();
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Randomly select a fragment among the fragments stored on nodes of the same rack with the host.
+   * @param host
+   * @return a randomly selected fragment
+   */
+  @Override
+  public FragmentPair getRackLocalFragment(String host) {
+    String rack = RackResolver.resolve(host).getNetworkLocation();
+    if (rackFragmentMapping.containsKey(rack)) {
+      Set<FragmentPair> fragmentPairs = rackFragmentMapping.get(rack);
+      if (!fragmentPairs.isEmpty()) {
+        return fragmentPairs.iterator().next();
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Randomly select a fragment among the total fragments.
+   * @return a randomly selected fragment
+   */
+  @Override
+  public FragmentPair getRandomFragment() {
+    if (!fragmentHostMapping.isEmpty()) {
+      return fragmentHostMapping.values().iterator().next().values().iterator().next().getFragmentPairIterator().next();
+    }
+    return null;
+  }
+
+  @Override
+  public FragmentPair[] getAllFragments() {
+    List<FragmentPair> fragmentPairs = new ArrayList<FragmentPair>();
+    for (Map<Integer, FragmentsPerDisk> eachDiskFragmentMap : fragmentHostMapping.values()) {
+      for (FragmentsPerDisk fragmentsPerDisk : eachDiskFragmentMap.values()) {
+        fragmentPairs.addAll(fragmentsPerDisk.fragmentPairSet);
+      }
+    }
+    return fragmentPairs.toArray(new FragmentPair[fragmentPairs.size()]);
+  }
+
+  @Override
+  public int size() {
+    return fragmentNum;
+  }
+}


[35/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
new file mode 100644
index 0000000..bf2bf7d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
@@ -0,0 +1,1187 @@
+/**
+ * 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.planner.global;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.eval.AggregationFunctionCallEval;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.eval.EvalTreeUtil;
+import org.apache.tajo.engine.eval.FieldEval;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.planner.rewrite.ProjectionPushDownRule;
+import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.*;
+
+/**
+ * Build DAG
+ */
+public class GlobalPlanner {
+  private static Log LOG = LogFactory.getLog(GlobalPlanner.class);
+
+  private final TajoConf conf;
+  private final CatalogProtos.StoreType storeType;
+  private CatalogService catalog;
+  private TajoWorker.WorkerContext workerContext;
+
+  public GlobalPlanner(final TajoConf conf, final CatalogService catalog) throws IOException {
+    this.conf = conf;
+    this.catalog = catalog;
+    this.storeType = CatalogProtos.StoreType.valueOf(conf.getVar(ConfVars.SHUFFLE_FILE_FORMAT).toUpperCase());
+    Preconditions.checkArgument(storeType != null);
+  }
+
+  public GlobalPlanner(final TajoConf conf, final TajoWorker.WorkerContext workerContext) throws IOException {
+    this.conf = conf;
+    this.workerContext = workerContext;
+    this.storeType = CatalogProtos.StoreType.valueOf(conf.getVar(ConfVars.SHUFFLE_FILE_FORMAT).toUpperCase());
+    Preconditions.checkArgument(storeType != null);
+  }
+
+  /**
+   * TODO: this is hack. it must be refactored at TAJO-602.
+   */
+  public CatalogService getCatalog() {
+    if (workerContext.getCatalog() != null) {
+      return workerContext.getCatalog();
+    } else if (catalog != null) {
+      return catalog;
+    } else {
+      throw new IllegalStateException("No Catalog Instance");
+    }
+  }
+
+  public class GlobalPlanContext {
+    MasterPlan plan;
+    Map<Integer, ExecutionBlock> execBlockMap = Maps.newHashMap();
+  }
+
+  /**
+   * Builds a master plan from the given logical plan.
+   */
+  public void build(MasterPlan masterPlan) throws IOException, PlanningException {
+
+    DistributedPlannerVisitor planner = new DistributedPlannerVisitor();
+    GlobalPlanContext globalPlanContext = new GlobalPlanContext();
+    globalPlanContext.plan = masterPlan;
+
+    LOG.info(masterPlan.getLogicalPlan());
+
+    // copy a logical plan in order to keep the original logical plan. The distributed planner can modify
+    // an input logical plan.
+    LogicalNode inputPlan = PlannerUtil.clone(masterPlan.getLogicalPlan(),
+        masterPlan.getLogicalPlan().getRootBlock().getRoot());
+
+    boolean autoBroadcast = conf.getBoolVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO);
+    if (autoBroadcast) {
+      // pre-visit finding broadcast join target table
+      // this visiting doesn't make any execution block and change plan
+      BroadcastJoinPlanVisitor broadcastJoinPlanVisitor = new BroadcastJoinPlanVisitor();
+      broadcastJoinPlanVisitor.visit(globalPlanContext,
+          masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack<LogicalNode>());
+    }
+
+    // create a distributed execution plan by visiting each logical node.
+    // Its output is a graph, where each vertex is an execution block, and each edge is a data channel.
+    // MasterPlan contains them.
+    LogicalNode lastNode = planner.visit(globalPlanContext,
+        masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack<LogicalNode>());
+    ExecutionBlock childExecBlock = globalPlanContext.execBlockMap.get(lastNode.getPID());
+
+    ExecutionBlock terminalBlock;
+    // TODO - consider two terminal types: specified output or not
+    if (childExecBlock.getPlan() != null) {
+      terminalBlock = masterPlan.createTerminalBlock();
+      DataChannel finalChannel = new DataChannel(childExecBlock.getId(), terminalBlock.getId());
+      setFinalOutputChannel(finalChannel, lastNode.getOutSchema());
+      masterPlan.addConnect(finalChannel);
+    } else { // if one or more unions is terminal
+      terminalBlock = childExecBlock;
+      for (DataChannel outputChannel : masterPlan.getIncomingChannels(terminalBlock.getId())) {
+        setFinalOutputChannel(outputChannel, lastNode.getOutSchema());
+      }
+    }
+
+    masterPlan.setTerminal(terminalBlock);
+    LOG.info(masterPlan.toString());
+  }
+
+  private static void setFinalOutputChannel(DataChannel outputChannel, Schema outputSchema) {
+    outputChannel.setShuffleType(NONE_SHUFFLE);
+    outputChannel.setShuffleOutputNum(1);
+    outputChannel.setStoreType(CatalogProtos.StoreType.CSV);
+    outputChannel.setSchema(outputSchema);
+  }
+
+  public static ScanNode buildInputExecutor(LogicalPlan plan, DataChannel channel) {
+    Preconditions.checkArgument(channel.getSchema() != null,
+        "Channel schema (" + channel.getSrcId().getId() + " -> " + channel.getTargetId().getId() +
+            ") is not initialized");
+    TableMeta meta = new TableMeta(channel.getStoreType(), new Options());
+    TableDesc desc = new TableDesc(channel.getSrcId().toString(), channel.getSchema(), meta, new Path("/"));
+    ScanNode scanNode = plan.createNode(ScanNode.class);
+    scanNode.init(desc);
+    return scanNode;
+  }
+
+  private DataChannel createDataChannelFromJoin(ExecutionBlock leftBlock, ExecutionBlock rightBlock,
+                                                ExecutionBlock parent, JoinNode join, boolean leftTable) {
+    ExecutionBlock childBlock = leftTable ? leftBlock : rightBlock;
+
+    DataChannel channel = new DataChannel(childBlock, parent, HASH_SHUFFLE, 32);
+    channel.setStoreType(storeType);
+    if (join.getJoinType() != JoinType.CROSS) {
+      Column [][] joinColumns = PlannerUtil.joinJoinKeyForEachTable(join.getJoinQual(),
+          leftBlock.getPlan().getOutSchema(), rightBlock.getPlan().getOutSchema());
+      if (leftTable) {
+        channel.setShuffleKeys(joinColumns[0]);
+      } else {
+        channel.setShuffleKeys(joinColumns[1]);
+      }
+    }
+    return channel;
+  }
+
+  /**
+   * It calculates the total volume of all descendent relation nodes.
+   */
+  public static long computeDescendentVolume(LogicalNode node) throws PlanningException {
+
+    if (node instanceof RelationNode) {
+      switch (node.getType()) {
+      case SCAN:
+      case PARTITIONS_SCAN:
+        ScanNode scanNode = (ScanNode) node;
+        if (scanNode.getTableDesc().getStats() == null) {
+          // TODO - this case means that data is not located in HDFS. So, we need additional
+          // broadcast method.
+          return Long.MAX_VALUE;
+        } else {
+          return scanNode.getTableDesc().getStats().getNumBytes();
+        }
+      case TABLE_SUBQUERY:
+        return computeDescendentVolume(((TableSubQueryNode) node).getSubQuery());
+      default:
+        throw new IllegalArgumentException("Not RelationNode");
+      }
+    } else if (node instanceof UnaryNode) {
+      return computeDescendentVolume(((UnaryNode) node).getChild());
+    } else if (node instanceof BinaryNode) {
+      BinaryNode binaryNode = (BinaryNode) node;
+      return computeDescendentVolume(binaryNode.getLeftChild()) + computeDescendentVolume(binaryNode.getRightChild());
+    }
+
+    throw new PlanningException("Invalid State");
+  }
+
+  private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNode,
+                                        ExecutionBlock leftBlock, ExecutionBlock rightBlock)
+      throws PlanningException {
+    MasterPlan masterPlan = context.plan;
+    ExecutionBlock currentBlock = null;
+
+    boolean autoBroadcast = conf.getBoolVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO);
+
+    if (autoBroadcast && joinNode.isCandidateBroadcast()) {
+      long broadcastThreshold = conf.getLongVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD);
+      List<LogicalNode> broadtargetTables = new ArrayList<LogicalNode>();
+      int numLargeTables = 0;
+      for(LogicalNode eachNode: joinNode.getBroadcastTargets()) {
+        ScanNode scanNode = (ScanNode)eachNode;
+        TableDesc tableDesc = scanNode.getTableDesc();
+        if (tableDesc.getStats().getNumBytes() < broadcastThreshold) {
+          broadtargetTables.add(scanNode);
+          LOG.info("The table " + scanNode.getCanonicalName() + " ("
+              + scanNode.getTableDesc().getStats().getNumBytes() + ") is marked a broadcasted table");
+        } else {
+          numLargeTables++;
+        }
+      }
+
+      //large table must be one
+      if (numLargeTables <= 1 && !broadtargetTables.isEmpty()) {
+        currentBlock = masterPlan.newExecutionBlock();
+        currentBlock.setPlan(joinNode);
+
+        for (LogicalNode eachBroadcastTargetNode: broadtargetTables) {
+          currentBlock.addBroadcastTable(((ScanNode)eachBroadcastTargetNode).getCanonicalName());
+        }
+
+        for (LogicalNode eachNode: joinNode.getBroadcastTargets()) {
+          context.execBlockMap.remove(eachNode.getPID());
+        }
+        return currentBlock;
+      }
+    }
+
+    // symmetric repartition join
+    currentBlock = masterPlan.newExecutionBlock();
+
+    DataChannel leftChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, true);
+    DataChannel rightChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, false);
+
+    ScanNode leftScan = buildInputExecutor(masterPlan.getLogicalPlan(), leftChannel);
+    ScanNode rightScan = buildInputExecutor(masterPlan.getLogicalPlan(), rightChannel);
+
+    joinNode.setLeftChild(leftScan);
+    joinNode.setRightChild(rightScan);
+    currentBlock.setPlan(joinNode);
+
+    masterPlan.addConnect(leftChannel);
+    masterPlan.addConnect(rightChannel);
+
+    return currentBlock;
+  }
+
+  private AggregationFunctionCallEval createSumFunction(EvalNode [] args) throws InternalException {
+    FunctionDesc functionDesc = getCatalog().getFunction("sum", CatalogProtos.FunctionType.AGGREGATION,
+        args[0].getValueType());
+    return new AggregationFunctionCallEval(functionDesc, (AggFunction) functionDesc.newInstance(), args);
+  }
+
+  private AggregationFunctionCallEval createCountFunction(EvalNode [] args) throws InternalException {
+    FunctionDesc functionDesc = getCatalog().getFunction("count", CatalogProtos.FunctionType.AGGREGATION,
+        args[0].getValueType());
+    return new AggregationFunctionCallEval(functionDesc, (AggFunction) functionDesc.newInstance(), args);
+  }
+
+  private AggregationFunctionCallEval createCountRowFunction(EvalNode[] args) throws InternalException {
+    FunctionDesc functionDesc = getCatalog().getFunction("count", CatalogProtos.FunctionType.AGGREGATION,
+        new TajoDataTypes.DataType[]{});
+    return new AggregationFunctionCallEval(functionDesc, (AggFunction) functionDesc.newInstance(), args);
+  }
+
+  private AggregationFunctionCallEval createMaxFunction(EvalNode [] args) throws InternalException {
+    FunctionDesc functionDesc = getCatalog().getFunction("max", CatalogProtos.FunctionType.AGGREGATION,
+        args[0].getValueType());
+    return new AggregationFunctionCallEval(functionDesc, (AggFunction) functionDesc.newInstance(), args);
+  }
+
+  private AggregationFunctionCallEval createMinFunction(EvalNode [] args) throws InternalException {
+    FunctionDesc functionDesc = getCatalog().getFunction("min", CatalogProtos.FunctionType.AGGREGATION,
+        args[0].getValueType());
+    return new AggregationFunctionCallEval(functionDesc, (AggFunction) functionDesc.newInstance(), args);
+  }
+
+  /**
+   * It contains transformed functions and it related data.
+   * Each non-distinct function is transformed into two functions for both first and second stages.
+   */
+  private static class RewrittenFunctions {
+    AggregationFunctionCallEval [] firstStageEvals;
+    Target [] firstStageTargets;
+    AggregationFunctionCallEval secondStageEvals;
+
+    public RewrittenFunctions(int firstStageEvalNum) {
+      firstStageEvals = new AggregationFunctionCallEval[firstStageEvalNum];
+      firstStageTargets = new Target[firstStageEvalNum];
+    }
+  }
+
+  /**
+   * Tajo uses three execution blocks for an aggregation operator including distinct aggregations.
+   * We call this approach <i><b>three-phase aggregation</b></i>.
+   *
+   * In this case, non-distinct set functions (i.e., <code>count(1), sum(col1)</code>) should be rewritten
+   * to other forms. Please see the following example. This is a rewriting case for a query which includes distinct
+   * aggregation functions. In this example, <code>count(*)</code> functions are transformed into two
+   * functions: count(*) in the inner query and sum() in the outer query.
+   *
+   * <h2>Original query</h2>
+   * <pre>
+   * SELECT
+   *   grp1, grp2, count(*) as total, count(distinct grp3) as distinct_col
+   * from
+   *   rel1
+   * group by
+   *   grp1, grp2;
+   * </pre>
+   *
+   * <h2>Rewritten query</h2>
+   * <pre>
+   * SELECT grp1, grp2, sum(cnt) as total, count(grp3) as distinct_col from (
+   *   SELECT
+   *     grp1, grp2, grp3, count(*) as cnt
+   *   from
+   *     rel1
+   *   group by
+   *     grp1, grp2, grp3) tmp1
+   * group by
+   *   grp1, grp2
+   * ) table1;
+   * </pre>
+   *
+   * The main objective of this method is to transform non-distinct aggregation functions for three-phase aggregation.
+   */
+  private RewrittenFunctions rewriteAggFunctionsForDistinctAggregation(GlobalPlanContext context,
+                                                                       AggregationFunctionCallEval function)
+      throws PlanningException {
+
+    LogicalPlan plan = context.plan.getLogicalPlan();
+    RewrittenFunctions rewritten = null;
+
+    try {
+      if (function.getName().equalsIgnoreCase("count")) {
+        rewritten = new RewrittenFunctions(1);
+
+        if (function.getArgs().length == 0) {
+          rewritten.firstStageEvals[0] = createCountRowFunction(function.getArgs());
+        } else {
+          rewritten.firstStageEvals[0] = createCountFunction(function.getArgs());
+        }
+        String referenceName = plan.generateUniqueColumnName(rewritten.firstStageEvals[0]);
+        FieldEval fieldEval = new FieldEval(referenceName, rewritten.firstStageEvals[0].getValueType());
+        rewritten.firstStageTargets[0] = new Target(fieldEval);
+        rewritten.secondStageEvals = createSumFunction(new EvalNode[] {fieldEval});
+      } else if (function.getName().equalsIgnoreCase("sum")) {
+        rewritten = new RewrittenFunctions(1);
+
+        rewritten.firstStageEvals[0] = createSumFunction(function.getArgs());
+        String referenceName = plan.generateUniqueColumnName(rewritten.firstStageEvals[0]);
+        FieldEval fieldEval = new FieldEval(referenceName, rewritten.firstStageEvals[0].getValueType());
+        rewritten.firstStageTargets[0] = new Target(fieldEval);
+        rewritten.secondStageEvals = createSumFunction(new EvalNode[] {fieldEval});
+
+      } else if (function.getName().equals("max")) {
+        rewritten = new RewrittenFunctions(1);
+
+        rewritten.firstStageEvals[0] = createMaxFunction(function.getArgs());
+        String referenceName = plan.generateUniqueColumnName(rewritten.firstStageEvals[0]);
+        FieldEval fieldEval = new FieldEval(referenceName, rewritten.firstStageEvals[0].getValueType());
+        rewritten.firstStageTargets[0] = new Target(fieldEval);
+        rewritten.secondStageEvals = createMaxFunction(new EvalNode[]{fieldEval});
+
+      } else if (function.getName().equals("min")) {
+
+        rewritten = new RewrittenFunctions(1);
+
+        rewritten.firstStageEvals[0] = createMinFunction(function.getArgs());
+        String referenceName = plan.generateUniqueColumnName(rewritten.firstStageEvals[0]);
+        FieldEval fieldEval = new FieldEval(referenceName, rewritten.firstStageEvals[0].getValueType());
+        rewritten.firstStageTargets[0] = new Target(fieldEval);
+        rewritten.secondStageEvals = createMinFunction(new EvalNode[]{fieldEval});
+
+      } else {
+        throw new PlanningException("Cannot support a mix of other functions");
+      }
+    } catch (InternalException e) {
+      LOG.error(e);
+    }
+
+    return rewritten;
+  }
+
+  /**
+   * If there are at least one distinct aggregation function, a query works as if the query is rewritten as follows:
+   *
+   * <h2>Original query</h2>
+   * <pre>
+   * SELECT
+   *   grp1, grp2, count(*) as total, count(distinct grp3) as distinct_col
+   * from
+   *   rel1
+   * group by
+   *   grp1, grp2;
+   * </pre>
+   *
+   * The query will work as if the query is rewritten into two queries as follows:
+   *
+   * <h2>Rewritten query</h2>
+   * <pre>
+   * SELECT grp1, grp2, sum(cnt) as total, count(grp3) as distinct_col from (
+   *   SELECT
+   *     grp1, grp2, grp3, count(*) as cnt
+   *   from
+   *     rel1
+   *   group by
+   *     grp1, grp2, grp3) tmp1
+   * group by
+   *   grp1, grp2
+   * ) table1;
+   * </pre>
+   *
+   * In more detail, the first aggregation aggregates not only original grouping fields but also distinct columns.
+   * Non-distinct aggregation functions should be transformed to proper functions.
+   * Then, the second aggregation aggregates only original grouping fields with distinct aggregation functions and
+   * transformed non-distinct aggregation functions.
+   *
+   * As a result, although a no-distinct aggregation requires two stages, a distinct aggregation requires three
+   * execution blocks.
+   */
+  private ExecutionBlock buildGroupByIncludingDistinctFunctions(GlobalPlanContext context,
+                                                                ExecutionBlock latestExecBlock,
+                                                                GroupbyNode groupbyNode) throws PlanningException {
+
+    Column [] originalGroupingColumns = groupbyNode.getGroupingColumns();
+    LinkedHashSet<Column> firstStageGroupingColumns =
+        Sets.newLinkedHashSet(Arrays.asList(groupbyNode.getGroupingColumns()));
+    List<AggregationFunctionCallEval> firstStageAggFunctions = Lists.newArrayList();
+    List<AggregationFunctionCallEval> secondPhaseEvalNodes = Lists.newArrayList();
+    List<Target> firstPhaseEvalNodeTargets = Lists.newArrayList();
+
+    for (AggregationFunctionCallEval aggFunction : groupbyNode.getAggFunctions()) {
+
+      if (aggFunction.isDistinct()) {
+        // add distinct columns to first stage's grouping columns
+        firstStageGroupingColumns.addAll(EvalTreeUtil.findUniqueColumns(aggFunction));
+        // keep distinct aggregation functions for the second stage
+        secondPhaseEvalNodes.add(aggFunction);
+
+      } else {
+        // Rewrite non-distinct aggregation functions
+        RewrittenFunctions rewritten = rewriteAggFunctionsForDistinctAggregation(context, aggFunction);
+        firstStageAggFunctions.addAll(Lists.newArrayList(rewritten.firstStageEvals));
+        firstPhaseEvalNodeTargets.addAll(Lists.newArrayList(rewritten.firstStageTargets));
+
+        // keep rewritten non-aggregation functions for the second stage
+        secondPhaseEvalNodes.add(rewritten.secondStageEvals);
+      }
+    }
+
+    int firstStageAggFunctionNum = firstStageAggFunctions.size();
+    int firstStageGroupingKeyNum = firstStageGroupingColumns.size();
+
+    int i = 0;
+    Target [] firstStageTargets = new Target[firstStageGroupingKeyNum + firstStageAggFunctionNum];
+    for (Column column : firstStageGroupingColumns) {
+      Target target = new Target(new FieldEval(column));
+      firstStageTargets[i++] = target;
+    }
+    for (Target target : firstPhaseEvalNodeTargets) {
+      firstStageTargets[i++] = target;
+    }
+    // Create the groupby node for the first stage and set all necessary descriptions
+    GroupbyNode firstStageGroupby = new GroupbyNode(context.plan.getLogicalPlan().newPID());
+    firstStageGroupby.setGroupingColumns(TUtil.toArray(firstStageGroupingColumns, Column.class));
+    firstStageGroupby.setAggFunctions(TUtil.toArray(firstStageAggFunctions, AggregationFunctionCallEval.class));
+    firstStageGroupby.setTargets(firstStageTargets);
+    firstStageGroupby.setChild(groupbyNode.getChild());
+    firstStageGroupby.setInSchema(groupbyNode.getInSchema());
+
+    // Makes two execution blocks for the first stage
+    ExecutionBlock firstStage = buildGroupBy(context, latestExecBlock, firstStageGroupby);
+
+    // Create the groupby node for the second stage.
+    GroupbyNode secondPhaseGroupby = new GroupbyNode(context.plan.getLogicalPlan().newPID());
+    secondPhaseGroupby.setGroupingColumns(originalGroupingColumns);
+    secondPhaseGroupby.setAggFunctions(TUtil.toArray(secondPhaseEvalNodes, AggregationFunctionCallEval.class));
+    secondPhaseGroupby.setTargets(groupbyNode.getTargets());
+
+    ExecutionBlock secondStage = context.plan.newExecutionBlock();
+    secondStage.setPlan(secondPhaseGroupby);
+    SortSpec [] sortSpecs = PlannerUtil.columnsToSortSpec(firstStageGroupingColumns);
+    secondStage.getEnforcer().enforceSortAggregation(secondPhaseGroupby.getPID(), sortSpecs);
+
+    // Create a data channel between the first and second stages
+    DataChannel channel;
+    channel = new DataChannel(firstStage, secondStage, HASH_SHUFFLE, 32);
+    channel.setShuffleKeys(secondPhaseGroupby.getGroupingColumns().clone());
+    channel.setSchema(firstStage.getPlan().getOutSchema());
+    channel.setStoreType(storeType);
+
+    // Setting for the second phase's logical plan
+    ScanNode scanNode = buildInputExecutor(context.plan.getLogicalPlan(), channel);
+    secondPhaseGroupby.setChild(scanNode);
+    secondPhaseGroupby.setInSchema(scanNode.getOutSchema());
+    secondStage.setPlan(secondPhaseGroupby);
+
+    context.plan.addConnect(channel);
+
+    return secondStage;
+  }
+
+  private ExecutionBlock buildGroupBy(GlobalPlanContext context, ExecutionBlock lastBlock,
+                                      GroupbyNode groupbyNode) throws PlanningException {
+
+    MasterPlan masterPlan = context.plan;
+    ExecutionBlock currentBlock;
+
+    if (groupbyNode.isDistinct()) { // if there is at one distinct aggregation function
+      return buildGroupByIncludingDistinctFunctions(context, lastBlock, groupbyNode);
+    } else {
+      GroupbyNode firstPhaseGroupby = createFirstPhaseGroupBy(masterPlan.getLogicalPlan(), groupbyNode);
+
+      if (hasUnionChild(firstPhaseGroupby)) {
+        currentBlock = buildGroupbyAndUnionPlan(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode);
+      } else {
+        // general hash-shuffled aggregation
+        currentBlock = buildTwoPhaseGroupby(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode);
+      }
+    }
+
+    return currentBlock;
+  }
+
+  public boolean hasUnionChild(UnaryNode node) {
+
+    // there are two cases:
+    //
+    // The first case is:
+    //
+    //  create table [tbname] as select * from ( select ... UNION select ...) T
+    //
+    // We can generalize this case as 'a store operator on the top of union'.
+    // In this case, a store operator determines a shuffle method.
+    //
+    // The second case is:
+    //
+    // select avg(..) from (select ... UNION select ) T
+    //
+    // We can generalize this case as 'a shuffle required operator on the top of union'.
+
+    if (node.getChild() instanceof UnaryNode) { // first case
+      UnaryNode child = node.getChild();
+
+      if (child.getChild().getType() == NodeType.PROJECTION) {
+        child = child.getChild();
+      }
+
+      if (child.getChild().getType() == NodeType.TABLE_SUBQUERY) {
+        TableSubQueryNode tableSubQuery = child.getChild();
+        return tableSubQuery.getSubQuery().getType() == NodeType.UNION;
+      }
+
+    } else if (node.getChild().getType() == NodeType.TABLE_SUBQUERY) { // second case
+      TableSubQueryNode tableSubQuery = node.getChild();
+      return tableSubQuery.getSubQuery().getType() == NodeType.UNION;
+    }
+
+    return false;
+  }
+
+  private ExecutionBlock buildGroupbyAndUnionPlan(MasterPlan masterPlan, ExecutionBlock lastBlock,
+                                                  GroupbyNode firstPhaseGroupBy, GroupbyNode secondPhaseGroupBy) {
+    DataChannel lastDataChannel = null;
+
+    // It pushes down the first phase group-by operator into all child blocks.
+    //
+    // (second phase)    G (currentBlock)
+    //                  /|\
+    //                / / | \
+    // (first phase) G G  G  G (child block)
+
+    // They are already connected one another.
+    // So, we don't need to connect them again.
+    for (DataChannel dataChannel : masterPlan.getIncomingChannels(lastBlock.getId())) {
+      if (firstPhaseGroupBy.isEmptyGrouping()) {
+        dataChannel.setShuffle(HASH_SHUFFLE, firstPhaseGroupBy.getGroupingColumns(), 1);
+      } else {
+        dataChannel.setShuffle(HASH_SHUFFLE, firstPhaseGroupBy.getGroupingColumns(), 32);
+      }
+      dataChannel.setSchema(firstPhaseGroupBy.getOutSchema());
+      ExecutionBlock childBlock = masterPlan.getExecBlock(dataChannel.getSrcId());
+
+      // Why must firstPhaseGroupby be copied?
+      //
+      // A groupby in each execution block can have different child.
+      // It affects groupby's input schema.
+      GroupbyNode firstPhaseGroupbyCopy = PlannerUtil.clone(masterPlan.getLogicalPlan(), firstPhaseGroupBy);
+      firstPhaseGroupbyCopy.setChild(childBlock.getPlan());
+      childBlock.setPlan(firstPhaseGroupbyCopy);
+
+      // just keep the last data channel.
+      lastDataChannel = dataChannel;
+    }
+
+    ScanNode scanNode = buildInputExecutor(masterPlan.getLogicalPlan(), lastDataChannel);
+    secondPhaseGroupBy.setChild(scanNode);
+    lastBlock.setPlan(secondPhaseGroupBy);
+    return lastBlock;
+  }
+
+  private ExecutionBlock buildTwoPhaseGroupby(MasterPlan masterPlan, ExecutionBlock latestBlock,
+                                                     GroupbyNode firstPhaseGroupby, GroupbyNode secondPhaseGroupby) {
+    ExecutionBlock childBlock = latestBlock;
+    childBlock.setPlan(firstPhaseGroupby);
+    ExecutionBlock currentBlock = masterPlan.newExecutionBlock();
+
+    DataChannel channel;
+    if (firstPhaseGroupby.isEmptyGrouping()) {
+      channel = new DataChannel(childBlock, currentBlock, HASH_SHUFFLE, 1);
+      channel.setShuffleKeys(firstPhaseGroupby.getGroupingColumns());
+    } else {
+      channel = new DataChannel(childBlock, currentBlock, HASH_SHUFFLE, 32);
+      channel.setShuffleKeys(firstPhaseGroupby.getGroupingColumns());
+    }
+    channel.setSchema(firstPhaseGroupby.getOutSchema());
+    channel.setStoreType(storeType);
+
+    ScanNode scanNode = buildInputExecutor(masterPlan.getLogicalPlan(), channel);
+    secondPhaseGroupby.setChild(scanNode);
+    secondPhaseGroupby.setInSchema(scanNode.getOutSchema());
+    currentBlock.setPlan(secondPhaseGroupby);
+
+    masterPlan.addConnect(channel);
+
+    return currentBlock;
+  }
+
+  public static GroupbyNode createFirstPhaseGroupBy(LogicalPlan plan, GroupbyNode groupBy) {
+    Preconditions.checkNotNull(groupBy);
+
+    GroupbyNode firstPhaseGroupBy = PlannerUtil.clone(plan, groupBy);
+    GroupbyNode secondPhaseGroupBy = groupBy;
+
+    // Set first phase expressions
+    if (secondPhaseGroupBy.hasAggFunctions()) {
+      int evalNum = secondPhaseGroupBy.getAggFunctions().length;
+      AggregationFunctionCallEval [] secondPhaseEvals = secondPhaseGroupBy.getAggFunctions();
+      AggregationFunctionCallEval [] firstPhaseEvals = new AggregationFunctionCallEval[evalNum];
+
+      String [] firstPhaseEvalNames = new String[evalNum];
+      for (int i = 0; i < evalNum; i++) {
+        try {
+          firstPhaseEvals[i] = (AggregationFunctionCallEval) secondPhaseEvals[i].clone();
+        } catch (CloneNotSupportedException e) {
+          throw new RuntimeException(e);
+        }
+
+        firstPhaseEvals[i].setFirstPhase();
+        firstPhaseEvalNames[i] = plan.generateUniqueColumnName(firstPhaseEvals[i]);
+        FieldEval param = new FieldEval(firstPhaseEvalNames[i], firstPhaseEvals[i].getValueType());
+        secondPhaseEvals[i].setArgs(new EvalNode[] {param});
+      }
+
+      secondPhaseGroupBy.setAggFunctions(secondPhaseEvals);
+      firstPhaseGroupBy.setAggFunctions(firstPhaseEvals);
+      Target [] firstPhaseTargets = ProjectionPushDownRule.buildGroupByTarget(firstPhaseGroupBy, null,
+          firstPhaseEvalNames);
+      firstPhaseGroupBy.setTargets(firstPhaseTargets);
+      secondPhaseGroupBy.setInSchema(PlannerUtil.targetToSchema(firstPhaseTargets));
+    }
+    return firstPhaseGroupBy;
+  }
+
+  private ExecutionBlock buildSortPlan(GlobalPlanContext context, ExecutionBlock childBlock, SortNode currentNode) {
+    MasterPlan masterPlan = context.plan;
+    ExecutionBlock currentBlock;
+
+    SortNode firstSortNode = PlannerUtil.clone(context.plan.getLogicalPlan(), currentNode);
+
+    if (firstSortNode.getChild().getType() == NodeType.TABLE_SUBQUERY &&
+        ((TableSubQueryNode)firstSortNode.getChild()).getSubQuery().getType() == NodeType.UNION) {
+
+      currentBlock = childBlock;
+      for (DataChannel channel : masterPlan.getIncomingChannels(childBlock.getId())) {
+        channel.setShuffle(RANGE_SHUFFLE, PlannerUtil.sortSpecsToSchema(currentNode.getSortKeys()).toArray(), 32);
+        channel.setSchema(firstSortNode.getOutSchema());
+
+        ExecutionBlock subBlock = masterPlan.getExecBlock(channel.getSrcId());
+        SortNode s1 = PlannerUtil.clone(context.plan.getLogicalPlan(), firstSortNode);
+        s1.setChild(subBlock.getPlan());
+        subBlock.setPlan(s1);
+
+        ScanNode secondScan = buildInputExecutor(masterPlan.getLogicalPlan(), channel);
+        currentNode.setChild(secondScan);
+        currentNode.setInSchema(secondScan.getOutSchema());
+        currentBlock.setPlan(currentNode);
+        currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys());
+      }
+    } else {
+      LogicalNode childBlockPlan = childBlock.getPlan();
+      firstSortNode.setChild(childBlockPlan);
+      // sort is a non-projectable operator. So, in/out schemas are the same to its child operator.
+      firstSortNode.setInSchema(childBlockPlan.getOutSchema());
+      firstSortNode.setOutSchema(childBlockPlan.getOutSchema());
+      childBlock.setPlan(firstSortNode);
+
+      currentBlock = masterPlan.newExecutionBlock();
+      DataChannel channel = new DataChannel(childBlock, currentBlock, RANGE_SHUFFLE, 32);
+      channel.setShuffleKeys(PlannerUtil.sortSpecsToSchema(currentNode.getSortKeys()).toArray());
+      channel.setSchema(firstSortNode.getOutSchema());
+
+      ScanNode secondScan = buildInputExecutor(masterPlan.getLogicalPlan(), channel);
+      currentNode.setChild(secondScan);
+      currentNode.setInSchema(secondScan.getOutSchema());
+      currentBlock.setPlan(currentNode);
+      currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys());
+      masterPlan.addConnect(channel);
+    }
+
+    return currentBlock;
+  }
+
+  /**
+   * It builds a distributed execution block for CTAS, InsertNode, and StoreTableNode.
+   */
+  private ExecutionBlock buildStorePlan(GlobalPlanContext context,
+                                        ExecutionBlock lastBlock,
+                                        StoreTableNode currentNode) throws PlanningException {
+
+
+    if(currentNode.hasPartition()) { // if a target table is a partitioned table
+
+      // Verify supported partition types
+      PartitionMethodDesc partitionMethod = currentNode.getPartitionMethod();
+      if (partitionMethod.getPartitionType() != CatalogProtos.PartitionType.COLUMN) {
+        throw new PlanningException(String.format("Not supported partitionsType :%s",
+            partitionMethod.getPartitionType()));
+      }
+
+      if (hasUnionChild(currentNode)) { // if it has union children
+        return buildShuffleAndStorePlanToPartitionedTableWithUnion(context, currentNode, lastBlock);
+      } else { // otherwise
+        return buildShuffleAndStorePlanToPartitionedTable(context, currentNode, lastBlock);
+      }
+    } else { // if result table is not a partitioned table, directly store it
+      return buildNoPartitionedStorePlan(context, currentNode, lastBlock);
+    }
+  }
+
+  /**
+   * It makes a plan to store directly union plans into a non-partitioned table.
+   */
+  private ExecutionBlock buildShuffleAndStorePlanNoPartitionedTableWithUnion(GlobalPlanContext context,
+                                                                             StoreTableNode currentNode,
+                                                                             ExecutionBlock childBlock) {
+    for (ExecutionBlock grandChildBlock : context.plan.getChilds(childBlock)) {
+      StoreTableNode copy = PlannerUtil.clone(context.plan.getLogicalPlan(), currentNode);
+      copy.setChild(grandChildBlock.getPlan());
+      grandChildBlock.setPlan(copy);
+    }
+    return childBlock;
+  }
+
+  /**
+   * It inserts shuffle and adds store plan on a partitioned table,
+   * and it push downs those plans into child unions.
+   */
+  private ExecutionBlock buildShuffleAndStorePlanToPartitionedTableWithUnion(GlobalPlanContext context,
+                                                                             StoreTableNode currentNode,
+                                                                             ExecutionBlock lastBlock)
+      throws PlanningException {
+
+    MasterPlan masterPlan = context.plan;
+    DataChannel lastChannel = null;
+    for (DataChannel channel : masterPlan.getIncomingChannels(lastBlock.getId())) {
+      ExecutionBlock childBlock = masterPlan.getExecBlock(channel.getSrcId());
+      setShuffleKeysFromPartitionedTableStore(currentNode, channel);
+      channel.setSchema(childBlock.getPlan().getOutSchema());
+      channel.setStoreType(storeType);
+      lastChannel = channel;
+    }
+
+    ScanNode scanNode = buildInputExecutor(masterPlan.getLogicalPlan(), lastChannel);
+    currentNode.setChild(scanNode);
+    currentNode.setInSchema(scanNode.getOutSchema());
+    lastBlock.setPlan(currentNode);
+    return lastBlock;
+  }
+
+  /**
+   * It inserts shuffle and adds store plan on a partitioned table.
+   */
+  private ExecutionBlock buildShuffleAndStorePlanToPartitionedTable(GlobalPlanContext context,
+                                                                    StoreTableNode currentNode,
+                                                                    ExecutionBlock lastBlock)
+      throws PlanningException {
+    MasterPlan masterPlan = context.plan;
+
+    ExecutionBlock nextBlock = masterPlan.newExecutionBlock();
+    DataChannel channel = new DataChannel(lastBlock, nextBlock, HASH_SHUFFLE, 32);
+    setShuffleKeysFromPartitionedTableStore(currentNode, channel);
+    channel.setSchema(lastBlock.getPlan().getOutSchema());
+    channel.setStoreType(storeType);
+
+    ScanNode scanNode = buildInputExecutor(masterPlan.getLogicalPlan(), channel);
+    currentNode.setChild(scanNode);
+    currentNode.setInSchema(scanNode.getOutSchema());
+    nextBlock.setPlan(currentNode);
+
+    masterPlan.addConnect(channel);
+
+    return nextBlock;
+  }
+
+  private ExecutionBlock buildNoPartitionedStorePlan(GlobalPlanContext context,
+                                                     StoreTableNode currentNode,
+                                                     ExecutionBlock childBlock) {
+    if (hasUnionChild(currentNode)) { // when the below is union
+      return buildShuffleAndStorePlanNoPartitionedTableWithUnion(context, currentNode, childBlock);
+    } else {
+      currentNode.setChild(childBlock.getPlan());
+      currentNode.setInSchema(childBlock.getPlan().getOutSchema());
+      childBlock.setPlan(currentNode);
+      return childBlock;
+    }
+  }
+
+  private void setShuffleKeysFromPartitionedTableStore(StoreTableNode node, DataChannel channel) {
+    Preconditions.checkState(node.hasTargetTable(), "A target table must be a partitioned table.");
+    PartitionMethodDesc partitionMethod = node.getPartitionMethod();
+
+    if (node.getType() == NodeType.INSERT) {
+      InsertNode insertNode = (InsertNode) node;
+      channel.setSchema(((InsertNode)node).getProjectedSchema());
+      Column [] shuffleKeys = new Column[partitionMethod.getExpressionSchema().size()];
+      int i = 0;
+      for (Column column : partitionMethod.getExpressionSchema().getColumns()) {
+        int id = insertNode.getTableSchema().getColumnId(column.getQualifiedName());
+        shuffleKeys[i++] = insertNode.getProjectedSchema().getColumn(id);
+      }
+      channel.setShuffleKeys(shuffleKeys);
+    } else {
+      channel.setShuffleKeys(partitionMethod.getExpressionSchema().toArray());
+    }
+    channel.setShuffleType(HASH_SHUFFLE);
+    channel.setShuffleOutputNum(32);
+  }
+
+  public class DistributedPlannerVisitor extends BasicLogicalPlanVisitor<GlobalPlanContext, LogicalNode> {
+
+    @Override
+    public LogicalNode visitRoot(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 LogicalRootNode node, Stack<LogicalNode> stack) throws PlanningException {
+      return super.visitRoot(context, plan, block, node, stack);
+    }
+
+    @Override
+    public LogicalNode visitProjection(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                       ProjectionNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitProjection(context, plan, block, node, stack);
+
+      ExecutionBlock execBlock = context.execBlockMap.remove(child.getPID());
+
+      if (child.getType() == NodeType.TABLE_SUBQUERY &&
+          ((TableSubQueryNode)child).getSubQuery().getType() == NodeType.UNION) {
+        MasterPlan masterPlan = context.plan;
+        for (DataChannel dataChannel : masterPlan.getIncomingChannels(execBlock.getId())) {
+          ExecutionBlock subBlock = masterPlan.getExecBlock(dataChannel.getSrcId());
+
+          ProjectionNode copy = PlannerUtil.clone(plan, node);
+          copy.setChild(subBlock.getPlan());
+          subBlock.setPlan(copy);
+        }
+        execBlock.setPlan(null);
+      } else {
+        node.setChild(execBlock.getPlan());
+        node.setInSchema(execBlock.getPlan().getOutSchema());
+        execBlock.setPlan(node);
+      }
+
+      context.execBlockMap.put(node.getPID(), execBlock);
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitLimit(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                  LimitNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitLimit(context, plan, block, node, stack);
+
+      ExecutionBlock execBlock;
+      execBlock = context.execBlockMap.remove(child.getPID());
+      if (child.getType() == NodeType.SORT) {
+        node.setChild(execBlock.getPlan());
+        execBlock.setPlan(node);
+
+        ExecutionBlock childBlock = context.plan.getChild(execBlock, 0);
+        LimitNode childLimit = PlannerUtil.clone(context.plan.getLogicalPlan(), node);
+        childLimit.setChild(childBlock.getPlan());
+        childBlock.setPlan(childLimit);
+
+        DataChannel channel = context.plan.getChannel(childBlock, execBlock);
+        channel.setShuffleOutputNum(1);
+        context.execBlockMap.put(node.getPID(), execBlock);
+      } else {
+        node.setChild(execBlock.getPlan());
+        execBlock.setPlan(node);
+
+        ExecutionBlock newExecBlock = context.plan.newExecutionBlock();
+        DataChannel newChannel = new DataChannel(execBlock, newExecBlock, HASH_SHUFFLE, 1);
+        newChannel.setShuffleKeys(new Column[]{});
+        newChannel.setSchema(node.getOutSchema());
+        newChannel.setStoreType(storeType);
+
+        ScanNode scanNode = buildInputExecutor(plan, newChannel);
+        LimitNode parentLimit = PlannerUtil.clone(context.plan.getLogicalPlan(), node);
+        parentLimit.setChild(scanNode);
+        newExecBlock.setPlan(parentLimit);
+        context.plan.addConnect(newChannel);
+        context.execBlockMap.put(parentLimit.getPID(), newExecBlock);
+        node = parentLimit;
+      }
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitSort(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 SortNode node, Stack<LogicalNode> stack) throws PlanningException {
+
+      LogicalNode child = super.visitSort(context, plan, block, node, stack);
+
+      ExecutionBlock childBlock = context.execBlockMap.remove(child.getPID());
+      ExecutionBlock newExecBlock = buildSortPlan(context, childBlock, node);
+      context.execBlockMap.put(node.getPID(), newExecBlock);
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitHaving(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                    HavingNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitHaving(context, plan, block, node, stack);
+
+      // Don't separate execution block. Having is pushed to the second grouping execution block.
+      ExecutionBlock childBlock = context.execBlockMap.remove(child.getPID());
+      node.setChild(childBlock.getPlan());
+      childBlock.setPlan(node);
+      context.execBlockMap.put(node.getPID(), childBlock);
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitGroupBy(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                    GroupbyNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitGroupBy(context, plan, block, node, stack);
+
+      ExecutionBlock childBlock = context.execBlockMap.remove(child.getPID());
+      ExecutionBlock newExecBlock = buildGroupBy(context, childBlock, node);
+      context.execBlockMap.put(newExecBlock.getPlan().getPID(), newExecBlock);
+
+      return newExecBlock.getPlan();
+    }
+
+    @Override
+    public LogicalNode visitFilter(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                   SelectionNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitFilter(context, plan, block, node, stack);
+
+      ExecutionBlock execBlock = context.execBlockMap.remove(child.getPID());
+      node.setChild(execBlock.getPlan());
+      node.setInSchema(execBlock.getPlan().getOutSchema());
+      execBlock.setPlan(node);
+      context.execBlockMap.put(node.getPID(), execBlock);
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitJoin(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 JoinNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode leftChild = visit(context, plan, block, node.getLeftChild(), stack);
+      LogicalNode rightChild = visit(context, plan, block, node.getRightChild(), stack);
+
+      ExecutionBlock leftChildBlock = context.execBlockMap.get(leftChild.getPID());
+      ExecutionBlock rightChildBlock = context.execBlockMap.get(rightChild.getPID());
+
+      ExecutionBlock newExecBlock = buildJoinPlan(context, node, leftChildBlock, rightChildBlock);
+      context.execBlockMap.put(node.getPID(), newExecBlock);
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitUnion(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                  UnionNode node, Stack<LogicalNode> stack) throws PlanningException {
+      stack.push(node);
+      LogicalPlan.QueryBlock leftQueryBlock = plan.getBlock(node.getLeftChild());
+      LogicalNode leftChild = visit(context, plan, leftQueryBlock, leftQueryBlock.getRoot(), stack);
+
+      LogicalPlan.QueryBlock rightQueryBlock = plan.getBlock(node.getRightChild());
+      LogicalNode rightChild = visit(context, plan, rightQueryBlock, rightQueryBlock.getRoot(), stack);
+      stack.pop();
+
+      List<ExecutionBlock> unionBlocks = Lists.newArrayList();
+      List<ExecutionBlock> queryBlockBlocks = Lists.newArrayList();
+
+      ExecutionBlock leftBlock = context.execBlockMap.remove(leftChild.getPID());
+      ExecutionBlock rightBlock = context.execBlockMap.remove(rightChild.getPID());
+      if (leftChild.getType() == NodeType.UNION) {
+        unionBlocks.add(leftBlock);
+      } else {
+        queryBlockBlocks.add(leftBlock);
+      }
+      if (rightChild.getType() == NodeType.UNION) {
+        unionBlocks.add(rightBlock);
+      } else {
+        queryBlockBlocks.add(rightBlock);
+      }
+
+      ExecutionBlock execBlock;
+      if (unionBlocks.size() == 0) {
+        execBlock = context.plan.newExecutionBlock();
+      } else {
+        execBlock = unionBlocks.get(0);
+      }
+
+      for (ExecutionBlock childBlocks : unionBlocks) {
+        for (ExecutionBlock grandChildBlock : context.plan.getChilds(childBlocks)) {
+          queryBlockBlocks.add(grandChildBlock);
+        }
+      }
+
+      for (ExecutionBlock childBlocks : queryBlockBlocks) {
+        DataChannel channel = new DataChannel(childBlocks, execBlock, NONE_SHUFFLE, 1);
+        channel.setStoreType(storeType);
+        context.plan.addConnect(channel);
+      }
+
+      context.execBlockMap.put(node.getPID(), execBlock);
+
+      return node;
+    }
+
+    private LogicalNode handleUnaryNode(GlobalPlanContext context, LogicalNode child, LogicalNode node) {
+      ExecutionBlock execBlock = context.execBlockMap.remove(child.getPID());
+      execBlock.setPlan(node);
+      context.execBlockMap.put(node.getPID(), execBlock);
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitExcept(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                   ExceptNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitExcept(context, plan, queryBlock, node, stack);
+      return handleUnaryNode(context, child, node);
+    }
+
+    @Override
+    public LogicalNode visitIntersect(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                      IntersectNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitIntersect(context, plan, queryBlock, node, stack);
+      return handleUnaryNode(context, child, node);
+    }
+
+    @Override
+    public LogicalNode visitTableSubQuery(GlobalPlanContext context, LogicalPlan plan,
+                                          LogicalPlan.QueryBlock queryBlock,
+                                          TableSubQueryNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitTableSubQuery(context, plan, queryBlock, node, stack);
+      node.setSubQuery(child);
+
+      ExecutionBlock currentBlock = context.execBlockMap.remove(child.getPID());
+
+      if (child.getType() == NodeType.UNION) {
+        for (ExecutionBlock childBlock : context.plan.getChilds(currentBlock.getId())) {
+          TableSubQueryNode copy = PlannerUtil.clone(plan, node);
+          copy.setSubQuery(childBlock.getPlan());
+          childBlock.setPlan(copy);
+        }
+      } else {
+        currentBlock.setPlan(node);
+      }
+      context.execBlockMap.put(node.getPID(), currentBlock);
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitScan(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                 ScanNode node, Stack<LogicalNode> stack) throws PlanningException {
+      ExecutionBlock newExecBlock = context.plan.newExecutionBlock();
+      newExecBlock.setPlan(node);
+      context.execBlockMap.put(node.getPID(), newExecBlock);
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitPartitionedTableScan(GlobalPlanContext context, LogicalPlan plan,
+                                                 LogicalPlan.QueryBlock block, PartitionedTableScanNode node,
+                                                 Stack<LogicalNode> stack)throws PlanningException {
+      ExecutionBlock newExecBlock = context.plan.newExecutionBlock();
+      newExecBlock.setPlan(node);
+      context.execBlockMap.put(node.getPID(), newExecBlock);
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitStoreTable(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                       StoreTableNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitStoreTable(context, plan, queryBlock, node, stack);
+
+      ExecutionBlock childBlock = context.execBlockMap.remove(child.getPID());
+      ExecutionBlock newExecBlock = buildStorePlan(context, childBlock, node);
+      context.execBlockMap.put(node.getPID(), newExecBlock);
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitCreateTable(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                       CreateTableNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode child = super.visitStoreTable(context, plan, queryBlock, node, stack);
+
+      ExecutionBlock childBlock = context.execBlockMap.remove(child.getPID());
+      ExecutionBlock newExecBlock = buildStorePlan(context, childBlock, node);
+      context.execBlockMap.put(node.getPID(), newExecBlock);
+
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitInsert(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                   InsertNode node, Stack<LogicalNode> stack)
+        throws PlanningException {
+      LogicalNode child = super.visitInsert(context, plan, queryBlock, node, stack);
+
+      ExecutionBlock childBlock = context.execBlockMap.remove(child.getPID());
+      ExecutionBlock newExecBlock = buildStorePlan(context, childBlock, node);
+      context.execBlockMap.put(node.getPID(), newExecBlock);
+
+      return node;
+    }
+  }
+
+  @SuppressWarnings("unused")
+  private class ConsecutiveUnionFinder extends BasicLogicalPlanVisitor<List<UnionNode>, LogicalNode> {
+    @Override
+    public LogicalNode visitUnion(List<UnionNode> unionNodeList, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock,
+                                  UnionNode node, Stack<LogicalNode> stack)
+        throws PlanningException {
+      if (node.getType() == NodeType.UNION) {
+        unionNodeList.add(node);
+      }
+
+      stack.push(node);
+      TableSubQueryNode leftSubQuery = node.getLeftChild();
+      TableSubQueryNode rightSubQuery = node.getRightChild();
+      if (leftSubQuery.getSubQuery().getType() == NodeType.UNION) {
+        visit(unionNodeList, plan, queryBlock, leftSubQuery, stack);
+      }
+      if (rightSubQuery.getSubQuery().getType() == NodeType.UNION) {
+        visit(unionNodeList, plan, queryBlock, rightSubQuery, stack);
+      }
+      stack.pop();
+
+      return node;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java
new file mode 100644
index 0000000..37b0db1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java
@@ -0,0 +1,268 @@
+/**
+ * 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.planner.global;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class MasterPlan {
+  private final QueryId queryId;
+  private final QueryContext context;
+  private final LogicalPlan plan;
+  private ExecutionBlock root;
+  private AtomicInteger nextId = new AtomicInteger(0);
+
+  private ExecutionBlock terminalBlock;
+  private Map<ExecutionBlockId, ExecutionBlock> execBlockMap = new HashMap<ExecutionBlockId, ExecutionBlock>();
+  private SimpleDirectedGraph<ExecutionBlockId, DataChannel> execBlockGraph =
+      new SimpleDirectedGraph<ExecutionBlockId, DataChannel>();
+
+  public ExecutionBlockId newExecutionBlockId() {
+    return new ExecutionBlockId(queryId, nextId.incrementAndGet());
+  }
+
+  public boolean isTerminal(ExecutionBlock execBlock) {
+    return terminalBlock.getId().equals(execBlock.getId());
+  }
+
+  public ExecutionBlock getTerminalBlock() {
+    return terminalBlock;
+  }
+
+  public ExecutionBlock createTerminalBlock() {
+    terminalBlock = newExecutionBlock();
+    return terminalBlock;
+  }
+
+  public MasterPlan(QueryId queryId, QueryContext context, LogicalPlan plan) {
+    this.queryId = queryId;
+    this.context = context;
+    this.plan = plan;
+  }
+
+  public QueryId getQueryId() {
+    return this.queryId;
+  }
+
+  public QueryContext getContext() {
+    return this.context;
+  }
+
+  public LogicalPlan getLogicalPlan() {
+    return this.plan;
+  }
+  
+  public void setTerminal(ExecutionBlock root) {
+    this.root = root;
+    this.terminalBlock = root;
+  }
+  
+  public ExecutionBlock getRoot() {
+    return this.root;
+  }
+
+  public ExecutionBlock newExecutionBlock() {
+    ExecutionBlock newExecBlock = new ExecutionBlock(newExecutionBlockId());
+    execBlockMap.put(newExecBlock.getId(), newExecBlock);
+    return newExecBlock;
+  }
+
+  public boolean containsExecBlock(ExecutionBlockId execBlockId) {
+    return execBlockMap.containsKey(execBlockId);
+  }
+
+  public ExecutionBlock getExecBlock(ExecutionBlockId execBlockId) {
+    return execBlockMap.get(execBlockId);
+  }
+
+  public void addConnect(DataChannel dataChannel) {
+    execBlockGraph.addEdge(dataChannel.getSrcId(), dataChannel.getTargetId(), dataChannel);
+  }
+
+  public void addConnect(ExecutionBlock src, ExecutionBlock target, TajoWorkerProtocol.ShuffleType type) {
+    addConnect(src.getId(), target.getId(), type);
+  }
+
+  public void addConnect(ExecutionBlockId src, ExecutionBlockId target, TajoWorkerProtocol.ShuffleType type) {
+    addConnect(new DataChannel(src, target, type));
+  }
+
+  public boolean isConnected(ExecutionBlock src, ExecutionBlock target) {
+    return isConnected(src.getId(), target.getId());
+  }
+
+  public boolean isConnected(ExecutionBlockId src, ExecutionBlockId target) {
+    return execBlockGraph.hasEdge(src, target);
+  }
+
+  public boolean isReverseConnected(ExecutionBlock target, ExecutionBlock src) {
+    return execBlockGraph.hasReversedEdge(target.getId(), src.getId());
+  }
+
+  public boolean isReverseConnected(ExecutionBlockId target, ExecutionBlockId src) {
+    return execBlockGraph.hasReversedEdge(target, src);
+  }
+
+  public DataChannel getChannel(ExecutionBlock src, ExecutionBlock target) {
+    return execBlockGraph.getEdge(src.getId(), target.getId());
+  }
+
+  public DataChannel getChannel(ExecutionBlockId src, ExecutionBlockId target) {
+    return execBlockGraph.getEdge(src, target);
+  }
+
+  public List<DataChannel> getOutgoingChannels(ExecutionBlockId src) {
+    return execBlockGraph.getOutgoingEdges(src);
+  }
+
+  public boolean isRoot(ExecutionBlock execBlock) {
+    if (!execBlock.getId().equals(terminalBlock.getId())) {
+      return execBlockGraph.getParent(execBlock.getId(), 0).equals(terminalBlock.getId());
+    } else {
+      return false;
+    }
+  }
+
+  public boolean isLeaf(ExecutionBlock execBlock) {
+    return execBlockGraph.isLeaf(execBlock.getId());
+  }
+
+  public boolean isLeaf(ExecutionBlockId id) {
+    return execBlockGraph.isLeaf(id);
+  }
+
+  public List<DataChannel> getIncomingChannels(ExecutionBlockId target) {
+    return execBlockGraph.getIncomingEdges(target);
+  }
+
+  public void disconnect(ExecutionBlock src, ExecutionBlock target) {
+    disconnect(src.getId(), target.getId());
+  }
+
+  public void disconnect(ExecutionBlockId src, ExecutionBlockId target) {
+    execBlockGraph.removeEdge(src, target);
+  }
+
+  public ExecutionBlock getParent(ExecutionBlock executionBlock) {
+    return execBlockMap.get(execBlockGraph.getParent(executionBlock.getId(), 0));
+  }
+
+  public List<ExecutionBlock> getChilds(ExecutionBlock execBlock) {
+    return getChilds(execBlock.getId());
+  }
+
+  public List<ExecutionBlock> getChilds(ExecutionBlockId id) {
+    List<ExecutionBlock> childBlocks = new ArrayList<ExecutionBlock>();
+    for (ExecutionBlockId cid : execBlockGraph.getChilds(id)) {
+      childBlocks.add(execBlockMap.get(cid));
+    }
+    return childBlocks;
+  }
+
+  public int getChildCount(ExecutionBlockId blockId) {
+    return execBlockGraph.getChildCount(blockId);
+  }
+
+  public ExecutionBlock getChild(ExecutionBlockId execBlockId, int idx) {
+    return execBlockMap.get(execBlockGraph.getChild(execBlockId, idx));
+  }
+
+  public ExecutionBlock getChild(ExecutionBlock executionBlock, int idx) {
+    return getChild(executionBlock.getId(), idx);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    ExecutionBlockCursor cursor = new ExecutionBlockCursor(this);
+    sb.append("-------------------------------------------------------------------------------\n");
+    sb.append("Execution Block Graph (TERMINAL - " + getTerminalBlock() + ")\n");
+    sb.append("-------------------------------------------------------------------------------\n");
+    sb.append(execBlockGraph.toStringGraph(getRoot().getId()));
+    sb.append("-------------------------------------------------------------------------------\n");
+
+    while(cursor.hasNext()) {
+      ExecutionBlock block = cursor.nextBlock();
+
+      boolean terminal = false;
+      sb.append("\n");
+      sb.append("=======================================================\n");
+      sb.append("Block Id: " + block.getId());
+      if (isTerminal(block)) {
+        sb.append(" [TERMINAL]");
+        terminal = true;
+      } else if (isRoot(block)) {
+        sb.append(" [ROOT]");
+      } else if (isLeaf(block)) {
+        sb.append(" [LEAF]");
+      } else {
+        sb.append(" [INTERMEDIATE]");
+      }
+      sb.append("\n");
+      sb.append("=======================================================\n");
+      if (terminal) {
+        continue;
+      }
+
+      if (!isLeaf(block)) {
+        sb.append("\n[Incoming]\n");
+        for (DataChannel channel : getIncomingChannels(block.getId())) {
+          sb.append(channel).append("\n");
+        }
+      }
+
+      if (!isRoot(block)) {
+        sb.append("\n[Outgoing]\n");
+        for (DataChannel channel : getOutgoingChannels(block.getId())) {
+          sb.append(channel);
+          sb.append("\n");
+        }
+      }
+
+      if (block.getEnforcer().getProperties().size() > 0) {
+        sb.append("\n[Enforcers]\n");
+        int i = 0;
+        for (TajoWorkerProtocol.EnforceProperty enforce : block.getEnforcer().getProperties()) {
+          sb.append(" ").append(i++).append(": ");
+          sb.append(Enforcer.toString(enforce));
+          sb.append("\n");
+        }
+      }
+
+      sb.append("\n").append(PlannerUtil.buildExplainString(block.getPlan()));
+    }
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java
new file mode 100644
index 0000000..c6f4aaa
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java
@@ -0,0 +1,64 @@
+/**
+ * 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.planner.graph;
+
+import org.apache.tajo.annotation.Nullable;
+
+import java.util.List;
+
+/**
+ * This represents a directed graph.
+ *
+ * @param <V> The vertex class type
+ * @param <E> The edge class type
+ */
+public interface DirectedGraph<V, E> extends Graph<V, E> {
+
+  boolean hasReversedEdge(V head, V tail);
+
+  E getReverseEdge(V head, V tail);
+
+  List<E> getIncomingEdges(V head);
+
+  List<E> getOutgoingEdges(V tail);
+
+  /////////////////////////////////
+  // belows are tree features
+  /////////////////////////////////
+  boolean isRoot(V v);
+
+  boolean isLeaf(V v);
+
+  int getParentCount(V block);
+
+  @Nullable V getParent(V block, int idx);
+
+  List<V> getParents(V block);
+
+  int getChildCount(V block);
+
+  @Nullable V getChild(V block, int idx);
+
+  List<V> getChilds(V block);
+
+  /**
+   * It visits all vertices in a post-order traverse way.
+   */
+  void accept(V src, DirectedGraphVisitor<V> visitor);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphCursor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphCursor.java
new file mode 100644
index 0000000..7db9b83
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphCursor.java
@@ -0,0 +1,65 @@
+/**
+ * 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.planner.graph;
+
+import java.util.ArrayList;
+
+public class DirectedGraphCursor<V,E> {
+  private DirectedGraph<V,E> graph;
+  private ArrayList<V> orderedVertices = new ArrayList<V>();
+  private int cursor = 0;
+
+  public DirectedGraphCursor(DirectedGraph<V, E> graph, V startVertex) {
+    this.graph = graph;
+    buildOrder(startVertex);
+  }
+
+  public int size() {
+    return orderedVertices.size();
+  }
+
+  private void buildOrder(V current) {
+    if (!graph.isLeaf(current)) {
+      for (V child : graph.getChilds(current)) {
+        buildOrder(child);
+      }
+    }
+    orderedVertices.add(current);
+  }
+
+  public boolean hasNext() {
+    return cursor < orderedVertices.size();
+  }
+
+  public V nextBlock() {
+    return orderedVertices.get(cursor++);
+  }
+
+  public V peek() {
+    return orderedVertices.get(cursor);
+  }
+
+  public V peek(int skip) {
+    return orderedVertices.get(cursor + skip);
+  }
+
+  public void reset() {
+    cursor = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java
new file mode 100644
index 0000000..3deec7f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java
@@ -0,0 +1,25 @@
+/**
+ * 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.planner.graph;
+
+import java.util.Stack;
+
+public interface DirectedGraphVisitor<V> {
+  void visit(Stack<V> stack, V v);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/Graph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/Graph.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/Graph.java
new file mode 100644
index 0000000..d380b94
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/Graph.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.graph;
+
+import org.apache.tajo.annotation.Nullable;
+
+import java.util.Collection;
+
+/**
+ * This is the topmost graph interface. It only provides essential graph features.
+ * @param <V> Vertex class
+ * @param <E> Edge Class
+ */
+
+public interface Graph<V, E> {
+  int getVertexSize();
+
+  int getEdgeNum();
+
+  void addEdge(V tail, V head, E edge);
+
+  void removeEdge(V tail, V head);
+
+  boolean hasEdge(V tail, V head);
+
+  @Nullable E getEdge(V tail, V head);
+
+  Collection<E> getEdgesAll();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java
new file mode 100644
index 0000000..396eb8b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java
@@ -0,0 +1,270 @@
+/**
+ * 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.planner.graph;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+/**
+ * This represents a simple directed graph. It does not support multiple edges between both vertices.
+ *
+ * @param <V> The vertex class type
+ * @param <E> The edge class type
+ */
+public class SimpleDirectedGraph<V, E> implements DirectedGraph<V,E> {
+  /** map: child -> parent */
+  protected Map<V, Map<V, E>> directedEdges = TUtil.newLinkedHashMap();
+  /** map: parent -> child */
+  protected Map<V, Map<V, E>> reversedEdges = TUtil.newLinkedHashMap();
+
+  @Override
+  public int getVertexSize() {
+    return directedEdges.size();
+  }
+
+  @Override
+  public int getEdgeNum() {
+    int edgeNum = 0;
+    for (Map<V, E> map : directedEdges.values()) {
+      edgeNum += map.values().size();
+    }
+
+    return edgeNum;
+  }
+
+  @Override
+  public void addEdge(V tail, V head, E edge) {
+    TUtil.putToNestedMap(directedEdges, tail, head, edge);
+    TUtil.putToNestedMap(reversedEdges, head, tail, edge);
+  }
+
+  @Override
+  public void removeEdge(V tail, V head) {
+    if (directedEdges.containsKey(tail)) {
+      directedEdges.get(tail).remove(head);
+      if (directedEdges.get(tail).isEmpty()) {
+        directedEdges.remove(tail);
+      }
+
+      reversedEdges.get(head).remove(tail);
+      if (reversedEdges.get(head).isEmpty()) {
+        reversedEdges.remove(head);
+      }
+    } else {
+      throw new RuntimeException("Not connected channel: " + tail + " -> " + head);
+    }
+  }
+
+  @Override
+  public boolean hasEdge(V tail, V head) {
+    return directedEdges.containsKey(tail) && directedEdges.get(tail).containsKey(head);
+  }
+
+  @Override
+  public boolean hasReversedEdge(V head, V tail) {
+    return reversedEdges.containsKey(head) && reversedEdges.get(head).containsKey(tail);
+  }
+
+  @Override
+  public @Nullable E getEdge(V tail, V head) {
+    if (hasEdge(tail, head)) {
+      return directedEdges.get(tail).get(head);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public @Nullable
+  E getReverseEdge(V head, V tail) {
+    if (hasReversedEdge(head, tail)) {
+      return reversedEdges.get(head).get(tail);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public Collection<E> getEdgesAll() {
+    List<E> edges = Lists.newArrayList();
+    for (Map<V, E> map : directedEdges.values()) {
+      edges.addAll(map.values());
+    }
+    return edges;
+  }
+
+  @Override
+  public int getChildCount(V v) {
+    if (reversedEdges.containsKey(v)) {
+      return reversedEdges.get(v).size();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public List<E> getIncomingEdges(V head) {
+    if (reversedEdges.containsKey(head)) {
+      return ImmutableList.copyOf(reversedEdges.get(head).values());
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public List<E> getOutgoingEdges(V tail) {
+    if (directedEdges.containsKey(tail)) {
+      return ImmutableList.copyOf(directedEdges.get(tail).values());
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public List<V> getChilds(V v) {
+    List<V> childBlocks = new ArrayList<V>();
+    if (reversedEdges.containsKey(v)) {
+      for (Map.Entry<V, E> entry: reversedEdges.get(v).entrySet()) {
+        childBlocks.add(entry.getKey());
+      }
+    }
+    return childBlocks;
+  }
+
+  @Override
+  public V getChild(V block, int idx) {
+    if (reversedEdges.containsKey(block)) {
+      int i = 0;
+      for (Map.Entry<V, E> entry: reversedEdges.get(block).entrySet()) {
+        if (idx == i++) {
+          return entry.getKey();
+        }
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public @Nullable V getParent(V block, int idx) {
+    if (directedEdges.containsKey(block)) {
+      int i = 0;
+      for (Map.Entry<V, E> entry: directedEdges.get(block).entrySet()) {
+        if (idx == i++) {
+          return entry.getKey();
+        }
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public List<V> getParents(V block) {
+    List<V> childBlocks = new ArrayList<V>();
+    if (directedEdges.containsKey(block)) {
+      for (Map.Entry<V, E> entry: directedEdges.get(block).entrySet()) {
+        childBlocks.add(entry.getKey());
+      }
+    }
+    return childBlocks;
+  }
+
+  @Override
+  public boolean isRoot(V v) {
+    return !directedEdges.containsKey(v);
+  }
+
+  @Override
+  public boolean isLeaf(V v) {
+    return !reversedEdges.containsKey(v);
+  }
+
+  @Override
+  public int getParentCount(V block) {
+    if (directedEdges.containsKey(block)) {
+      return directedEdges.get(block).size();
+    } else {
+      return -1;
+    }
+  }
+
+  @Override
+  public void accept(V source, DirectedGraphVisitor<V> visitor) {
+    Stack<V> stack = new Stack<V>();
+    visitRecursive(stack, source, visitor);
+  }
+
+  private void visitRecursive(Stack<V> stack, V current, DirectedGraphVisitor<V> visitor) {
+    stack.push(current);
+    for (V child : getChilds(current)) {
+      visitRecursive(stack, child, visitor);
+    }
+    stack.pop();
+    visitor.visit(stack, current);
+  }
+
+  public String toString() {
+    return "G (|v| = " + directedEdges.size() +")";
+  }
+
+  public String printDepthString(DepthString planStr) {
+    StringBuilder output = new StringBuilder();
+    String pad = new String(new char[planStr.depth * 3]).replace('\0', ' ');
+    output.append(pad + "|-" + planStr.vertexStr).append("\n");
+
+    return output.toString();
+  }
+
+  public String toStringGraph(V vertex) {
+    StringBuilder sb = new StringBuilder();
+    QueryGraphTopologyStringBuilder visitor = new QueryGraphTopologyStringBuilder();
+    accept(vertex, visitor);
+    Stack<DepthString> depthStrings = visitor.getDepthStrings();
+    while(!depthStrings.isEmpty()) {
+      sb.append(printDepthString(depthStrings.pop()));
+    }
+    return sb.toString();
+  }
+
+  private class DepthString {
+    int depth;
+    String vertexStr;
+
+    DepthString(int depth, String vertexStr) {
+      this.depth = depth;
+      this.vertexStr = vertexStr;
+    }
+  }
+
+  private class QueryGraphTopologyStringBuilder implements DirectedGraphVisitor<V> {
+    Stack<DepthString> depthString = new Stack<DepthString>();
+
+    @Override
+    public void visit(Stack<V> stack, V vertex) {
+      depthString.push(new DepthString(stack.size(), vertex.toString()));
+    }
+
+    public Stack<DepthString> getDepthStrings() {
+      return depthString;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleUndirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleUndirectedGraph.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleUndirectedGraph.java
new file mode 100644
index 0000000..3822ad7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/SimpleUndirectedGraph.java
@@ -0,0 +1,102 @@
+/**
+ * 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.planner.graph;
+
+import com.google.common.collect.Lists;
+import org.apache.tajo.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This implementation is based on a directed graph implementation. Each edge is connected in bidirection
+ * between two vertices.
+ *
+ * @param <V> Vertex Class
+ * @param <E> Edge Class
+ */
+public class SimpleUndirectedGraph<V, E> extends SimpleDirectedGraph<V, E> implements UndirectedGraph<V, E> {
+
+  @Override
+  public @Nullable E getEdge(V tail, V head) {
+    E edge = super.getEdge(tail, head);
+    if (edge != null) {
+      return edge;
+    }
+    edge = super.getEdge(head, tail);
+    if (edge != null) {
+      return edge;
+    }
+    return null;
+  }
+
+  @Override
+  public Collection<E> getEdges(V v) {
+    List<E> edges = Lists.newArrayList();
+    List<E> outgoingEdges = getOutgoingEdges(v);
+    if (outgoingEdges != null) {
+      edges.addAll(outgoingEdges);
+    }
+    List<E> incomingEdges = getIncomingEdges(v);
+    if (incomingEdges != null) {
+      edges.addAll(incomingEdges);
+    }
+    return edges;
+  }
+
+  @Override
+  public int getDegree(V v) {
+    return getEdges(v).size();
+  }
+
+  @Override
+  public Collection<E> getEdgesAll() {
+    List<E> edges = Lists.newArrayList();
+    for (Map<V, E> map : directedEdges.values()) {
+      edges.addAll(map.values());
+    }
+    return edges;
+  }
+
+  @Override
+  public V getParent(V v, int idx) {
+    throw new UnsupportedOperationException("Cannot support getParent(V v) in UndirectedGraph");
+  }
+
+  @Override
+  public int getParentCount(V v) {
+    throw new UnsupportedOperationException("Cannot support getParent(V v) in UndirectedGraph");
+  }
+
+  @Override
+  public List<V> getParents(V v) {
+    throw new UnsupportedOperationException("Cannot support getParent(V v) in UndirectedGraph");
+  }
+
+  @Override
+  public boolean isRoot(V v) {
+    throw new UnsupportedOperationException("Cannot support isRoot(V v) in UndirectedGraph");
+  }
+
+  @Override
+  public boolean isLeaf(V v) {
+    throw new UnsupportedOperationException("Cannot support isLeaf(V v) in UndirectedGraph");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/UndirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/UndirectedGraph.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/UndirectedGraph.java
new file mode 100644
index 0000000..7f74cd1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/graph/UndirectedGraph.java
@@ -0,0 +1,30 @@
+/**
+ * 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.planner.graph;
+
+
+import org.apache.tajo.annotation.NotNull;
+
+import java.util.Collection;
+
+public interface UndirectedGraph<V, E> extends Graph<V, E> {
+  Collection<E> getEdges(@NotNull V v);
+
+  int getDegree(@NotNull V v);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java
new file mode 100644
index 0000000..76a47d0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java
@@ -0,0 +1,134 @@
+/**
+ * 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.planner.logical;
+
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.algebra.AlterTableOpType;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.planner.PlanString;
+
+public class AlterTableNode extends LogicalNode {
+
+  @Expose
+  private String tableName;
+  @Expose
+  private String newTableName;
+  @Expose
+  private String columnName;
+  @Expose
+  private String newColumnName;
+  @Expose
+  private Column addNewColumn;
+  @Expose
+  private AlterTableOpType alterTableOpType;
+
+  public AlterTableNode(int pid) {
+    super(pid, NodeType.ALTER_TABLE);
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public String getNewTableName() {
+    return newTableName;
+  }
+
+  public void setNewTableName(String newTableName) {
+    this.newTableName = newTableName;
+  }
+
+  public String getColumnName() {
+    return columnName;
+  }
+
+  public void setColumnName(String columnName) {
+    this.columnName = columnName;
+  }
+
+  public String getNewColumnName() {
+    return newColumnName;
+  }
+
+  public void setNewColumnName(String newColumnName) {
+    this.newColumnName = newColumnName;
+  }
+
+  public Column getAddNewColumn() {
+    return addNewColumn;
+  }
+
+  public void setAddNewColumn(Column addNewColumn) {
+    this.addNewColumn = addNewColumn;
+  }
+
+  public AlterTableOpType getAlterTableOpType() {
+    return alterTableOpType;
+  }
+
+  public void setAlterTableOpType(AlterTableOpType alterTableOpType) {
+    this.alterTableOpType = alterTableOpType;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof AlterTableNode) {
+      AlterTableNode other = (AlterTableNode) obj;
+      return super.equals(other);
+    } else {
+      return false;
+    }
+  }
+
+    /*@Override
+    public Object clone() throws CloneNotSupportedException {
+        AlterTableNode alterTableNode = (AlterTableNode) super.clone();
+        alterTableNode.tableName = tableName;
+        alterTableNode.newTableName = newTableName;
+        alterTableNode.columnName = columnName;
+        alterTableNode.newColumnName=newColumnName;
+        alterTableNode.addNewColumn =(Column) addNewColumn.clone();
+        return alterTableNode;
+    }*/
+
+  @Override
+  public String toString() {
+    return "AlterTable (table=" + tableName + ")";
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+}


[04/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
new file mode 100644
index 0000000..1dbbcf0
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
@@ -0,0 +1,209 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestHashAntiJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestHashJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private LogicalOptimizer optimizer;
+  private AbstractStorageManager sm;
+  private Path testDir;
+
+  private TableDesc employee;
+  private TableDesc people;
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema employeeSchema = new Schema();
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
+        employeePath);
+    appender.init();
+    Tuple tuple = new VTuple(employeeSchema.size());
+
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), // empid [0-9]
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
+      appender.addTuple(tuple);
+    }
+
+    appender.flush();
+    appender.close();
+    employee = CatalogUtil.newTableDesc("default.employee", employeeSchema, employeeMeta, employeePath);
+    catalog.createTable(employee);
+
+    Schema peopleSchema = new Schema();
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path peoplePath = new Path(testDir, "people.csv");
+    appender = StorageManagerFactory.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender.init();
+    tuple = new VTuple(peopleSchema.size());
+    for (int i = 1; i < 10; i += 2) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(i), // empid [1, 3, 5, 7, 9]
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
+      appender.addTuple(tuple);
+    }
+
+    appender.flush();
+    appender.close();
+
+    people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
+    catalog.createTable(people);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+    optimizer = new LogicalOptimizer(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+
+  // relation descriptions
+  // employee (managerid, empid, memid, deptname)
+  // people (empid, fk_memid, name, age)
+
+  String[] QUERIES = {
+      "select managerId, e.empId, deptName, e.memId from employee as e, people as p where e.empId = p.empId"
+  };
+
+  @Test
+  public final void testHashAntiJoin() throws IOException, PlanningException {
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashAntiJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    optimizer.optimize(plan);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    // replace an equal join with an hash anti join.
+    if (exec instanceof MergeJoinExec) {
+      MergeJoinExec join = (MergeJoinExec) exec;
+      ExternalSortExec sortLeftChild = (ExternalSortExec) join.getLeftChild();
+      ExternalSortExec sortRightChild = (ExternalSortExec) join.getRightChild();
+      SeqScanExec scanLeftChild = sortLeftChild.getChild();
+      SeqScanExec scanRightChild = sortRightChild.getChild();
+
+      // 'people' should be outer table. So, the below code guarantees that people becomes the outer table.
+      if (scanLeftChild.getTableName().equals("default.people")) {
+        exec = new HashLeftAntiJoinExec(ctx, join.getPlan(), scanRightChild, scanLeftChild);
+      } else {
+        exec = new HashLeftAntiJoinExec(ctx, join.getPlan(), scanLeftChild, scanRightChild);
+      }
+    } else if (exec instanceof HashJoinExec) {
+      HashJoinExec join = (HashJoinExec) exec;
+      SeqScanExec scanLeftChild = (SeqScanExec) join.getLeftChild();
+
+      // 'people' should be outer table. So, the below code guarantees that people becomes the outer table.
+      if (scanLeftChild.getTableName().equals("default.people")) {
+        exec = new HashLeftAntiJoinExec(ctx, join.getPlan(), join.getRightChild(), join.getLeftChild());
+      } else {
+        exec = new HashLeftAntiJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild());
+      }
+    }
+
+    Tuple tuple;
+    int count = 0;
+    int i = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      count++;
+      assertTrue(i == tuple.get(0).asInt4());
+      assertTrue(i == tuple.get(1).asInt4()); // expected empid [0, 2, 4, 6, 8]
+      assertTrue(("dept_" + i).equals(tuple.get(2).asChars()));
+      assertTrue(10 + i == tuple.get(3).asInt4());
+
+      i += 2;
+    }
+    exec.close();
+    assertEquals(5 , count); // the expected result : [0, 2, 4, 6, 8]
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
new file mode 100644
index 0000000..66222da
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
@@ -0,0 +1,275 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.junit.Assert.*;
+
+public class TestHashJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestHashJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+  private final Session session = LocalTajoTestingUtility.createDummySession();
+
+  private TableDesc employee;
+  private TableDesc people;
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema employeeSchema = new Schema();
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
+        employeePath);
+    appender.init();
+    Tuple tuple = new VTuple(employeeSchema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
+      appender.addTuple(tuple);
+    }
+
+    appender.flush();
+    appender.close();
+    employee = CatalogUtil.newTableDesc("default.employee", employeeSchema, employeeMeta, employeePath);
+    catalog.createTable(employee);
+
+    Schema peopleSchema = new Schema();
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path peoplePath = new Path(testDir, "people.csv");
+    appender = StorageManagerFactory.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender.init();
+    tuple = new VTuple(peopleSchema.size());
+    for (int i = 1; i < 10; i += 2) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
+      appender.addTuple(tuple);
+    }
+
+    appender.flush();
+    appender.close();
+
+    people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
+    catalog.createTable(people);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      "select managerId, e.empId, deptName, e.memId from employee as e inner join " +
+          "people as p on e.empId = p.empId and e.memId = p.fk_memId"
+  };
+
+  @Test
+  public final void testHashInnerJoin() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
+
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(), Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashInnerJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof HashJoinExec);
+
+    Tuple tuple;
+    int count = 0;
+    int i = 1;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      count++;
+      assertTrue(i == tuple.get(0).asInt4());
+      assertTrue(i == tuple.get(1).asInt4());
+      assertTrue(("dept_" + i).equals(tuple.get(2).asChars()));
+      assertTrue(10 + i == tuple.get(3).asInt4());
+
+      i += 2;
+    }
+    exec.close();
+    assertEquals(10 / 2, count);
+  }
+
+  @Test
+  public final void testCheckIfInMemoryInnerJoinIsPossible() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
+
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashInnerJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    TajoConf localConf = new TajoConf(conf);
+    localConf.setLongVar(TajoConf.ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD, 100l);
+    PhysicalPlannerImpl phyPlanner = new PhysicalPlannerImpl(localConf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof HashJoinExec);
+    HashJoinExec joinExec = proj.getChild();
+
+    assertCheckInnerJoinRelatedFunctions(ctx, phyPlanner, joinNode, joinExec);
+  }
+
+  /**
+   * It checks inner-join related functions. It will return TRUE if left relations is smaller than right relations.
+   *
+   * The below unit tests will work according to which side is smaller. In this unit tests, we use two tables: p and e.
+   * The table p is 75 bytes, and the table e is 140 bytes. Since we cannot expect that which side is smaller,
+   * we use some boolean variable <code>leftSmaller</code> to indicate which side is small.
+   */
+  private static boolean assertCheckInnerJoinRelatedFunctions(TaskAttemptContext ctx,
+                                                       PhysicalPlannerImpl phyPlanner,
+                                                       JoinNode joinNode, BinaryPhysicalExec joinExec) throws
+      IOException {
+
+    String [] left = PlannerUtil.getRelationLineage(joinNode.getLeftChild());
+    String [] right = PlannerUtil.getRelationLineage(joinNode.getRightChild());
+
+    boolean leftSmaller;
+    if (left[0].equals("default.p")) {
+      leftSmaller = true;
+    } else {
+      leftSmaller = false;
+    }
+
+    long leftSize = phyPlanner.estimateSizeRecursive(ctx, left);
+    long rightSize = phyPlanner.estimateSizeRecursive(ctx, right);
+
+    // The table p is 75 bytes, and the table e is 140 bytes.
+    if (leftSmaller) { // if left one is smaller
+      assertEquals(75, leftSize);
+      assertEquals(140, rightSize);
+    } else { // if right one is smaller
+      assertEquals(140, leftSize);
+      assertEquals(75, rightSize);
+    }
+
+    if (leftSmaller) {
+      PhysicalExec [] ordered = phyPlanner.switchJoinSidesIfNecessary(ctx, joinNode, joinExec.getLeftChild(),
+          joinExec.getRightChild());
+      assertEquals(ordered[0], joinExec.getLeftChild());
+      assertEquals(ordered[1], joinExec.getRightChild());
+
+      assertEquals("default.p", left[0]);
+      assertEquals("default.e", right[0]);
+    } else {
+      PhysicalExec [] ordered = phyPlanner.switchJoinSidesIfNecessary(ctx, joinNode, joinExec.getLeftChild(),
+          joinExec.getRightChild());
+      assertEquals(ordered[1], joinExec.getLeftChild());
+      assertEquals(ordered[0], joinExec.getRightChild());
+
+      assertEquals("default.e", left[0]);
+      assertEquals("default.p", right[0]);
+    }
+
+    if (leftSmaller) {
+      assertTrue(phyPlanner.checkIfInMemoryInnerJoinIsPossible(ctx, joinNode.getLeftChild(), true));
+      assertFalse(phyPlanner.checkIfInMemoryInnerJoinIsPossible(ctx, joinNode.getRightChild(), false));
+    } else {
+      assertFalse(phyPlanner.checkIfInMemoryInnerJoinIsPossible(ctx, joinNode.getLeftChild(), true));
+      assertTrue(phyPlanner.checkIfInMemoryInnerJoinIsPossible(ctx, joinNode.getRightChild(), false));
+    }
+
+    return leftSmaller;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashPartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashPartitioner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashPartitioner.java
new file mode 100644
index 0000000..f0d846c
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashPartitioner.java
@@ -0,0 +1,84 @@
+/**
+ * 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.planner.physical;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHashPartitioner {
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public final void testGetPartition() {   
+    Tuple tuple1 = new VTuple(3);    
+    tuple1.put(new Datum[] {
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(3)
+    });
+    Tuple tuple2 = new VTuple(3);    
+    tuple2.put(new Datum[] {
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(4)
+    });
+    Tuple tuple3 = new VTuple(3);    
+    tuple3.put(new Datum[] {
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(5)
+    });
+    Tuple tuple4 = new VTuple(3);    
+    tuple4.put(new Datum[] {
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(3)
+    });
+    Tuple tuple5 = new VTuple(3);    
+    tuple5.put(new Datum[] {
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(4)
+    });
+    
+    int [] partKeys = {0,1};
+    Partitioner p = new HashPartitioner(partKeys, 2);
+    
+    int part1 = p.getPartition(tuple1);
+    assertEquals(part1, p.getPartition(tuple2));
+    assertEquals(part1, p.getPartition(tuple3));
+    
+    int part2 = p.getPartition(tuple4);
+    assertEquals(part2, p.getPartition(tuple5));    
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
new file mode 100644
index 0000000..4e5de98
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
@@ -0,0 +1,214 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestHashSemiJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestHashJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private LogicalOptimizer optimizer;
+  private AbstractStorageManager sm;
+  private Path testDir;
+
+  private TableDesc employee;
+  private TableDesc people;
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema employeeSchema = new Schema();
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
+        employeePath);
+    appender.init();
+    Tuple tuple = new VTuple(employeeSchema.size());
+
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), // empid [0-9]
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
+      appender.addTuple(tuple);
+    }
+
+    appender.flush();
+    appender.close();
+    employee = CatalogUtil.newTableDesc("default.employee", employeeSchema, employeeMeta, employeePath);
+    catalog.createTable(employee);
+
+    Schema peopleSchema = new Schema();
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path peoplePath = new Path(testDir, "people.csv");
+    appender = StorageManagerFactory.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender.init();
+    tuple = new VTuple(peopleSchema.size());
+    // make 27 tuples
+    for (int i = 1; i < 10; i += 2) {
+      // make three duplicated tuples for each tuples
+      for (int j = 0; j < 3; j++) {
+        tuple.put(new Datum[] {
+            DatumFactory.createInt4(i), // empid [1, 3, 5, 7, 9]
+            DatumFactory.createInt4(10 + i),
+            DatumFactory.createText("name_" + i),
+            DatumFactory.createInt4(30 + i) });
+        appender.addTuple(tuple);
+      }
+    }
+
+    appender.flush();
+    appender.close();
+
+    people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
+    catalog.createTable(people);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+    optimizer = new LogicalOptimizer(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+
+  // relation descriptions
+  // employee (managerid, empid, memid, deptname)
+  // people (empid, fk_memid, name, age)
+
+  String[] QUERIES = {
+      "select managerId, e.empId, deptName, e.memId from employee as e, people as p where e.empId = p.empId"
+  };
+
+  @Test
+  public final void testHashSemiJoin() throws IOException, PlanningException {
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashSemiJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    optimizer.optimize(plan);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    // replace an equal join with an hash anti join.
+    if (exec instanceof MergeJoinExec) {
+      MergeJoinExec join = (MergeJoinExec) exec;
+      ExternalSortExec sortLeftChild = (ExternalSortExec) join.getLeftChild();
+      ExternalSortExec sortRightChild = (ExternalSortExec) join.getRightChild();
+      SeqScanExec scanLeftChild = (SeqScanExec) sortLeftChild.getChild();
+      SeqScanExec scanRightChild = (SeqScanExec) sortRightChild.getChild();
+
+      // 'people' should be outer table. So, the below code guarantees that people becomes the outer table.
+      if (scanLeftChild.getTableName().equals("default.people")) {
+        exec = new HashLeftSemiJoinExec(ctx, join.getPlan(), scanRightChild, scanLeftChild);
+      } else {
+        exec = new HashLeftSemiJoinExec(ctx, join.getPlan(), scanLeftChild, scanRightChild);
+      }
+    } else if (exec instanceof HashJoinExec) {
+      HashJoinExec join = (HashJoinExec) exec;
+      SeqScanExec scanLeftChild = (SeqScanExec) join.getLeftChild();
+
+      // 'people' should be outer table. So, the below code guarantees that people becomes the outer table.
+      if (scanLeftChild.getTableName().equals("default.people")) {
+        exec = new HashLeftSemiJoinExec(ctx, join.getPlan(), join.getRightChild(), join.getLeftChild());
+      } else {
+        exec = new HashLeftSemiJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild());
+      }
+    }
+
+    Tuple tuple;
+    int count = 0;
+    int i = 1;
+    exec.init();
+    // expect result without duplicated tuples.
+    while ((tuple = exec.next()) != null) {
+      count++;
+      assertTrue(i == tuple.get(0).asInt4());
+      assertTrue(i == tuple.get(1).asInt4());
+      assertTrue(("dept_" + i).equals(tuple.get(2).asChars()));
+      assertTrue(10 + i == tuple.get(3).asInt4());
+
+      i += 2;
+    }
+    exec.close();
+    assertEquals(5 , count); // the expected result: [1, 3, 5, 7, 9]
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
new file mode 100644
index 0000000..de3d298
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
@@ -0,0 +1,463 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestLeftOuterHashJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestLeftOuterHashJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+  private Session session = LocalTajoTestingUtility.createDummySession();
+
+  private TableDesc dep3;
+  private TableDesc job3;
+  private TableDesc emp3;
+  private TableDesc phone3;
+
+  private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3");
+  private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3");
+  private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3");
+  private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3");
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    //----------------- dep3 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    Schema dep3Schema = new Schema();
+    dep3Schema.addColumn("dep_id", Type.INT4);
+    dep3Schema.addColumn("dep_name", Type.TEXT);
+    dep3Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep3Path = new Path(testDir, "dep3.csv");
+    Appender appender1 = StorageManagerFactory.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    appender1.init();
+    Tuple tuple = new VTuple(dep3Schema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+                    DatumFactory.createText("dept_" + i),
+                    DatumFactory.createInt4(1000 + i) });
+      appender1.addTuple(tuple);
+    }
+
+    appender1.flush();
+    appender1.close();
+    dep3 = CatalogUtil.newTableDesc(DEP3_NAME, dep3Schema, dep3Meta, dep3Path);
+    catalog.createTable(dep3);
+
+    //----------------- job3 ------------------------------
+    //  job_id  | job_title
+    // ----------------------
+    //   101    |  job_101
+    //   102    |  job_102
+    //   103    |  job_103
+
+    Schema job3Schema = new Schema();
+    job3Schema.addColumn("job_id", Type.INT4);
+    job3Schema.addColumn("job_title", Type.TEXT);
+
+
+    TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path job3Path = new Path(testDir, "job3.csv");
+    Appender appender2 = StorageManagerFactory.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    appender2.init();
+    Tuple tuple2 = new VTuple(job3Schema.size());
+    for (int i = 1; i < 4; i++) {
+      int x = 100 + i;
+      tuple2.put(new Datum[] { DatumFactory.createInt4(100 + i),
+                    DatumFactory.createText("job_" + x) });
+      appender2.addTuple(tuple2);
+    }
+
+    appender2.flush();
+    appender2.close();
+    job3 = CatalogUtil.newTableDesc(JOB3_NAME, job3Schema, job3Meta, job3Path);
+    catalog.createTable(job3);
+
+
+
+    //---------------------emp3 --------------------
+    // emp_id  | first_name | last_name | dep_id | salary | job_id
+    // ------------------------------------------------------------
+    //  11     |  fn_11     |  ln_11    |  1     | 123    | 101
+    //  13     |  fn_13     |  ln_13    |  3     | 369    | 103
+    //  15     |  fn_15     |  ln_15    |  5     | 615    | null
+    //  17     |  fn_17     |  ln_17    |  7     | 861    | null
+    //  19     |  fn_19     |  ln_19    |  9     | 1107   | null
+    //  21     |  fn_21     |  ln_21    |  1     | 123    | 101
+    //  23     |  fn_23     |  ln_23    |  3     | 369    | 103
+
+    Schema emp3Schema = new Schema();
+    emp3Schema.addColumn("emp_id", Type.INT4);
+    emp3Schema.addColumn("first_name", Type.TEXT);
+    emp3Schema.addColumn("last_name", Type.TEXT);
+    emp3Schema.addColumn("dep_id", Type.INT4);
+    emp3Schema.addColumn("salary", Type.FLOAT4);
+    emp3Schema.addColumn("job_id", Type.INT4);
+
+
+    TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path emp3Path = new Path(testDir, "emp3.csv");
+    Appender appender3 = StorageManagerFactory.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    appender3.init();
+    Tuple tuple3 = new VTuple(emp3Schema.size());
+
+    for (int i = 1; i < 4; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+
+      int y = 20 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(20 + i),
+          DatumFactory.createText("firstname_" + y),
+          DatumFactory.createText("lastname_" + y),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+    }
+
+    for (int i = 5; i < 10; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createNullDatum() });
+      appender3.addTuple(tuple3);
+    }
+
+    appender3.flush();
+    appender3.close();
+    emp3 = CatalogUtil.newTableDesc(EMP3_NAME, emp3Schema, emp3Meta, emp3Path);
+    catalog.createTable(emp3);
+
+    //---------------------phone3 --------------------
+    // emp_id  | phone_number
+    // -----------------------------------------------
+    // this table is empty, no rows
+
+    Schema phone3Schema = new Schema();
+    phone3Schema.addColumn("emp_id", Type.INT4);
+    phone3Schema.addColumn("phone_number", Type.TEXT);
+
+
+    TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path phone3Path = new Path(testDir, "phone3.csv");
+    Appender appender5 = StorageManagerFactory.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
+        phone3Path);
+    appender5.init();
+    
+    appender5.flush();
+    appender5.close();
+    phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path);
+    catalog.createTable(phone3);
+
+
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      // [0] no nulls
+      "select dep3.dep_id, dep_name, emp_id, salary from dep3 left outer join emp3 on dep3.dep_id = emp3.dep_id",
+      // [1] nulls on the right operand
+      "select job3.job_id, job_title, emp_id, salary from job3 left outer join emp3 on job3.job_id=emp3.job_id",
+      // [2] nulls on the left side
+      "select job3.job_id, job_title, emp_id, salary from emp3 left outer join job3 on job3.job_id=emp3.job_id",
+      // [3] one operand is empty
+      "select emp3.emp_id, first_name, phone_number from emp3 left outer join phone3 on emp3.emp_id = phone3.emp_id",
+      // [4] one operand is empty
+      "select phone_number, emp3.emp_id, first_name from phone3 left outer join emp3 on emp3.emp_id = phone3.emp_id"
+  };
+
+  @Test
+  public final void testLeftOuterHashJoinExec0() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
+
+    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuterHashJoinExec0");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof HashLeftOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    exec.close();
+    assertEquals(12, count);
+  }
+
+
+  @Test
+  public final void testLeftOuter_HashJoinExec1() throws IOException, PlanningException {
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec1");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof NLLeftOuterJoinExec) {
+       //for this small data set this is not likely to happen
+      
+      assertEquals(1, 0);
+    }
+    else{
+       Tuple tuple;
+       int count = 0;
+       int i = 1;
+       exec.init();
+  
+       while ((tuple = exec.next()) != null) {
+         //TODO check contents
+         count = count + 1;
+       }
+       exec.close();
+       assertEquals(5, count);
+    }
+  }
+
+    @Test
+  public final void testLeftOuter_HashJoinExec2() throws IOException, PlanningException {
+    
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec2");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[2]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof NLLeftOuterJoinExec) {
+      //for this small data set this is not likely to happen
+      
+      assertEquals(1, 0);
+    }
+    else{
+       Tuple tuple;
+       int count = 0;
+       int i = 1;
+       exec.init();
+  
+       while ((tuple = exec.next()) != null) {
+         //TODO check contents
+         count = count + 1;
+       }
+       exec.close();
+       assertEquals(7, count);
+    }
+  }
+
+
+   @Test
+  public final void testLeftOuter_HashJoinExec3() throws IOException, PlanningException {
+    
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec3");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[3]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof NLLeftOuterJoinExec) {
+      //for this small data set this is not likely to happen
+      
+      assertEquals(1, 0);
+    }
+    else{
+       Tuple tuple;
+       int count = 0;
+       int i = 1;
+       exec.init();
+  
+       while ((tuple = exec.next()) != null) {
+         //TODO check contents
+         count = count + 1;
+       }
+       exec.close();
+       assertEquals(7, count);
+    }
+  }
+
+  
+   @Test
+  public final void testLeftOuter_HashJoinExec4() throws IOException, PlanningException {
+    
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, "default.emp3", emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = StorageManager.splitNG(conf, "default.phone3", phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec4");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[4]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof NLLeftOuterJoinExec) {
+      //for this small data set this is not likely to happen
+      
+      assertEquals(1, 0);
+    }
+    else{
+       Tuple tuple;
+       int count = 0;
+       int i = 1;
+       exec.init();
+  
+       while ((tuple = exec.next()) != null) {
+         //TODO check contents
+         count = count + 1;
+       }
+       exec.close();
+       assertEquals(0, count);
+    }
+  }
+  
+
+
+}
+ //--camelia

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
new file mode 100644
index 0000000..e806e55
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
@@ -0,0 +1,474 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestLeftOuterNLJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestLeftOuterNLJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private static final Session session = LocalTajoTestingUtility.createDummySession();
+  private AbstractStorageManager sm;
+  private Path testDir;
+
+  private TableDesc dep3;
+  private TableDesc job3;
+  private TableDesc emp3;
+  private TableDesc phone3;
+
+  private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3");
+  private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3");
+  private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3");
+  private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3");
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    //----------------- dep3 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    Schema dep3Schema = new Schema();
+    dep3Schema.addColumn("dep_id", Type.INT4);
+    dep3Schema.addColumn("dep_name", Type.TEXT);
+    dep3Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep3Path = new Path(testDir, "dep3.csv");
+    Appender appender1 = StorageManagerFactory.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    appender1.init();
+    Tuple tuple = new VTuple(dep3Schema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+                    DatumFactory.createText("dept_" + i),
+                    DatumFactory.createInt4(1000 + i) });
+      appender1.addTuple(tuple);
+    }
+
+    appender1.flush();
+    appender1.close();
+    dep3 = CatalogUtil.newTableDesc(DEP3_NAME, dep3Schema, dep3Meta, dep3Path);
+    catalog.createTable(dep3);
+
+    //----------------- job3 ------------------------------
+    //  job_id  | job_title
+    // ----------------------
+    //   101    |  job_101
+    //   102    |  job_102
+    //   103    |  job_103
+
+    Schema job3Schema = new Schema();
+    job3Schema.addColumn("job_id", Type.INT4);
+    job3Schema.addColumn("job_title", Type.TEXT);
+
+
+    TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path job3Path = new Path(testDir, "job3.csv");
+    Appender appender2 = StorageManagerFactory.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    appender2.init();
+    Tuple tuple2 = new VTuple(job3Schema.size());
+    for (int i = 1; i < 4; i++) {
+      int x = 100 + i;
+      tuple2.put(new Datum[] { DatumFactory.createInt4(100 + i),
+                    DatumFactory.createText("job_" + x) });
+      appender2.addTuple(tuple2);
+    }
+
+    appender2.flush();
+    appender2.close();
+    job3 = CatalogUtil.newTableDesc(JOB3_NAME, job3Schema, job3Meta, job3Path);
+    catalog.createTable(job3);
+
+
+
+    //---------------------emp3 --------------------
+    // emp_id  | first_name | last_name | dep_id | salary | job_id
+    // ------------------------------------------------------------
+    //  11     |  fn_11     |  ln_11    |  1     | 123    | 101
+    //  13     |  fn_13     |  ln_13    |  3     | 369    | 103
+    //  15     |  fn_15     |  ln_15    |  5     | 615    | null
+    //  17     |  fn_17     |  ln_17    |  7     | 861    | null
+    //  19     |  fn_19     |  ln_19    |  9     | 1107   | null
+    //  21     |  fn_21     |  ln_21    |  1     | 123    | 101
+    //  23     |  fn_23     |  ln_23    |  3     | 369    | 103
+
+    Schema emp3Schema = new Schema();
+    emp3Schema.addColumn("emp_id", Type.INT4);
+    emp3Schema.addColumn("first_name", Type.TEXT);
+    emp3Schema.addColumn("last_name", Type.TEXT);
+    emp3Schema.addColumn("dep_id", Type.INT4);
+    emp3Schema.addColumn("salary", Type.FLOAT4);
+    emp3Schema.addColumn("job_id", Type.INT4);
+
+
+    TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path emp3Path = new Path(testDir, "emp3.csv");
+    Appender appender3 = StorageManagerFactory.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    appender3.init();
+    Tuple tuple3 = new VTuple(emp3Schema.size());
+
+    for (int i = 1; i < 4; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+
+      int y = 20 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(20 + i),
+          DatumFactory.createText("firstname_" + y),
+          DatumFactory.createText("lastname_" + y),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+    }
+
+    for (int i = 5; i < 10; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createNullDatum() });
+      appender3.addTuple(tuple3);
+    }
+
+    appender3.flush();
+    appender3.close();
+    emp3 = CatalogUtil.newTableDesc(EMP3_NAME, emp3Schema, emp3Meta, emp3Path);
+    catalog.createTable(emp3);
+
+    // ---------------------phone3 --------------------
+    // emp_id  | phone_number
+    // -----------------------------------------------
+    // this table is empty, no rows
+
+    Schema phone3Schema = new Schema();
+    phone3Schema.addColumn("emp_id", Type.INT4);
+    phone3Schema.addColumn("phone_number", Type.TEXT);
+
+
+    TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path phone3Path = new Path(testDir, "phone3.csv");
+    Appender appender5 = StorageManagerFactory.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
+        phone3Path);
+    appender5.init();
+    
+    appender5.flush();
+    appender5.close();
+    phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path);
+    catalog.createTable(phone3);
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+  
+  String[] QUERIES = {
+      "select dep3.dep_id, dep_name, emp_id, salary from dep3 left outer join emp3 on dep3.dep_id = emp3.dep_id", //0 no nulls
+      "select job3.job_id, job_title, emp_id, salary from job3 left outer join emp3 on job3.job_id=emp3.job_id", //1 nulls on the right operand
+      "select job3.job_id, job_title, emp_id, salary from emp3 left outer join job3 on job3.job_id=emp3.job_id", //2 nulls on the left side
+      "select emp3.emp_id, first_name, phone_number from emp3 left outer join phone3 on emp3.emp_id = phone3.emp_id", //3 one operand is empty
+      "select phone_number, emp3.emp_id, first_name from phone3 left outer join emp3 on emp3.emp_id = phone3.emp_id" //4 one operand is empty
+  };
+
+  @Test
+  public final void testLeftOuterNLJoinExec0() throws IOException, PlanningException {
+    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuterNLJoinExec0");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context =  analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof HashLeftOuterJoinExec) {
+      HashLeftOuterJoinExec join = proj.getChild();
+      NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild());
+      proj.setChild(aJoin);
+      exec = proj;
+    }
+
+    int count = 0;
+    exec.init();
+    while (exec.next() != null) {
+       //TODO check contents
+         count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(12, count);
+  }
+
+
+  @Test
+  public final void testLeftOuterNLJoinExec1() throws IOException, PlanningException {
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec1");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context =  analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    
+    //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof HashLeftOuterJoinExec) {
+      HashLeftOuterJoinExec join = proj.getChild();
+      NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild());
+      proj.setChild(aJoin);
+      exec = proj;
+     
+    }
+
+
+    Tuple tuple;
+    int i = 1;
+    int count = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+       //TODO check contents
+         count = count + 1;
+      
+    }
+    exec.close();
+    assertEquals(5, count);
+  }
+
+  @Test
+  public final void testLeftOuter_NLJoinExec2() throws IOException, PlanningException {
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec2");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context =  analyzer.parse(QUERIES[2]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    
+    //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof HashLeftOuterJoinExec) {
+      HashLeftOuterJoinExec join = proj.getChild();
+      NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild());
+      proj.setChild(aJoin);
+      exec = proj;
+     
+    }
+
+
+    Tuple tuple;
+    int i = 1;
+    int count = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+       //TODO check contents
+         count = count + 1;
+      
+    }
+    exec.close();
+    assertEquals(7, count);
+  }
+
+
+  @Test
+  public final void testLeftOuter_NLJoinExec3() throws IOException, PlanningException {
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec3");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context =  analyzer.parse(QUERIES[3]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    
+    //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof HashLeftOuterJoinExec) {
+      HashLeftOuterJoinExec join = proj.getChild();
+      NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild());
+      proj.setChild(aJoin);
+      exec = proj;
+     
+    }
+
+
+    Tuple tuple;
+    int i = 1;
+    int count = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+       //TODO check contents
+         count = count + 1;
+      
+    }
+    exec.close();
+    assertEquals(7, count);
+  }
+
+    @Test
+  public final void testLeftOuter_NLJoinExec4() throws IOException, PlanningException {
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec4");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context =  analyzer.parse(QUERIES[4]);
+    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    
+    //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof HashLeftOuterJoinExec) {
+      HashLeftOuterJoinExec join = proj.getChild();
+      NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild());
+      proj.setChild(aJoin);
+      exec = proj;
+     
+    }
+
+
+    Tuple tuple;
+    int i = 1;
+    int count = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+       //TODO check contents
+         count = count + 1;
+      
+    }
+    exec.close();
+    assertEquals(0, count);
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
new file mode 100644
index 0000000..0e4fd9a
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
@@ -0,0 +1,193 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestMergeJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestMergeJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+
+  private TableDesc employee;
+  private TableDesc people;
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    Path testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    FileSystem fs = testDir.getFileSystem(conf);
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema employeeSchema = new Schema();
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
+        employeePath);
+    appender.init();
+    Tuple tuple = new VTuple(employeeSchema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
+      appender.addTuple(tuple);
+    }
+    for (int i = 11; i < 20; i+=2) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
+      appender.addTuple(tuple);
+    }
+
+    appender.flush();
+    appender.close();
+    employee = CatalogUtil.newTableDesc("default.employee", employeeSchema, employeeMeta, employeePath);
+    catalog.createTable(employee);
+
+    Schema peopleSchema = new Schema();
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path peoplePath = new Path(testDir, "people.csv");
+    appender = StorageManagerFactory.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender.init();
+    tuple = new VTuple(peopleSchema.size());
+    for (int i = 1; i < 10; i += 2) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
+      appender.addTuple(tuple);
+    }
+    for (int i = 10; i < 20; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
+      appender.addTuple(tuple);
+    }
+
+    appender.flush();
+    appender.close();
+
+    people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
+    catalog.createTable(people);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      "select managerId, e.empId, deptName, e.memId from employee as e inner join " +
+          "people as p on e.empId = p.empId and e.memId = p.fk_memId"
+  };
+
+  @Test
+  public final void testMergeInnerJoin() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalNode root = plan.getRootBlock().getRoot();
+
+    JoinNode joinNode = PlannerUtil.findTopNode(root, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] empFrags = sm.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(), Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = sm.splitNG(conf, "default.p", people.getMeta(), people.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testMergeInnerJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, root);
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof MergeJoinExec);
+
+    Tuple tuple;
+    int count = 0;
+    int i = 1;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      count++;
+      assertTrue(i == tuple.get(0).asInt4());
+      assertTrue(i == tuple.get(1).asInt4());
+      assertTrue(("dept_" + i).equals(tuple.get(2).asChars()));
+      assertTrue((10 + i) == tuple.get(3).asInt4());
+
+      i += 2;
+    }
+    exec.close();
+    assertEquals(10, count); // expected 10 * 5
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
new file mode 100644
index 0000000..120113f
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
@@ -0,0 +1,209 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestNLJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestNLJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+
+  private TableDesc employee;
+  private TableDesc people;
+
+  private MasterPlan masterPlan;
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema schema = new Schema();
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("memid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    appender.init();
+    Tuple tuple = new VTuple(schema.size());
+    for (int i = 0; i < 50; i++) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i)});
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+    employee = CatalogUtil.newTableDesc("default.employee", schema, employeeMeta, employeePath);
+    catalog.createTable(employee);
+    
+    Schema peopleSchema = new Schema();
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path peoplePath = new Path(testDir, "people.csv");
+    appender = StorageManagerFactory.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender.init();
+    tuple = new VTuple(peopleSchema.size());
+    for (int i = 1; i < 50; i += 2) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i)});
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+    
+    people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
+    catalog.createTable(people);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+
+    masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+  
+  String[] QUERIES = {
+    "select managerId, e.empId, deptName, e.memId from employee as e, people p",
+    "select managerId, e.empId, deptName, e.memId from employee as e inner join people as p on " +
+        "e.empId = p.empId and e.memId = p.fk_memId"
+  };
+  
+  @Test
+  public final void testNLCrossJoin() throws IOException, PlanningException {
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(),
+        Integer.MAX_VALUE);
+    
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testNLCrossJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),
+        context).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    int i = 0;
+    exec.init();
+    while (exec.next() != null) {
+      i++;
+    }
+    exec.close();
+    assertEquals(50*50/2, i); // expected 10 * 5
+  }
+
+  @Test
+  public final void testNLInnerJoin() throws IOException, PlanningException {
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(),
+        Integer.MAX_VALUE);
+    
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testNLInnerJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context =  analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),
+        context).getRootBlock().getRoot();
+    //LogicalOptimizer.optimize(ctx, plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    
+    Tuple tuple;
+    int i = 1;
+    int count = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      count++;
+      assertTrue(i == tuple.get(0).asInt4());
+      assertTrue(i == tuple.get(1).asInt4());
+      assertTrue(("dept_" + i).equals(tuple.get(2).asChars()));
+      assertTrue(10 + i == tuple.get(3).asInt4());
+      i += 2;
+    }
+    exec.close();
+    assertEquals(50 / 2, count);
+  }
+}


[26/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
new file mode 100644
index 0000000..c968a73
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -0,0 +1,754 @@
+/**
+ * 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.master;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.exception.NoSuchDatabaseException;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.ClientProtos.*;
+import org.apache.tajo.ipc.TajoMasterClientProtocol;
+import org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService;
+import org.apache.tajo.master.TajoMaster.MasterContext;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.querymaster.QueryJobEvent;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.rm.Worker;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.master.session.InvalidSessionException;
+import org.apache.tajo.master.session.NoSuchSessionVariableException;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.rpc.BlockingRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.ProtoUtil;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.*;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+
+public class TajoMasterClientService extends AbstractService {
+  private final static Log LOG = LogFactory.getLog(TajoMasterClientService.class);
+  private final MasterContext context;
+  private final TajoConf conf;
+  private final CatalogService catalog;
+  private final TajoMasterClientProtocolServiceHandler clientHandler;
+  private BlockingRpcServer server;
+  private InetSocketAddress bindAddress;
+
+  private final BoolProto BOOL_TRUE =
+      BoolProto.newBuilder().setValue(true).build();
+  private final BoolProto BOOL_FALSE =
+      BoolProto.newBuilder().setValue(false).build();
+
+  public TajoMasterClientService(MasterContext context) {
+    super(TajoMasterClientService.class.getName());
+    this.context = context;
+    this.conf = context.getConf();
+    this.catalog = context.getCatalog();
+    this.clientHandler = new TajoMasterClientProtocolServiceHandler();
+  }
+
+  @Override
+  public void start() {
+
+    // start the rpc server
+    String confClientServiceAddr = conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS);
+    InetSocketAddress initIsa = NetUtils.createSocketAddr(confClientServiceAddr);
+    int workerNum = conf.getIntVar(ConfVars.MASTER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM);
+    try {
+      server = new BlockingRpcServer(TajoMasterClientProtocol.class, clientHandler, initIsa, workerNum);
+    } catch (Exception e) {
+      LOG.error(e);
+      throw new RuntimeException(e);
+    }
+    server.start();
+
+    bindAddress = NetUtils.getConnectAddress(server.getListenAddress());
+    this.conf.setVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress));
+    LOG.info("Instantiated TajoMasterClientService at " + this.bindAddress);
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if (server != null) {
+      server.shutdown();
+    }
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddress() {
+    return this.bindAddress;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // TajoMasterClientProtocolService
+  /////////////////////////////////////////////////////////////////////////////
+  public class TajoMasterClientProtocolServiceHandler implements TajoMasterClientProtocolService.BlockingInterface {
+    @Override
+    public CreateSessionResponse createSession(RpcController controller, CreateSessionRequest request)
+        throws ServiceException {
+      try {
+        // create a new session with base database name. If no database name is give, we use default database.
+        String databaseName = request.hasBaseDatabaseName() ? request.getBaseDatabaseName() : DEFAULT_DATABASE_NAME;
+
+        if (!context.getCatalog().existDatabase(databaseName)) {
+          LOG.info("Session creation is canceled due to absent base database \"" + databaseName + "\".");
+          throw new NoSuchDatabaseException(databaseName);
+        }
+
+        String sessionId =
+            context.getSessionManager().createSession(request.getUsername(), databaseName);
+        CreateSessionResponse.Builder builder = CreateSessionResponse.newBuilder();
+        builder.setState(CreateSessionResponse.ResultState.SUCCESS);
+        builder.setSessionId(TajoIdProtos.SessionIdProto.newBuilder().setId(sessionId).build());
+        return builder.build();
+      } catch (NoSuchDatabaseException nsde) {
+        CreateSessionResponse.Builder builder = CreateSessionResponse.newBuilder();
+        builder.setState(CreateSessionResponse.ResultState.FAILED);
+        builder.setMessage(nsde.getMessage());
+        return builder.build();
+      } catch (InvalidSessionException e) {
+        CreateSessionResponse.Builder builder = CreateSessionResponse.newBuilder();
+        builder.setState(CreateSessionResponse.ResultState.FAILED);
+        builder.setMessage(e.getMessage());
+        return builder.build();
+      }
+    }
+
+    @Override
+    public BoolProto removeSession(RpcController controller, TajoIdProtos.SessionIdProto request)
+        throws ServiceException {
+      if (request != null) {
+        context.getSessionManager().removeSession(request.getId());
+      }
+      return ProtoUtil.TRUE;
+    }
+
+    @Override
+    public BoolProto updateSessionVariables(RpcController controller, UpdateSessionVariableRequest request)
+        throws ServiceException {
+      try {
+        String sessionId = request.getSessionId().getId();
+        for (CatalogProtos.KeyValueProto kv : request.getSetVariables().getKeyvalList()) {
+          context.getSessionManager().setVariable(sessionId, kv.getKey(), kv.getValue());
+        }
+        for (String unsetVariable : request.getUnsetVariablesList()) {
+          context.getSessionManager().removeVariable(sessionId, unsetVariable);
+        }
+        return ProtoUtil.TRUE;
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public StringProto getSessionVariable(RpcController controller, SessionedStringProto request)
+        throws ServiceException {
+
+      try {
+        return ProtoUtil.convertString(
+            context.getSessionManager().getVariable(request.getSessionId().getId(), request.getValue()));
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public BoolProto existSessionVariable(RpcController controller, SessionedStringProto request) throws ServiceException {
+      try {
+        String value = context.getSessionManager().getVariable(request.getSessionId().getId(), request.getValue());
+        if (value != null) {
+          return ProtoUtil.TRUE;
+        } else {
+          return ProtoUtil.FALSE;
+        }
+      } catch (NoSuchSessionVariableException nssv) {
+        return ProtoUtil.FALSE;
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public CatalogProtos.KeyValueSetProto getAllSessionVariables(RpcController controller,
+                                                                 TajoIdProtos.SessionIdProto request)
+        throws ServiceException {
+      try {
+        String sessionId = request.getId();
+        Options options = new Options();
+        options.putAll(context.getSessionManager().getAllVariables(sessionId));
+        return options.getProto();
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public StringProto getCurrentDatabase(RpcController controller, TajoIdProtos.SessionIdProto request)
+        throws ServiceException {
+      try {
+        String sessionId = request.getId();
+        return ProtoUtil.convertString(context.getSessionManager().getSession(sessionId).getCurrentDatabase());
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public BoolProto selectDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
+      try {
+        String sessionId = request.getSessionId().getId();
+        String databaseName = request.getValue();
+
+        if (context.getCatalog().existDatabase(databaseName)) {
+          context.getSessionManager().getSession(sessionId).selectDatabase(databaseName);
+          return ProtoUtil.TRUE;
+        } else {
+          throw new ServiceException(new NoSuchDatabaseException(databaseName));
+        }
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public SubmitQueryResponse submitQuery(RpcController controller, QueryRequest request) throws ServiceException {
+
+
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Query [" + request.getQuery() + "] is submitted");
+        }
+        return context.getGlobalEngine().executeQuery(session, request.getQuery());
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        SubmitQueryResponse.Builder responseBuilder = ClientProtos.SubmitQueryResponse.newBuilder();
+        responseBuilder.setUserName(context.getConf().getVar(ConfVars.USERNAME));
+        responseBuilder.setResultCode(ResultCode.ERROR);
+        if (e.getMessage() != null) {
+          responseBuilder.setErrorMessage(ExceptionUtils.getStackTrace(e));
+        } else {
+          responseBuilder.setErrorMessage("Internal Error");
+        }
+        return responseBuilder.build();
+      }
+    }
+
+    @Override
+    public UpdateQueryResponse updateQuery(RpcController controller, QueryRequest request) throws ServiceException {
+
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        UpdateQueryResponse.Builder builder = UpdateQueryResponse.newBuilder();
+        try {
+          context.getGlobalEngine().updateQuery(session, request.getQuery());
+          builder.setResultCode(ResultCode.OK);
+          return builder.build();
+        } catch (Exception e) {
+          builder.setResultCode(ResultCode.ERROR);
+          if (e.getMessage() == null) {
+            builder.setErrorMessage(ExceptionUtils.getStackTrace(e));
+          }
+          return builder.build();
+        }
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public GetQueryResultResponse getQueryResult(RpcController controller,
+                                                 GetQueryResultRequest request) throws ServiceException {
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+        QueryId queryId = new QueryId(request.getQueryId());
+        QueryInProgress queryInProgress = context.getQueryJobManager().getQueryInProgress(queryId);
+
+        // if we cannot get a QueryInProgress instance from QueryJobManager,
+        // the instance can be in the finished query list.
+        if (queryInProgress == null) {
+          queryInProgress = context.getQueryJobManager().getFinishedQuery(queryId);
+        }
+
+        GetQueryResultResponse.Builder builder = GetQueryResultResponse.newBuilder();
+
+        // If we cannot the QueryInProgress instance from the finished list,
+        // the query result was expired due to timeout.
+        // In this case, we will result in error.
+        if (queryInProgress == null) {
+          builder.setErrorMessage("No such query: " + queryId.toString());
+          return builder.build();
+        }
+
+        QueryInfo queryInfo = queryInProgress.getQueryInfo();
+
+        try {
+          //TODO After implementation Tajo's user security feature, Should be modified.
+          builder.setTajoUserName(UserGroupInformation.getCurrentUser().getUserName());
+        } catch (IOException e) {
+          LOG.warn("Can't get current user name");
+        }
+        switch (queryInfo.getQueryState()) {
+          case QUERY_SUCCEEDED:
+            // TODO check this logic needed
+            //builder.setTableDesc((TableDescProto) queryJobManager.getResultDesc().getProto());
+            break;
+          case QUERY_FAILED:
+          case QUERY_ERROR:
+            builder.setErrorMessage("Query " + queryId + " is failed");
+          default:
+            builder.setErrorMessage("Query " + queryId + " is still running");
+        }
+
+        return builder.build();
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public GetQueryListResponse getRunningQueryList(RpcController controller, GetQueryListRequest request)
+
+        throws ServiceException {
+
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+        GetQueryListResponse.Builder builder= GetQueryListResponse.newBuilder();
+
+        Collection<QueryInProgress> queries
+          = context.getQueryJobManager().getRunningQueries();
+
+        BriefQueryInfo.Builder infoBuilder = BriefQueryInfo.newBuilder();
+
+        for (QueryInProgress queryInProgress : queries) {
+          QueryInfo queryInfo = queryInProgress.getQueryInfo();
+
+          infoBuilder.setQueryId(queryInfo.getQueryId().getProto());
+          infoBuilder.setState(queryInfo.getQueryState());
+          infoBuilder.setQuery(queryInfo.getSql());
+          infoBuilder.setStartTime(queryInfo.getStartTime());
+          long endTime = (queryInfo.getFinishTime() == 0) ?
+                         System.currentTimeMillis() : queryInfo.getFinishTime();
+          infoBuilder.setFinishTime(endTime);
+          infoBuilder.setProgress(queryInfo.getProgress());
+          infoBuilder.setQueryMasterPort(queryInfo.getQueryMasterPort());
+          infoBuilder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+
+          builder.addQueryList(infoBuilder.build());
+        }
+
+        GetQueryListResponse result = builder.build();
+        return result;
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public GetQueryListResponse getFinishedQueryList(RpcController controller, GetQueryListRequest request)
+        throws ServiceException {
+
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+        GetQueryListResponse.Builder builder = GetQueryListResponse.newBuilder();
+
+        Collection<QueryInProgress> queries
+            = context.getQueryJobManager().getFinishedQueries();
+
+        BriefQueryInfo.Builder infoBuilder = BriefQueryInfo.newBuilder();
+
+        for (QueryInProgress queryInProgress : queries) {
+          QueryInfo queryInfo = queryInProgress.getQueryInfo();
+
+          infoBuilder.setQueryId(queryInfo.getQueryId().getProto());
+          infoBuilder.setState(queryInfo.getQueryState());
+          infoBuilder.setQuery(queryInfo.getSql());
+          infoBuilder.setStartTime(queryInfo.getStartTime());
+          long endTime = (queryInfo.getFinishTime() == 0) ?
+              System.currentTimeMillis() : queryInfo.getFinishTime();
+          infoBuilder.setFinishTime(endTime);
+          infoBuilder.setProgress(queryInfo.getProgress());
+          infoBuilder.setQueryMasterPort(queryInfo.getQueryMasterPort());
+          infoBuilder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+
+          builder.addQueryList(infoBuilder.build());
+        }
+
+        GetQueryListResponse result = builder.build();
+        return result;
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public GetQueryStatusResponse getQueryStatus(RpcController controller,
+                                                 GetQueryStatusRequest request)
+        throws ServiceException {
+
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+
+        GetQueryStatusResponse.Builder builder = GetQueryStatusResponse.newBuilder();
+        QueryId queryId = new QueryId(request.getQueryId());
+        builder.setQueryId(request.getQueryId());
+
+        if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
+          builder.setResultCode(ResultCode.OK);
+          builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
+        } else {
+          QueryInProgress queryInProgress = context.getQueryJobManager().getQueryInProgress(queryId);
+
+          // It will try to find a query status from a finished query list.
+          if (queryInProgress == null) {
+            queryInProgress = context.getQueryJobManager().getFinishedQuery(queryId);
+          }
+          if (queryInProgress != null) {
+            QueryInfo queryInfo = queryInProgress.getQueryInfo();
+            builder.setResultCode(ResultCode.OK);
+            builder.setState(queryInfo.getQueryState());
+            builder.setProgress(queryInfo.getProgress());
+            builder.setSubmitTime(queryInfo.getStartTime());
+            if(queryInfo.getQueryMasterHost() != null) {
+              builder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+              builder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
+            }
+            if (queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+              builder.setFinishTime(queryInfo.getFinishTime());
+            } else {
+              builder.setFinishTime(System.currentTimeMillis());
+            }
+          } else {
+            builder.setResultCode(ResultCode.ERROR);
+            builder.setErrorMessage("No such query: " + queryId.toString());
+          }
+        }
+        return builder.build();
+
+      } catch (Throwable t) {
+        throw new  ServiceException(t);
+      }
+    }
+
+    /**
+     * It is invoked by TajoContainerProxy.
+     */
+    @Override
+    public BoolProto killQuery(RpcController controller, KillQueryRequest request) throws ServiceException {
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+        QueryId queryId = new QueryId(request.getQueryId());
+        QueryJobManager queryJobManager = context.getQueryJobManager();
+        queryJobManager.getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_KILL,
+            new QueryInfo(queryId)));
+        return BOOL_TRUE;
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public GetClusterInfoResponse getClusterInfo(RpcController controller,
+                                                 GetClusterInfoRequest request)
+        throws ServiceException {
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+        GetClusterInfoResponse.Builder builder= GetClusterInfoResponse.newBuilder();
+
+        Map<String, Worker> workers = context.getResourceManager().getWorkers();
+
+        List<String> wokerKeys = new ArrayList<String>(workers.keySet());
+        Collections.sort(wokerKeys);
+
+        WorkerResourceInfo.Builder workerBuilder
+          = WorkerResourceInfo.newBuilder();
+
+        for(Worker worker: workers.values()) {
+          WorkerResource workerResource = worker.getResource();
+          workerBuilder.setAllocatedHost(worker.getHostName());
+          workerBuilder.setDiskSlots(workerResource.getDiskSlots());
+          workerBuilder.setCpuCoreSlots(workerResource.getCpuCoreSlots());
+          workerBuilder.setMemoryMB(workerResource.getMemoryMB());
+          workerBuilder.setLastHeartbeat(worker.getLastHeartbeatTime());
+          workerBuilder.setUsedMemoryMB(workerResource.getUsedMemoryMB());
+          workerBuilder.setUsedCpuCoreSlots(workerResource.getUsedCpuCoreSlots());
+          workerBuilder.setUsedDiskSlots(workerResource.getUsedDiskSlots());
+          workerBuilder.setWorkerStatus(worker.getState().toString());
+          workerBuilder.setQueryMasterMode(workerResource.isQueryMasterMode());
+          workerBuilder.setTaskRunnerMode(workerResource.isTaskRunnerMode());
+          workerBuilder.setPeerRpcPort(worker.getPeerRpcPort());
+          workerBuilder.setQueryMasterPort(worker.getQueryMasterPort());
+          workerBuilder.setClientPort(worker.getClientPort());
+          workerBuilder.setPullServerPort(worker.getPullServerPort());
+          workerBuilder.setHttpPort(worker.getHttpPort());
+          workerBuilder.setMaxHeap(workerResource.getMaxHeap());
+          workerBuilder.setFreeHeap(workerResource.getFreeHeap());
+          workerBuilder.setTotalHeap(workerResource.getTotalHeap());
+          workerBuilder.setNumRunningTasks(workerResource.getNumRunningTasks());
+          workerBuilder.setNumQueryMasterTasks(workerResource.getNumQueryMasterTasks());
+
+          builder.addWorkerList(workerBuilder.build());
+        }
+
+        return builder.build();
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public BoolProto createDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        if (context.getGlobalEngine().createDatabase(session, request.getValue(), null, false)) {
+          return BOOL_TRUE;
+        } else {
+          return BOOL_FALSE;
+        }
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+    }
+
+    @Override
+    public BoolProto existDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+        if (catalog.existDatabase(request.getValue())) {
+          return BOOL_TRUE;
+        } else {
+          return BOOL_FALSE;
+        }
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+    }
+
+    @Override
+    public BoolProto dropDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+
+        if (context.getGlobalEngine().dropDatabase(session, request.getValue(), false)) {
+          return BOOL_TRUE;
+        } else {
+          return BOOL_FALSE;
+        }
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+    }
+
+    @Override
+    public PrimitiveProtos.StringListProto getAllDatabases(RpcController controller, TajoIdProtos.SessionIdProto
+        request) throws ServiceException {
+      try {
+        context.getSessionManager().touch(request.getId());
+        return ProtoUtil.convertStrings(catalog.getAllDatabaseNames());
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+    }
+
+    @Override
+    public BoolProto existTable(RpcController controller, SessionedStringProto request) throws ServiceException {
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+
+        String databaseName;
+        String tableName;
+        if (CatalogUtil.isFQTableName(request.getValue())) {
+          String [] splitted = CatalogUtil.splitFQTableName(request.getValue());
+          databaseName = splitted[0];
+          tableName = splitted[1];
+        } else {
+          databaseName = session.getCurrentDatabase();
+          tableName = request.getValue();
+        }
+
+        if (catalog.existsTable(databaseName, tableName)) {
+          return BOOL_TRUE;
+        } else {
+          return BOOL_FALSE;
+        }
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+    }
+
+    @Override
+    public GetTableListResponse getTableList(RpcController controller,
+                                             GetTableListRequest request) throws ServiceException {
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        String databaseName;
+        if (request.hasDatabaseName()) {
+          databaseName = request.getDatabaseName();
+        } else {
+          databaseName = session.getCurrentDatabase();
+        }
+        Collection<String> tableNames = catalog.getAllTableNames(databaseName);
+        GetTableListResponse.Builder builder = GetTableListResponse.newBuilder();
+        builder.addAllTables(tableNames);
+        return builder.build();
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public TableResponse getTableDesc(RpcController controller, GetTableDescRequest request) throws ServiceException {
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+
+        String databaseName;
+        String tableName;
+        if (CatalogUtil.isFQTableName(request.getTableName())) {
+          String [] splitted = CatalogUtil.splitFQTableName(request.getTableName());
+          databaseName = splitted[0];
+          tableName = splitted[1];
+        } else {
+          databaseName = session.getCurrentDatabase();
+          tableName = request.getTableName();
+        }
+
+        if (catalog.existsTable(databaseName, tableName)) {
+          return TableResponse.newBuilder()
+              .setResultCode(ResultCode.OK)
+              .setTableDesc(catalog.getTableDesc(databaseName, tableName).getProto())
+              .build();
+        } else {
+          return TableResponse.newBuilder()
+              .setResultCode(ResultCode.ERROR)
+              .setErrorMessage("ERROR: no such a table: " + request.getTableName())
+              .build();
+        }
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public TableResponse createExternalTable(RpcController controller, CreateTableRequest request)
+        throws ServiceException {
+      try {
+        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+
+        Path path = new Path(request.getPath());
+        FileSystem fs = path.getFileSystem(conf);
+
+        if (!fs.exists(path)) {
+          throw new IOException("No such a directory: " + path);
+        }
+
+        Schema schema = new Schema(request.getSchema());
+        TableMeta meta = new TableMeta(request.getMeta());
+        PartitionMethodDesc partitionDesc = null;
+        if (request.hasPartition()) {
+          partitionDesc = new PartitionMethodDesc(request.getPartition());
+        }
+
+        TableDesc desc;
+        try {
+          desc = context.getGlobalEngine().createTableOnPath(session, request.getName(), schema,
+              meta, path, true, partitionDesc, false);
+        } catch (Exception e) {
+          return TableResponse.newBuilder()
+              .setResultCode(ResultCode.ERROR)
+              .setErrorMessage(e.getMessage()).build();
+        }
+
+        return TableResponse.newBuilder()
+            .setResultCode(ResultCode.OK)
+            .setTableDesc(desc.getProto()).build();
+      } catch (InvalidSessionException ise) {
+        return TableResponse.newBuilder()
+            .setResultCode(ResultCode.ERROR)
+            .setErrorMessage(ise.getMessage()).build();
+      } catch (IOException ioe) {
+        return TableResponse.newBuilder()
+            .setResultCode(ResultCode.ERROR)
+            .setErrorMessage(ioe.getMessage()).build();
+      }
+    }
+
+    @Override
+    public BoolProto dropTable(RpcController controller, DropTableRequest dropTable) throws ServiceException {
+      try {
+        Session session = context.getSessionManager().getSession(dropTable.getSessionId().getId());
+        context.getGlobalEngine().dropTable(session, dropTable.getName(), false, dropTable.getPurge());
+        return BOOL_TRUE;
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+
+    @Override
+    public FunctionResponse getFunctionList(RpcController controller, SessionedStringProto request)
+        throws ServiceException {
+
+      try {
+        context.getSessionManager().touch(request.getSessionId().getId());
+
+        String functionName = request.getValue();
+        Collection<FunctionDesc> functions = catalog.getFunctions();
+
+        List<CatalogProtos.FunctionDescProto> functionProtos = new ArrayList<CatalogProtos.FunctionDescProto>();
+
+        for (FunctionDesc eachFunction: functions) {
+          if (functionName == null || functionName.isEmpty()) {
+            functionProtos.add(eachFunction.getProto());
+          } else {
+            if(functionName.equals(eachFunction.getSignature())) {
+              functionProtos.add(eachFunction.getProto());
+            }
+          }
+        }
+        return FunctionResponse.newBuilder()
+            .setResultCode(ResultCode.OK)
+            .addAllFunctions(functionProtos)
+            .build();
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
new file mode 100644
index 0000000..5e9f729
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
@@ -0,0 +1,172 @@
+/**
+ * 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.master;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.rm.Worker;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.AsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.List;
+
+public class TajoMasterService extends AbstractService {
+  private final static Log LOG = LogFactory.getLog(TajoMasterService.class);
+
+  private final TajoMaster.MasterContext context;
+  private final TajoConf conf;
+  private final TajoMasterServiceHandler masterHandler;
+  private AsyncRpcServer server;
+  private InetSocketAddress bindAddress;
+
+  private final BoolProto BOOL_TRUE = BoolProto.newBuilder().setValue(true).build();
+  private final BoolProto BOOL_FALSE = BoolProto.newBuilder().setValue(false).build();
+
+  public TajoMasterService(TajoMaster.MasterContext context) {
+    super(TajoMasterService.class.getName());
+    this.context = context;
+    this.conf = context.getConf();
+    this.masterHandler = new TajoMasterServiceHandler();
+  }
+
+  @Override
+  public void start() {
+    String confMasterServiceAddr = conf.getVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS);
+    InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr);
+    int workerNum = conf.getIntVar(TajoConf.ConfVars.MASTER_RPC_SERVER_WORKER_THREAD_NUM);
+    try {
+      server = new AsyncRpcServer(TajoMasterProtocol.class, masterHandler, initIsa, workerNum);
+    } catch (Exception e) {
+      LOG.error(e);
+    }
+    server.start();
+    bindAddress = NetUtils.getConnectAddress(server.getListenAddress());
+    this.conf.setVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS,
+        NetUtils.normalizeInetSocketAddress(bindAddress));
+    LOG.info("Instantiated TajoMasterService at " + this.bindAddress);
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(server != null) {
+      server.shutdown();
+      server = null;
+    }
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddress() {
+    return bindAddress;
+  }
+
+  public class TajoMasterServiceHandler
+      implements TajoMasterProtocol.TajoMasterProtocolService.Interface {
+    @Override
+    public void heartbeat(
+        RpcController controller,
+        TajoMasterProtocol.TajoHeartbeat request, RpcCallback<TajoMasterProtocol.TajoHeartbeatResponse> done) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Received QueryHeartbeat:" + request.getTajoWorkerHost() + ":" + request.getTajoQueryMasterPort());
+      }
+
+      TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand command = null;
+
+      QueryJobManager queryJobManager = context.getQueryJobManager();
+      command = queryJobManager.queryHeartbeat(request);
+
+      TajoMasterProtocol.TajoHeartbeatResponse.Builder builder = TajoMasterProtocol.TajoHeartbeatResponse.newBuilder();
+      builder.setHeartbeatResult(BOOL_TRUE);
+      if(command != null) {
+        builder.setResponseCommand(command);
+      }
+
+      builder.setClusterResourceSummary(context.getResourceManager().getClusterResourceSummary());
+      done.run(builder.build());
+    }
+
+    @Override
+    public void allocateWorkerResources(
+        RpcController controller,
+        TajoMasterProtocol.WorkerResourceAllocationRequest request,
+        RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> done) {
+      context.getResourceManager().allocateWorkerResources(request, done);
+    }
+
+    @Override
+    public void releaseWorkerResource(RpcController controller,
+                                           TajoMasterProtocol.WorkerResourceReleaseRequest request,
+                                           RpcCallback<PrimitiveProtos.BoolProto> done) {
+      List<YarnProtos.ContainerIdProto> containerIds = request.getContainerIdsList();
+
+      for(YarnProtos.ContainerIdProto eachContainer: containerIds) {
+        context.getResourceManager().releaseWorkerResource(eachContainer);
+      }
+      done.run(BOOL_TRUE);
+    }
+
+    @Override
+    public void stopQueryMaster(RpcController controller, TajoIdProtos.QueryIdProto request,
+                                RpcCallback<BoolProto> done) {
+      context.getQueryJobManager().stopQuery(new QueryId(request));
+      done.run(BOOL_TRUE);
+    }
+
+    @Override
+    public void getAllWorkerResource(RpcController controller, PrimitiveProtos.NullProto request,
+                                     RpcCallback<TajoMasterProtocol.WorkerResourcesRequest> done) {
+
+      TajoMasterProtocol.WorkerResourcesRequest.Builder builder =
+          TajoMasterProtocol.WorkerResourcesRequest.newBuilder();
+      Collection<Worker> workers = context.getResourceManager().getWorkers().values();
+
+      for(Worker worker: workers) {
+        WorkerResource resource = worker.getResource();
+
+        TajoMasterProtocol.WorkerResourceProto.Builder workerResource =
+            TajoMasterProtocol.WorkerResourceProto.newBuilder();
+
+        workerResource.setHost(worker.getHostName());
+        workerResource.setPeerRpcPort(worker.getPeerRpcPort());
+        workerResource.setInfoPort(worker.getHttpPort());
+        workerResource.setQueryMasterPort(worker.getQueryMasterPort());
+        workerResource.setMemoryMB(resource.getMemoryMB());
+        workerResource.setDiskSlots(resource.getDiskSlots());
+        workerResource.setQueryMasterPort(worker.getQueryMasterPort());
+
+        builder.addWorkerResources(workerResource);
+      }
+      done.run(builder.build());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
new file mode 100644
index 0000000..1e6655c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
@@ -0,0 +1,51 @@
+/**
+ * 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.master;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+
+import java.util.Collection;
+
+public class TaskRunnerGroupEvent extends AbstractEvent<EventType> {
+  public enum EventType {
+    CONTAINER_REMOTE_LAUNCH,
+    CONTAINER_REMOTE_CLEANUP
+  }
+
+  protected final ExecutionBlockId executionBlockId;
+  protected final Collection<Container> containers;
+  public TaskRunnerGroupEvent(EventType eventType,
+                              ExecutionBlockId executionBlockId,
+                              Collection<Container> containers) {
+    super(eventType);
+    this.executionBlockId = executionBlockId;
+    this.containers = containers;
+  }
+
+  public Collection<Container> getContainers() {
+    return containers;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerLauncher.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerLauncher.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerLauncher.java
new file mode 100644
index 0000000..9086e65
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerLauncher.java
@@ -0,0 +1,25 @@
+/**
+ * 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.master;
+
+import org.apache.hadoop.yarn.event.EventHandler;
+
+public interface TaskRunnerLauncher extends EventHandler<TaskRunnerGroupEvent> {
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java
new file mode 100644
index 0000000..755df5a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java
@@ -0,0 +1,65 @@
+/*
+ * 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.master;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+
+public class TaskSchedulerContext {
+  private QueryMasterTask.QueryMasterTaskContext masterContext;
+  private boolean isLeafQuery;
+  private ExecutionBlockId blockId;
+  private int taskSize;
+  private int estimatedTaskNum;
+
+  public TaskSchedulerContext(QueryMasterTask.QueryMasterTaskContext masterContext, boolean isLeafQuery,
+                              ExecutionBlockId blockId) {
+    this.masterContext = masterContext;
+    this.isLeafQuery = isLeafQuery;
+    this.blockId = blockId;
+  }
+
+  public QueryMasterTask.QueryMasterTaskContext getMasterContext() {
+    return masterContext;
+  }
+
+  public boolean isLeafQuery() {
+    return isLeafQuery;
+  }
+
+  public ExecutionBlockId getBlockId() {
+    return blockId;
+  }
+
+  public int getTaskSize() {
+    return taskSize;
+  }
+
+  public int getEstimatedTaskNum() {
+    return estimatedTaskNum;
+  }
+
+  public void setTaskSize(int taskSize) {
+    this.taskSize = taskSize;
+  }
+
+  public void setEstimatedTaskNum(int estimatedTaskNum) {
+    this.estimatedTaskNum = estimatedTaskNum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
new file mode 100644
index 0000000..520ecd3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
@@ -0,0 +1,69 @@
+/*
+ * 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.master;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.master.querymaster.SubQuery;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.Map;
+
+public class TaskSchedulerFactory {
+  private static Class<? extends AbstractTaskScheduler> CACHED_ALGORITHM_CLASS;
+  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
+  private static final Class<?>[] DEFAULT_PARAMS = { TaskSchedulerContext.class, SubQuery.class };
+
+  public static Class<? extends AbstractTaskScheduler> getTaskSchedulerClass(Configuration conf)
+      throws IOException {
+    if (CACHED_ALGORITHM_CLASS != null) {
+      return CACHED_ALGORITHM_CLASS;
+    } else {
+      CACHED_ALGORITHM_CLASS = conf.getClass("tajo.querymaster.task-scheduler", null, AbstractTaskScheduler.class);
+    }
+
+    if (CACHED_ALGORITHM_CLASS == null) {
+      throw new IOException("Task scheduler is null");
+    }
+    return CACHED_ALGORITHM_CLASS;
+  }
+
+  public static <T extends AbstractTaskScheduler> T get(Class<T> clazz, TaskSchedulerContext context,
+                                                        SubQuery subQuery) {
+    T result;
+    try {
+      Constructor<T> constructor = (Constructor<T>) CONSTRUCTOR_CACHE.get(clazz);
+      if (constructor == null) {
+        constructor = clazz.getDeclaredConstructor(DEFAULT_PARAMS);
+        constructor.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(clazz, constructor);
+      }
+      result = constructor.newInstance(new Object[]{context, subQuery});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return result;
+  }
+
+  public static AbstractTaskScheduler get(Configuration conf, TaskSchedulerContext context, SubQuery subQuery)
+      throws IOException {
+    return get(getTaskSchedulerClass(conf), context, subQuery);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/TaskState.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskState.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskState.java
new file mode 100644
index 0000000..67d2ebc
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TaskState.java
@@ -0,0 +1,23 @@
+/**
+ * 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.master;
+
+public enum TaskState {
+  NEW, SCHEDULED, RUNNING, SUCCEEDED, FAILED, KILL_WAIT, KILLED;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/YarnContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
new file mode 100644
index 0000000..4f178fb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
@@ -0,0 +1,414 @@
+/**
+ * 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.master;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.pullserver.PullServerAuxService;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedAction;
+import java.util.*;
+
+public class YarnContainerProxy extends ContainerProxy {
+  private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+
+  protected final YarnRPC yarnRPC;
+  final protected String containerMgrAddress;
+  protected Token containerToken;
+
+  public YarnContainerProxy(QueryMasterTask.QueryMasterTaskContext context, Configuration conf, YarnRPC yarnRPC,
+                                  Container container, ExecutionBlockId executionBlockId) {
+    super(context, conf, executionBlockId, container);
+    this.yarnRPC = yarnRPC;
+
+    NodeId nodeId = container.getNodeId();
+    this.containerMgrAddress = nodeId.getHost() + ":" + nodeId.getPort();
+    this.containerToken = container.getContainerToken();
+  }
+
+  protected ContainerManagementProtocol getCMProxy(ContainerId containerID,
+                                                   final String containerManagerBindAddr,
+                                                   Token containerToken)
+      throws IOException {
+    String [] hosts = containerManagerBindAddr.split(":");
+    final InetSocketAddress cmAddr =
+        new InetSocketAddress(hosts[0], Integer.parseInt(hosts[1]));
+    UserGroupInformation user = UserGroupInformation.getCurrentUser();
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      org.apache.hadoop.security.token.Token<ContainerTokenIdentifier> token =
+          ConverterUtils.convertFromYarn(containerToken, cmAddr);
+      // the user in createRemoteUser in this context has to be ContainerID
+      user = UserGroupInformation.createRemoteUser(containerID.toString());
+      user.addToken(token);
+    }
+
+    ContainerManagementProtocol proxy = user.doAs(new PrivilegedAction<ContainerManagementProtocol>() {
+      @Override
+      public ContainerManagementProtocol run() {
+        return (ContainerManagementProtocol) yarnRPC.getProxy(ContainerManagementProtocol.class,
+            cmAddr, conf);
+      }
+    });
+
+    return proxy;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public synchronized void launch(ContainerLaunchContext commonContainerLaunchContext) {
+    LOG.info("Launching Container with Id: " + containerID);
+    if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
+      state = ContainerState.DONE;
+      LOG.error("Container (" + containerID + " was killed before it was launched");
+      return;
+    }
+
+    ContainerManagementProtocol proxy = null;
+    try {
+
+      proxy = getCMProxy(containerID, containerMgrAddress,
+          containerToken);
+
+      // Construct the actual Container
+      ContainerLaunchContext containerLaunchContext = createContainerLaunchContext(commonContainerLaunchContext);
+
+      // Now launch the actual container
+      List<StartContainerRequest> startRequestList = new ArrayList<StartContainerRequest>();
+      StartContainerRequest startRequest = Records
+          .newRecord(StartContainerRequest.class);
+      startRequest.setContainerLaunchContext(containerLaunchContext);
+      startRequestList.add(startRequest);
+      StartContainersRequest startRequests = Records.newRecord(StartContainersRequest.class);
+      startRequests.setStartContainerRequests(startRequestList);
+      StartContainersResponse response = proxy.startContainers(startRequests);
+
+      ByteBuffer portInfo = response.getAllServicesMetaData().get(PullServerAuxService.PULLSERVER_SERVICEID);
+
+      if(portInfo != null) {
+        port = PullServerAuxService.deserializeMetaData(portInfo);
+      }
+
+      LOG.info("PullServer port returned by ContainerManager for "
+          + containerID + " : " + port);
+
+      if(port < 0) {
+        this.state = ContainerState.FAILED;
+        throw new IllegalStateException("Invalid shuffle port number "
+            + port + " returned for " + containerID);
+      }
+
+      this.state = ContainerState.RUNNING;
+      this.hostName = containerMgrAddress.split(":")[0];
+      context.getResourceAllocator().addContainer(containerID, this);
+    } catch (Throwable t) {
+      String message = "Container launch failed for " + containerID + " : "
+          + StringUtils.stringifyException(t);
+      this.state = ContainerState.FAILED;
+      LOG.error(message);
+    } finally {
+      if (proxy != null) {
+        yarnRPC.stopProxy(proxy, conf);
+      }
+    }
+  }
+
+
+  public ContainerLaunchContext createContainerLaunchContext(ContainerLaunchContext commonContainerLaunchContext) {
+    // Setup environment by cloning from common env.
+    Map<String, String> env = commonContainerLaunchContext.getEnvironment();
+    Map<String, String> myEnv = new HashMap<String, String>(env.size());
+    myEnv.putAll(env);
+
+    // Duplicate the ByteBuffers for access by multiple containers.
+    Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
+    for (Map.Entry<String, ByteBuffer> entry : commonContainerLaunchContext.getServiceData().entrySet()) {
+      myServiceData.put(entry.getKey(), entry.getValue().duplicate());
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the local resources
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the necessary command to execute the application master
+    Vector<CharSequence> vargs = new Vector<CharSequence>(30);
+
+    // Set java executable command
+    //LOG.info("Setting up app master command");
+    vargs.add("${JAVA_HOME}" + "/bin/java");
+    // Set Xmx based on am memory size
+    vargs.add("-Xmx2000m");
+    // Set Remote Debugging
+    //if (!context.getQuery().getSubQuery(event.getExecutionBlockId()).isLeafQuery()) {
+    //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
+    //}
+    // Set class name
+    //vargs.add(getRunnerClass());
+    vargs.add(TajoWorker.class.getCanonicalName());
+    vargs.add("tr");     //workerMode
+    vargs.add(getId()); // subqueryId
+    vargs.add(containerMgrAddress); // nodeId
+    vargs.add(containerID.toString()); // containerId
+    Vector<CharSequence> taskParams = getTaskParams();
+    if(taskParams != null) {
+      vargs.addAll(taskParams);
+    }
+
+    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+    // Get final commmand
+    StringBuilder command = new StringBuilder();
+    for (CharSequence str : vargs) {
+      command.append(str).append(" ");
+    }
+
+    LOG.info("Completed setting up TaskRunner command " + command.toString());
+    List<String> commands = new ArrayList<String>();
+    commands.add(command.toString());
+
+    return BuilderUtils.newContainerLaunchContext(commonContainerLaunchContext.getLocalResources(),
+        myEnv,
+        commands,
+        myServiceData,
+        null,
+        new HashMap<ApplicationAccessType, String>());
+  }
+
+  public static ContainerLaunchContext createCommonContainerLaunchContext(Configuration config,
+                                                                          String queryId, boolean isMaster) {
+    TajoConf conf = (TajoConf)config;
+
+    ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class);
+
+    try {
+      ByteBuffer userToken = ByteBuffer.wrap(UserGroupInformation.getCurrentUser().getShortUserName().getBytes());
+      ctx.setTokens(userToken);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the env variables to be setup
+    ////////////////////////////////////////////////////////////////////////////
+    LOG.info("Set the environment for the application master");
+
+    Map<String, String> environment = new HashMap<String, String>();
+    //String initialClassPath = getInitialClasspath(conf);
+    environment.put(ApplicationConstants.Environment.SHELL.name(), "/bin/bash");
+    if(System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()) != null) {
+      environment.put(ApplicationConstants.Environment.JAVA_HOME.name(), System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()));
+    }
+
+    // TODO - to be improved with org.apache.tajo.sh shell script
+    Properties prop = System.getProperties();
+
+    if (prop.getProperty("tajo.test", "FALSE").equalsIgnoreCase("TRUE") ||
+        (System.getenv("tajo.test") != null && System.getenv("tajo.test").equalsIgnoreCase("TRUE"))) {
+      LOG.info("tajo.test is TRUE");
+      environment.put(ApplicationConstants.Environment.CLASSPATH.name(), prop.getProperty("java.class.path", null));
+      environment.put("tajo.test", "TRUE");
+    } else {
+      // Add AppMaster.jar location to classpath
+      // At some point we should not be required to add
+      // the hadoop specific classpaths to the env.
+      // It should be provided out of the box.
+      // For now setting all required classpaths including
+      // the classpath to "." for the application jar
+      StringBuilder classPathEnv = new StringBuilder("./");
+      //for (String c : conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
+      for (String c : YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH) {
+        classPathEnv.append(':');
+        classPathEnv.append(c.trim());
+      }
+
+      classPathEnv.append(":" + System.getenv("TAJO_BASE_CLASSPATH"));
+      classPathEnv.append(":./log4j.properties:./*");
+      if(System.getenv("HADOOP_HOME") != null) {
+        environment.put("HADOOP_HOME", System.getenv("HADOOP_HOME"));
+        environment.put(
+            ApplicationConstants.Environment.HADOOP_COMMON_HOME.name(),
+            System.getenv("HADOOP_HOME"));
+        environment.put(
+            ApplicationConstants.Environment.HADOOP_HDFS_HOME.name(),
+            System.getenv("HADOOP_HOME"));
+        environment.put(
+            ApplicationConstants.Environment.HADOOP_YARN_HOME.name(),
+            System.getenv("HADOOP_HOME"));
+      }
+
+      if(System.getenv("TAJO_BASE_CLASSPATH") != null) {
+        environment.put("TAJO_BASE_CLASSPATH", System.getenv("TAJO_BASE_CLASSPATH"));
+      }
+      environment.put(ApplicationConstants.Environment.CLASSPATH.name(), classPathEnv.toString());
+    }
+
+    ctx.setEnvironment(environment);
+
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("=================================================");
+      for(Map.Entry<String, String> entry: environment.entrySet()) {
+        LOG.debug(entry.getKey() + "=" + entry.getValue());
+      }
+      LOG.debug("=================================================");
+    }
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the local resources
+    ////////////////////////////////////////////////////////////////////////////
+    Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
+    LOG.info("defaultFS: " + conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+
+    try {
+      FileSystem fs = FileSystem.get(conf);
+      FileContext fsCtx = FileContext.getFileContext(conf);
+      Path systemConfPath = TajoConf.getSystemConfPath(conf);
+      if (!fs.exists(systemConfPath)) {
+        LOG.error("system_conf.xml (" + systemConfPath.toString() + ") Not Found");
+      }
+      LocalResource systemConfResource = createApplicationResource(fsCtx, systemConfPath, LocalResourceType.FILE);
+      localResources.put(TajoConstants.SYSTEM_CONF_FILENAME, systemConfResource);
+      ctx.setLocalResources(localResources);
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
+    try {
+      serviceData.put(PullServerAuxService.PULLSERVER_SERVICEID, PullServerAuxService.serializeMetaData(0));
+    } catch (IOException ioe) {
+      LOG.error(ioe);
+    }
+    ctx.setServiceData(serviceData);
+
+    return ctx;
+  }
+
+  private static LocalResource createApplicationResource(FileContext fs,
+                                                         Path p, LocalResourceType type)
+      throws IOException {
+    LocalResource rsrc = recordFactory.newRecordInstance(LocalResource.class);
+    FileStatus rsrcStat = fs.getFileStatus(p);
+    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs
+        .getDefaultFileSystem().resolvePath(rsrcStat.getPath())));
+    rsrc.setSize(rsrcStat.getLen());
+    rsrc.setTimestamp(rsrcStat.getModificationTime());
+    rsrc.setType(type);
+    rsrc.setVisibility(LocalResourceVisibility.APPLICATION);
+    return rsrc;
+  }
+
+  private static void writeConf(Configuration conf, Path queryConfFile)
+      throws IOException {
+    // Write job file to Tajo's fs
+    FileSystem fs = queryConfFile.getFileSystem(conf);
+    FSDataOutputStream out =
+        FileSystem.create(fs, queryConfFile,
+            new FsPermission(QUERYCONF_FILE_PERMISSION));
+    try {
+      conf.writeXml(out);
+    } finally {
+      out.close();
+    }
+  }
+
+  @Override
+  public synchronized void stopContainer() {
+
+    if(isCompletelyDone()) {
+      return;
+    }
+    if(this.state == ContainerState.PREP) {
+      this.state = ContainerState.KILLED_BEFORE_LAUNCH;
+    } else {
+      LOG.info("KILLING " + containerID);
+
+      ContainerManagementProtocol proxy = null;
+      try {
+        proxy = getCMProxy(this.containerID, this.containerMgrAddress,
+            this.containerToken);
+
+        // kill the remote container if already launched
+        List<ContainerId> willBeStopedIds = new ArrayList<ContainerId>();
+        willBeStopedIds.add(this.containerID);
+        StopContainersRequest stopRequests = Records.newRecord(StopContainersRequest.class);
+        stopRequests.setContainerIds(willBeStopedIds);
+        proxy.stopContainers(stopRequests);
+        // If stopContainer returns without an error, assuming the stop made
+        // it over to the NodeManager.
+//          context.getEventHandler().handle(
+//              new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
+        context.getResourceAllocator().removeContainer(containerID);
+      } catch (Throwable t) {
+
+        // ignore the cleanup failure
+        String message = "cleanup failed for container "
+            + this.containerID + " : "
+            + StringUtils.stringifyException(t);
+//          context.getEventHandler().handle(
+//              new AMContainerEventStopFailed(containerID, message));
+        LOG.warn(message);
+        this.state = ContainerState.DONE;
+        return;
+      } finally {
+        if (proxy != null) {
+          yarnRPC.stopProxy(proxy, conf);
+        }
+      }
+      this.state = ContainerState.DONE;
+    }
+  }
+
+  protected Vector<CharSequence> getTaskParams() {
+    String queryMasterHost = context.getQueryMasterContext().getWorkerContext()
+        .getTajoWorkerManagerService().getBindAddr().getHostName();
+    int queryMasterPort = context.getQueryMasterContext().getWorkerContext()
+        .getTajoWorkerManagerService().getBindAddr().getPort();
+
+    Vector<CharSequence> taskParams = new Vector<CharSequence>();
+    taskParams.add(queryMasterHost); // queryMaster hostname
+    taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
+    taskParams.add(context.getStagingDir().toString());
+    return taskParams;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java b/tajo-core/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
new file mode 100644
index 0000000..8b18b5a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
@@ -0,0 +1,200 @@
+/**
+ * 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.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class YarnTaskRunnerLauncherImpl extends AbstractService implements TaskRunnerLauncher {
+
+  /** Class Logger */
+  private static final Log LOG = LogFactory.getLog(YarnTaskRunnerLauncherImpl.class);
+  //private final YarnRPC yarnRPC;
+  private final static RecordFactory recordFactory =
+      RecordFactoryProvider.getRecordFactory(null);
+  private QueryMasterTask.QueryMasterTaskContext context;
+
+  // For ContainerLauncherSpec
+  private static AtomicBoolean initialClasspathFlag = new AtomicBoolean();
+  private static String initialClasspath = null;
+  private static final Object classpathLock = new Object();
+  private ContainerLaunchContext commonContainerSpec = null;
+
+  final public static FsPermission QUERYCONF_FILE_PERMISSION =
+      FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+  /** for launching TaskRunners in parallel */
+  private final ExecutorService executorService;
+
+  private YarnRPC yarnRPC;
+
+  public YarnTaskRunnerLauncherImpl(QueryMasterTask.QueryMasterTaskContext context, YarnRPC yarnRPC) {
+    super(YarnTaskRunnerLauncherImpl.class.getName());
+    this.context = context;
+    this.yarnRPC = yarnRPC;
+    executorService = Executors.newFixedThreadPool(
+        context.getConf().getIntVar(TajoConf.ConfVars.YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
+  }
+
+  public void start() {
+    super.start();
+  }
+
+  public void stop() {
+    executorService.shutdownNow();
+
+    Map<ContainerId, ContainerProxy> containers = context.getResourceAllocator().getContainers();
+    List<ContainerProxy> list = new ArrayList<ContainerProxy>(containers.values());
+    for(ContainerProxy eachProxy:  list) {
+      try {
+        eachProxy.stopContainer();
+      } catch (Exception e) {
+      }
+    }
+    super.stop();
+  }
+
+  @Override
+  public void handle(TaskRunnerGroupEvent event) {
+    if (event.getType() == EventType.CONTAINER_REMOTE_LAUNCH) {
+     launchTaskRunners(event.executionBlockId, event.getContainers());
+    } else if (event.getType() == EventType.CONTAINER_REMOTE_CLEANUP) {
+      stopTaskRunners(event.getContainers());
+    }
+  }
+
+  private void launchTaskRunners(ExecutionBlockId executionBlockId, Collection<Container> containers) {
+    commonContainerSpec = YarnContainerProxy.createCommonContainerLaunchContext(getConfig(),
+        executionBlockId.getQueryId().toString(), false);
+    for (Container container : containers) {
+      final ContainerProxy proxy = new YarnContainerProxy(context, getConfig(),
+          yarnRPC, container, executionBlockId);
+      executorService.submit(new LaunchRunner(container.getId(), proxy));
+    }
+  }
+
+  protected class LaunchRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public LaunchRunner(ContainerId id, ContainerProxy proxy) {
+      this.proxy = proxy;
+      this.id = id;
+    }
+    @Override
+    public void run() {
+      proxy.launch(commonContainerSpec);
+      LOG.info("ContainerProxy started:" + id);
+    }
+  }
+
+  private void stopTaskRunners(Collection<Container> containers) {
+    for (Container container : containers) {
+      final ContainerProxy proxy = context.getResourceAllocator().getContainer(container.getId());
+      executorService.submit(new StopContainerRunner(container.getId(), proxy));
+    }
+  }
+
+  private static class StopContainerRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public StopContainerRunner(ContainerId id, ContainerProxy proxy) {
+      this.id = id;
+      this.proxy = proxy;
+    }
+
+    @Override
+    public void run() {
+      proxy.stopContainer();
+      LOG.info("ContainerProxy stopped:" + id);
+    }
+  }
+
+
+  /**
+   * Lock this on initialClasspath so that there is only one fork in the AM for
+   * getting the initial class-path. TODO: We already construct
+   * a parent CLC and use it for all the containers, so this should go away
+   * once the mr-generated-classpath stuff is gone.
+   */
+  private static String getInitialClasspath(Configuration conf) {
+    synchronized (classpathLock) {
+      if (initialClasspathFlag.get()) {
+        return initialClasspath;
+      }
+      Map<String, String> env = new HashMap<String, String>();
+
+      initialClasspath = env.get(Environment.CLASSPATH.name());
+      initialClasspathFlag.set(true);
+      return initialClasspath;
+    }
+  }
+
+//  public class TaskRunnerContainerProxy extends ContainerProxy {
+//    private final ExecutionBlockId executionBlockId;
+//
+//    public TaskRunnerContainerProxy(QueryMasterTask.QueryContext context, Configuration conf, YarnRPC yarnRPC,
+//                                    Container container, ExecutionBlockId executionBlockId) {
+//      super(context, conf, yarnRPC, container);
+//      this.executionBlockId = executionBlockId;
+//    }
+//
+//    @Override
+//    protected void containerStarted() {
+//      context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
+//    }
+//
+//    @Override
+//    protected String getId() {
+//      return executionBlockId.toString();
+//    }
+//
+//    @Override
+//    protected String getRunnerClass() {
+//      return TaskRunner.class.getCanonicalSignature();
+//    }
+//
+//    @Override
+//    protected Vector<CharSequence> getTaskParams() {
+//      Vector<CharSequence> taskParams = new Vector<CharSequence>();
+//      taskParams.add(queryMasterHost); // queryMaster hostname
+//      taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
+//
+//      return taskParams;
+//    }
+//  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.java b/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.java
new file mode 100644
index 0000000..028af65
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.java
@@ -0,0 +1,123 @@
+/**
+ * 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.master.cluster;
+
+public class ServerName implements Comparable<ServerName> {
+  /**
+   * This character is used as separator between server hostname and port.
+   */
+  public static final String SERVERNAME_SEPARATOR = ":";
+
+  private final String serverName;
+  private final String hostname;
+  private final int port;
+
+
+  public ServerName(final String hostname, final int port) {
+    this.hostname = hostname;
+    this.port = port;
+    this.serverName = getServerName(hostname, port);
+  }
+
+  public ServerName(final String serverName) {
+    this(parseHostname(serverName), parsePort(serverName));
+  }
+  
+  public static ServerName create(final String serverName) {
+	  return new ServerName(serverName);
+  }
+
+  public static ServerName createWithDefaultPort(final String serverName,
+                                                 final int defaultPort) {
+    if (serverName == null || serverName.length() <= 0) {
+      throw new IllegalArgumentException("Passed hostname is null or empty ("
+          + serverName + ")");
+    }
+    int index = serverName.indexOf(SERVERNAME_SEPARATOR);
+    if (index == -1) {
+      return new ServerName(parseHostname(serverName), defaultPort);
+    } else {
+      return new ServerName(parseHostname(serverName), parsePort(serverName));
+    }
+  }
+
+  public static String parseHostname(final String serverName) {
+    if (serverName == null || serverName.length() <= 0) {
+      throw new IllegalArgumentException("Passed hostname is null or empty ("
+          + serverName + ")");
+    }
+    int index = serverName.indexOf(SERVERNAME_SEPARATOR);
+    if (index == -1) { // if a port is missing, the index will be set to -1.
+      throw new IllegalArgumentException("Passed port is missing ("
+          + serverName + ")");
+    }
+    return serverName.substring(0, index);
+  }
+
+  public static int parsePort(final String serverName) {
+    String [] split = serverName.split(SERVERNAME_SEPARATOR);
+    return Integer.parseInt(split[1]);
+  }
+
+  @Override
+  public String toString() {
+    return getServerName();
+  }
+
+  public String getServerName() {
+    return serverName;
+  }
+
+  public String getHostname() {
+    return hostname;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public static String getServerName(String hostName, int port) {
+    final StringBuilder name = new StringBuilder(hostName.length() + 4);
+    name.append(hostName);
+    name.append(SERVERNAME_SEPARATOR);
+    name.append(port);
+    return name.toString();
+  }
+
+  @Override
+  public int compareTo(ServerName other) {
+    int compare = this.getHostname().toLowerCase().
+      compareTo(other.getHostname().toLowerCase());
+    if (compare != 0) return compare;
+    return this.getPort() - other.getPort();        
+  }
+
+  @Override
+  public int hashCode() {
+    return getServerName().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null) return false;
+    if (!(o instanceof ServerName)) return false;
+    return this.compareTo((ServerName)o) == 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
new file mode 100644
index 0000000..c3a9a59
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
@@ -0,0 +1,77 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.ExecutionBlockId;
+
+public class ContainerAllocationEvent extends AbstractEvent<ContainerAllocatorEventType>  {
+
+  private final ExecutionBlockId executionBlockId;
+  private final Priority priority;
+  private final Resource resource;
+  private final boolean isLeafQuery;
+  private final int requiredNum;
+  private final float progress;
+
+  public ContainerAllocationEvent(ContainerAllocatorEventType eventType,
+                                  ExecutionBlockId executionBlockId,
+                                  Priority priority,
+                                  Resource resource,
+                                  int requiredNum,
+                                  boolean isLeafQuery, float progress) {
+    super(eventType);
+    this.executionBlockId = executionBlockId;
+    this.priority = priority;
+    this.resource = resource;
+    this.requiredNum = requiredNum;
+    this.isLeafQuery = isLeafQuery;
+    this.progress = progress;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
+
+  public Priority getPriority() {
+    return priority;
+  }
+
+  public int getRequiredNum() {
+    return requiredNum;
+  }
+
+  public boolean isLeafQuery() {
+    return isLeafQuery;
+  }
+
+  public Resource getCapability() {
+    return resource;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  public Resource getResource() {
+    return resource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocatorEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocatorEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocatorEventType.java
new file mode 100644
index 0000000..4d10efe
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocatorEventType.java
@@ -0,0 +1,26 @@
+/**
+ * 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.master.event;
+
+public enum ContainerAllocatorEventType {
+  // producer: QueryUnitAttempt, consumer: ContainerAllocator
+  CONTAINER_REQ,
+  CONTAINER_DEALLOCATE,
+  CONTAINER_FAILED
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java
new file mode 100644
index 0000000..723ac1a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java
@@ -0,0 +1,37 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.master.event.ContainerEvent.EventType;
+
+public class ContainerEvent extends AbstractEvent<EventType> {
+  public enum EventType {
+    CONTAINER_LAUNCHED,
+    CONTAINER_STOPPED
+  }
+
+  private final ContainerId cId;
+
+  public ContainerEvent(EventType eventType, ContainerId cId) {
+    super(eventType);
+    this.cId = cId;
+  }
+}


[17/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java
new file mode 100644
index 0000000..0973aa7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java
@@ -0,0 +1,198 @@
+/**
+ * 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.worker;
+
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.util.FileUtil;
+
+import java.net.URI;
+import java.util.Collection;
+import java.util.Map;
+
+public class TaskHistory {
+  private long startTime;
+  private long finishTime;
+
+  private String status;
+  private String outputPath;
+  private String workingPath;
+  private float progress;
+
+  private TableStats inputStats;
+  private TableStats outputStats;
+
+  Map<URI, FetcherHistory> fetchers;
+
+  public static class FetcherHistory {
+    private long startTime;
+    private long finishTime;
+
+    private String status;
+    private String uri;
+    private long fileLen;
+    private int messageReceiveCount;
+
+    public long getStartTime() {
+      return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+      this.startTime = startTime;
+    }
+
+    public long getFinishTime() {
+      return finishTime;
+    }
+
+    public void setFinishTime(long finishTime) {
+      this.finishTime = finishTime;
+    }
+
+    public String getStatus() {
+      return status;
+    }
+
+    public void setStatus(String status) {
+      this.status = status;
+    }
+
+    public String getUri() {
+      return uri;
+    }
+
+    public void setUri(String uri) {
+      this.uri = uri;
+    }
+
+    public long getFileLen() {
+      return fileLen;
+    }
+
+    public void setFileLen(long fileLen) {
+      this.fileLen = fileLen;
+    }
+
+    public int getMessageReceiveCount() {
+      return messageReceiveCount;
+    }
+
+    public void setMessageReceiveCount(int messageReceiveCount) {
+      this.messageReceiveCount = messageReceiveCount;
+    }
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  public void setFinishTime(long finishTime) {
+    this.finishTime = finishTime;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  public void setStatus(String status) {
+    this.status = status;
+  }
+
+  public String getOutputPath() {
+    return outputPath;
+  }
+
+  public void setOutputPath(String outputPath) {
+    this.outputPath = outputPath;
+  }
+
+  public String getWorkingPath() {
+    return workingPath;
+  }
+
+  public void setWorkingPath(String workingPath) {
+    this.workingPath = workingPath;
+  }
+
+  public Collection<FetcherHistory> getFetchers() {
+    return fetchers.values();
+  }
+
+  public void setFetchers(Map<URI, FetcherHistory> fetchers) {
+    this.fetchers = fetchers;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  public void setProgress(float progress) {
+    this.progress = progress;
+  }
+
+  public boolean hasFetcher() {
+    return fetchers != null && !fetchers.isEmpty();
+  }
+
+  public TableStats getInputStats() {
+    return inputStats;
+  }
+
+  public void setInputStats(TableStats inputStats) {
+    this.inputStats = inputStats;
+  }
+
+  public TableStats getOutputStats() {
+    return outputStats;
+  }
+
+  public void setOutputStats(TableStats outputStats) {
+    this.outputStats = outputStats;
+  }
+
+  public static String toInputStatsString(TableStats tableStats) {
+    if (tableStats == null) {
+      return "No input statistics";
+    }
+
+    String result = "";
+    result += "TotalBytes: " + FileUtil.humanReadableByteCount(tableStats.getNumBytes(), false) + " ("
+        + tableStats.getNumBytes() + " B)";
+    result += ", ReadBytes: " + FileUtil.humanReadableByteCount(tableStats.getReadBytes(), false) + " ("
+        + tableStats.getReadBytes() + " B)";
+    result += ", ReadRows: " + (tableStats.getNumRows() == 0 ? "-" : tableStats.getNumRows());
+
+    return result;
+  }
+
+  public static String toOutputStatsString(TableStats tableStats) {
+    if (tableStats == null) {
+      return "No output statistics";
+    }
+
+    return tableStats.toJson();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
new file mode 100644
index 0000000..9e904cd
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
@@ -0,0 +1,431 @@
+/**
+ * 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.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoProtos.TaskAttemptState;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.planner.physical.SeqScanExec;
+import org.apache.tajo.engine.query.QueryUnitRequestImpl;
+import org.apache.tajo.engine.utils.TupleCache;
+import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
+import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.*;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.*;
+
+/**
+ * The driver class for Tajo QueryUnit processing.
+ */
+public class TaskRunner extends AbstractService {
+  /** class logger */
+  private static final Log LOG = LogFactory.getLog(TaskRunner.class);
+
+  private TajoConf systemConf;
+
+  private volatile boolean stopped = false;
+
+  private ExecutionBlockId executionBlockId;
+  private QueryId queryId;
+  private NodeId nodeId;
+  private ContainerId containerId;
+
+  // Cluster Management
+  //private TajoWorkerProtocol.TajoWorkerProtocolService.Interface master;
+
+  // for temporal or intermediate files
+  private FileSystem localFS;
+  // for input files
+  private FileSystem defaultFS;
+
+  private TajoQueryEngine queryEngine;
+
+  // for Fetcher
+  private final ExecutorService fetchLauncher;
+  // It keeps all of the query unit attempts while a TaskRunner is running.
+  private final Map<QueryUnitAttemptId, Task> tasks = new ConcurrentHashMap<QueryUnitAttemptId, Task>();
+
+  private final Map<QueryUnitAttemptId, TaskHistory> taskHistories =
+      new ConcurrentHashMap<QueryUnitAttemptId, TaskHistory>();
+
+  private LocalDirAllocator lDirAllocator;
+
+  // A thread to receive each assigned query unit and execute the query unit
+  private Thread taskLauncher;
+
+  // Contains the object references related for TaskRunner
+  private TaskRunnerContext taskRunnerContext;
+  // for the doAs block
+  private UserGroupInformation taskOwner;
+
+  // for the local temporal dir
+  private String baseDir;
+  private Path baseDirPath;
+
+  private TaskRunnerManager taskRunnerManager;
+
+  private long finishTime;
+
+  private RpcConnectionPool connPool;
+
+  private InetSocketAddress qmMasterAddr;
+
+  public TaskRunner(TaskRunnerManager taskRunnerManager, TajoConf conf, String[] args) {
+    super(TaskRunner.class.getName());
+
+    this.taskRunnerManager = taskRunnerManager;
+    this.connPool = RpcConnectionPool.getPool(conf);
+    this.fetchLauncher = Executors.newFixedThreadPool(
+        conf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM));
+    try {
+      final ExecutionBlockId executionBlockId = TajoIdUtils.createExecutionBlockId(args[1]);
+
+      LOG.info("Tajo Root Dir: " + conf.getVar(ConfVars.ROOT_DIR));
+      LOG.info("Worker Local Dir: " + conf.getVar(ConfVars.WORKER_TEMPORAL_DIR));
+
+      UserGroupInformation.setConfiguration(conf);
+
+      // QueryBlockId from String
+      // NodeId has a form of hostname:port.
+      NodeId nodeId = ConverterUtils.toNodeId(args[2]);
+      this.containerId = ConverterUtils.toContainerId(args[3]);
+
+      // QueryMaster's address
+      String host = args[4];
+      int port = Integer.parseInt(args[5]);
+      this.qmMasterAddr = NetUtils.createSocketAddrForHost(host, port);
+
+      LOG.info("QueryMaster Address:" + qmMasterAddr);
+      // TODO - 'load credential' should be implemented
+      // Getting taskOwner
+      UserGroupInformation taskOwner = UserGroupInformation.createRemoteUser(conf.getVar(ConfVars.USERNAME));
+      //taskOwner.addToken(token);
+
+      // initialize MasterWorkerProtocol as an actual task owner.
+//      this.client =
+//          taskOwner.doAs(new PrivilegedExceptionAction<AsyncRpcClient>() {
+//            @Override
+//            public AsyncRpcClient run() throws Exception {
+//              return new AsyncRpcClient(TajoWorkerProtocol.class, masterAddr);
+//            }
+//          });
+//      this.master = client.getStub();
+
+      this.executionBlockId = executionBlockId;
+      this.queryId = executionBlockId.getQueryId();
+      this.nodeId = nodeId;
+      this.taskOwner = taskOwner;
+
+      this.taskRunnerContext = new TaskRunnerContext();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  public String getId() {
+    return executionBlockId + "," + containerId;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.systemConf = (TajoConf)conf;
+
+    try {
+      // initialize DFS and LocalFileSystems
+      defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(conf);
+      localFS = FileSystem.getLocal(conf);
+
+      // the base dir for an output dir
+      baseDir = queryId.toString() + "/output" + "/" + executionBlockId.getId();
+
+      // initialize LocalDirAllocator
+      lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
+
+      baseDirPath = localFS.makeQualified(lDirAllocator.getLocalPathForWrite(baseDir, conf));
+      LOG.info("TaskRunner basedir is created (" + baseDir +")");
+
+      // Setup QueryEngine according to the query plan
+      // Here, we can setup row-based query engine or columnar query engine.
+      this.queryEngine = new TajoQueryEngine(systemConf);
+    } catch (Throwable t) {
+      t.printStackTrace();
+      LOG.error(t);
+    }
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+    run();
+  }
+
+  @Override
+  public void stop() {
+    if(isStopped()) {
+      return;
+    }
+    finishTime = System.currentTimeMillis();
+    // If this flag become true, taskLauncher will be terminated.
+    this.stopped = true;
+
+    // If TaskRunner is stopped, all running or pending tasks will be marked as failed.
+    for (Task task : tasks.values()) {
+      if (task.getStatus() == TaskAttemptState.TA_PENDING ||
+          task.getStatus() == TaskAttemptState.TA_RUNNING) {
+        task.setState(TaskAttemptState.TA_FAILED);
+      }
+    }
+
+    tasks.clear();
+    fetchLauncher.shutdown();
+    this.queryEngine = null;
+
+    TupleCache.getInstance().removeBroadcastCache(executionBlockId);
+
+    LOG.info("Stop TaskRunner: " + executionBlockId);
+    synchronized (this) {
+      notifyAll();
+    }
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  public class TaskRunnerContext {
+    public TajoConf getConf() {
+      return systemConf;
+    }
+
+    public String getNodeId() {
+      return nodeId.toString();
+    }
+
+    public FileSystem getLocalFS() {
+      return localFS;
+    }
+
+    public FileSystem getDefaultFS() {
+      return defaultFS;
+    }
+
+    public LocalDirAllocator getLocalDirAllocator() {
+      return lDirAllocator;
+    }
+
+    public TajoQueryEngine getTQueryEngine() {
+      return queryEngine;
+    }
+
+    public Map<QueryUnitAttemptId, Task> getTasks() {
+      return tasks;
+    }
+
+    public Task getTask(QueryUnitAttemptId taskId) {
+      return tasks.get(taskId);
+    }
+
+    public ExecutorService getFetchLauncher() {
+      return fetchLauncher;
+    }
+
+    public Path getBaseDir() {
+      return baseDirPath;
+    }
+
+    public ExecutionBlockId getExecutionBlockId() {
+      return executionBlockId;
+    }
+
+    public void addTaskHistory(QueryUnitAttemptId quAttemptId, TaskHistory taskHistory) {
+      taskHistories.put(quAttemptId, taskHistory);
+    }
+
+    public TaskHistory getTaskHistory(QueryUnitAttemptId quAttemptId) {
+      return taskHistories.get(quAttemptId);
+    }
+
+    public Map<QueryUnitAttemptId, TaskHistory> getTaskHistories() {
+      return taskHistories;
+    }
+  }
+
+  public TaskRunnerContext getContext() {
+    return taskRunnerContext;
+  }
+
+  static void fatalError(QueryMasterProtocolService.Interface qmClientService,
+                         QueryUnitAttemptId taskAttemptId, String message) {
+    if (message == null) {
+       message = "No error message";
+    }
+    TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder()
+        .setId(taskAttemptId.getProto())
+        .setErrorMessage(message);
+
+    qmClientService.fatalError(null, builder.build(), NullCallback.get());
+  }
+
+  public void run() {
+    LOG.info("TaskRunner startup");
+
+    try {
+
+      taskLauncher = new Thread(new Runnable() {
+        @Override
+        public void run() {
+          int receivedNum = 0;
+          CallFuture<QueryUnitRequestProto> callFuture = null;
+          QueryUnitRequestProto taskRequest = null;
+
+          while(!stopped) {
+            NettyClientBase qmClient = null;
+            QueryMasterProtocolService.Interface qmClientService = null;
+            try {
+              qmClient = connPool.getConnection(qmMasterAddr, QueryMasterProtocol.class, true);
+              qmClientService = qmClient.getStub();
+
+              if (callFuture == null) {
+                callFuture = new CallFuture<QueryUnitRequestProto>();
+                LOG.info("Request GetTask: " + getId());
+                GetTaskRequestProto request = GetTaskRequestProto.newBuilder()
+                    .setExecutionBlockId(executionBlockId.getProto())
+                    .setContainerId(((ContainerIdPBImpl) containerId).getProto())
+                    .build();
+
+                qmClientService.getTask(null, request, callFuture);
+              }
+              try {
+                // wait for an assigning task for 3 seconds
+                taskRequest = callFuture.get(3, TimeUnit.SECONDS);
+              } catch (InterruptedException e) {
+                if(stopped) {
+                  break;
+                }
+              } catch (TimeoutException te) {
+                if(stopped) {
+                  break;
+                }
+                // if there has been no assigning task for a given period,
+                // TaskRunner will retry to request an assigning task.
+                LOG.info("Retry assigning task:" + getId());
+                continue;
+              }
+
+              if (taskRequest != null) {
+                // QueryMaster can send the terminal signal to TaskRunner.
+                // If TaskRunner receives the terminal signal, TaskRunner will be terminated
+                // immediately.
+                if (taskRequest.getShouldDie()) {
+                  LOG.info("Received ShouldDie flag:" + getId());
+                  stop();
+                  if(taskRunnerManager != null) {
+                    //notify to TaskRunnerManager
+                    taskRunnerManager.stopTask(getId());
+                  }
+                } else {
+                  taskRunnerManager.getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc();
+                  LOG.info("Accumulated Received Task: " + (++receivedNum));
+
+                  QueryUnitAttemptId taskAttemptId = new QueryUnitAttemptId(taskRequest.getId());
+                  if (tasks.containsKey(taskAttemptId)) {
+                    LOG.error("Duplicate Task Attempt: " + taskAttemptId);
+                    fatalError(qmClientService, taskAttemptId, "Duplicate Task Attempt: " + taskAttemptId);
+                    continue;
+                  }
+
+                  LOG.info("Initializing: " + taskAttemptId);
+                  Task task;
+                  try {
+                    task = new Task(taskAttemptId, taskRunnerContext, qmClientService,
+                        new QueryUnitRequestImpl(taskRequest));
+                    tasks.put(taskAttemptId, task);
+
+                    task.init();
+                    if (task.hasFetchPhase()) {
+                      task.fetch(); // The fetch is performed in an asynchronous way.
+                    }
+                    // task.run() is a blocking call.
+                    task.run();
+                  } catch (Throwable t) {
+                    LOG.error(t.getMessage(), t);
+                    fatalError(qmClientService, taskAttemptId, t.getMessage());
+                  } finally {
+                    callFuture = null;
+                    taskRequest = null;
+                  }
+                }
+              }
+            } catch (Throwable t) {
+              t.printStackTrace();
+            } finally {
+              connPool.releaseConnection(qmClient);
+            }
+          }
+        }
+      });
+      taskLauncher.start();
+    } catch (Throwable t) {
+      LOG.fatal("Unhandled exception. Starting shutdown.", t);
+    } finally {
+      for (Task t : tasks.values()) {
+        if (t.getStatus() != TaskAttemptState.TA_SUCCEEDED) {
+          t.abort();
+        }
+      }
+    }
+  }
+
+  /**
+   * @return true if a stop has been requested.
+   */
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return this.executionBlockId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
new file mode 100644
index 0000000..da434e4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
@@ -0,0 +1,238 @@
+/**
+ * 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.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.conf.TajoConf;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TaskRunnerManager extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class);
+
+  private final Map<String, TaskRunner> taskRunnerMap = new HashMap<String, TaskRunner>();
+  private final Map<String, TaskRunner> finishedTaskRunnerMap = new HashMap<String, TaskRunner>();
+  private TajoWorker.WorkerContext workerContext;
+  private TajoConf tajoConf;
+  private AtomicBoolean stop = new AtomicBoolean(false);
+  private FinishedTaskCleanThread finishedTaskCleanThread;
+
+  public TaskRunnerManager(TajoWorker.WorkerContext workerContext) {
+    super(TaskRunnerManager.class.getName());
+
+    this.workerContext = workerContext;
+  }
+
+  public TajoWorker.WorkerContext getWorkerContext() {
+    return workerContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    tajoConf = (TajoConf)conf;
+    super.init(tajoConf);
+  }
+
+  @Override
+  public void start() {
+    finishedTaskCleanThread = new FinishedTaskCleanThread();
+    finishedTaskCleanThread.start();
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(stop.get()) {
+      return;
+    }
+    stop.set(true);
+    synchronized(taskRunnerMap) {
+      for(TaskRunner eachTaskRunner: taskRunnerMap.values()) {
+        if(!eachTaskRunner.isStopped()) {
+          eachTaskRunner.stop();
+        }
+      }
+    }
+
+    if(finishedTaskCleanThread != null) {
+      finishedTaskCleanThread.interrupted();
+    }
+    super.stop();
+    if(workerContext.isYarnContainerMode()) {
+      workerContext.stopWorker(true);
+    }
+  }
+
+  public void stopTask(String id) {
+    LOG.info("Stop Task:" + id);
+    synchronized(taskRunnerMap) {
+      TaskRunner taskRunner = taskRunnerMap.remove(id);
+      if(taskRunner != null) {
+        finishedTaskRunnerMap.put(id, taskRunner);
+      }
+    }
+    if(workerContext.isYarnContainerMode()) {
+      stop();
+    }
+  }
+
+  public Collection<TaskRunner> getTaskRunners() {
+    synchronized(taskRunnerMap) {
+      return Collections.unmodifiableCollection(taskRunnerMap.values());
+    }
+  }
+
+  public Collection<TaskRunner> getFinishedTaskRunners() {
+    synchronized(finishedTaskRunnerMap) {
+      return Collections.unmodifiableCollection(finishedTaskRunnerMap.values());
+    }
+  }
+
+  public TaskRunner findTaskRunner(String taskRunnerId) {
+    synchronized(taskRunnerMap) {
+      if(taskRunnerMap.containsKey(taskRunnerId)) {
+        return taskRunnerMap.get(taskRunnerId);
+      }
+    }
+    synchronized(finishedTaskRunnerMap) {
+      return finishedTaskRunnerMap.get(taskRunnerId);
+    }
+  }
+
+  public Task findTaskByQueryUnitAttemptId(QueryUnitAttemptId quAttemptId) {
+    ExecutionBlockId ebid = quAttemptId.getQueryUnitId().getExecutionBlockId();
+    synchronized(taskRunnerMap) {
+      for (TaskRunner eachTaskRunner: taskRunnerMap.values()) {
+        if (eachTaskRunner.getExecutionBlockId().equals(ebid)) {
+          Task task = eachTaskRunner.getContext().getTask(quAttemptId);
+          if (task != null) {
+            return task;
+          }
+        }
+      }
+    }
+    synchronized(finishedTaskRunnerMap) {
+      for (TaskRunner eachTaskRunner: finishedTaskRunnerMap.values()) {
+        if (eachTaskRunner.getExecutionBlockId().equals(ebid)) {
+          Task task = eachTaskRunner.getContext().getTask(quAttemptId);
+          if (task != null) {
+            return task;
+          }
+        }
+      }
+    }
+    return null;
+  }
+
+  public TaskHistory findTaskHistoryByQueryUnitAttemptId(QueryUnitAttemptId quAttemptId) {
+    ExecutionBlockId ebid = quAttemptId.getQueryUnitId().getExecutionBlockId();
+    synchronized(taskRunnerMap) {
+      for (TaskRunner eachTaskRunner: taskRunnerMap.values()) {
+        if (eachTaskRunner.getExecutionBlockId().equals(ebid)) {
+          TaskHistory taskHistory = eachTaskRunner.getContext().getTaskHistory(quAttemptId);
+          if (taskHistory != null) {
+            return taskHistory;
+          }
+        }
+      }
+    }
+    synchronized(finishedTaskRunnerMap) {
+      for (TaskRunner eachTaskRunner: finishedTaskRunnerMap.values()) {
+        if (eachTaskRunner.getExecutionBlockId().equals(ebid)) {
+          TaskHistory taskHistory = eachTaskRunner.getContext().getTaskHistory(quAttemptId);
+          if (taskHistory != null) {
+            return taskHistory;
+          }
+        }
+      }
+    }
+
+    return null;
+  }
+
+  public int getNumTasks() {
+    synchronized(taskRunnerMap) {
+      return taskRunnerMap.size();
+    }
+  }
+
+  public void startTask(final String[] params) {
+    //TODO change to use event dispatcher
+    Thread t = new Thread() {
+      public void run() {
+        try {
+          TajoConf systemConf = new TajoConf(tajoConf);
+          TaskRunner taskRunner = new TaskRunner(TaskRunnerManager.this, systemConf, params);
+          LOG.info("Start TaskRunner:" + taskRunner.getId());
+          synchronized(taskRunnerMap) {
+            taskRunnerMap.put(taskRunner.getId(), taskRunner);
+          }
+          taskRunner.init(systemConf);
+          taskRunner.start();
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+          throw new RuntimeException(e.getMessage(), e);
+        }
+      }
+    };
+
+    t.start();
+  }
+
+  class FinishedTaskCleanThread extends Thread {
+    public void run() {
+      int expireIntervalTime = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HISTORY_EXPIRE_PERIOD);
+      LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime);
+      while(!stop.get()) {
+        try {
+          Thread.sleep(60 * 1000 * 60);   // hourly check
+        } catch (InterruptedException e) {
+          break;
+        }
+        try {
+          long expireTime = System.currentTimeMillis() - expireIntervalTime * 60 * 1000;
+          cleanExpiredFinishedQueryMasterTask(expireTime);
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+        }
+      }
+    }
+
+    private void cleanExpiredFinishedQueryMasterTask(long expireTime) {
+      synchronized(finishedTaskRunnerMap) {
+        List<String> expiredIds = new ArrayList<String>();
+        for(Map.Entry<String, TaskRunner> entry: finishedTaskRunnerMap.entrySet()) {
+          if(entry.getValue().getStartTime() > expireTime) {
+            expiredIds.add(entry.getKey());
+          }
+        }
+
+        for(String eachId: expiredIds) {
+          finishedTaskRunnerMap.remove(eachId);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java b/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java
new file mode 100644
index 0000000..007bcbf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java
@@ -0,0 +1,289 @@
+/**
+ * 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.worker;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoResourceTrackerProtocol;
+import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.storage.v2.DiskDeviceInfo;
+import org.apache.tajo.storage.v2.DiskMountInfo;
+import org.apache.tajo.storage.v2.DiskUtil;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.NodeHeartbeat;
+
+/**
+ * It periodically sends heartbeat to {@link org.apache.tajo.master.rm.TajoResourceTracker} via asynchronous rpc.
+ */
+public class WorkerHeartbeatService extends AbstractService {
+  /** class logger */
+  private final static Log LOG = LogFactory.getLog(WorkerHeartbeatService.class);
+
+  private final TajoWorker.WorkerContext context;
+  private TajoConf systemConf;
+  private RpcConnectionPool connectionPool;
+  private WorkerHeartbeatThread thread;
+  private static final float HDFS_DATANODE_STORAGE_SIZE;
+
+  static {
+    HDFS_DATANODE_STORAGE_SIZE = DiskUtil.getDataNodeStorageSize();
+  }
+
+  public WorkerHeartbeatService(TajoWorker.WorkerContext context) {
+    super(WorkerHeartbeatService.class.getSimpleName());
+    this.context = context;
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    Preconditions.checkArgument(conf instanceof TajoConf, "Configuration must be a TajoConf instance.");
+    this.systemConf = (TajoConf) conf;
+
+    connectionPool = RpcConnectionPool.getPool(systemConf);
+    thread = new WorkerHeartbeatThread();
+    thread.start();
+    super.init(conf);
+  }
+
+  @Override
+  public void serviceStop() {
+    thread.stopped.set(true);
+    synchronized (thread) {
+      thread.notifyAll();
+    }
+    super.stop();
+  }
+
+  class WorkerHeartbeatThread extends Thread {
+    private volatile AtomicBoolean stopped = new AtomicBoolean(false);
+    TajoMasterProtocol.ServerStatusProto.System systemInfo;
+    List<TajoMasterProtocol.ServerStatusProto.Disk> diskInfos =
+        new ArrayList<TajoMasterProtocol.ServerStatusProto.Disk>();
+    float workerDiskSlots;
+    int workerMemoryMB;
+    List<DiskDeviceInfo> diskDeviceInfos;
+
+    public WorkerHeartbeatThread() {
+      int workerCpuCoreNum;
+
+      boolean dedicatedResource = systemConf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DEDICATED);
+      int workerCpuCoreSlots = Runtime.getRuntime().availableProcessors();
+
+      try {
+        diskDeviceInfos = DiskUtil.getDiskDeviceInfos();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+      }
+
+      if(dedicatedResource) {
+        float dedicatedMemoryRatio = systemConf.getFloatVar(TajoConf.ConfVars.WORKER_RESOURCE_DEDICATED_MEMORY_RATIO);
+        int totalMemory = getTotalMemoryMB();
+        workerMemoryMB = (int) ((float) (totalMemory) * dedicatedMemoryRatio);
+        workerCpuCoreNum = Runtime.getRuntime().availableProcessors();
+
+        if(diskDeviceInfos == null) {
+          workerDiskSlots = TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.defaultIntVal;
+        } else {
+          workerDiskSlots = diskDeviceInfos.size();
+        }
+      } else {
+        workerMemoryMB = systemConf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB);
+        workerCpuCoreNum = systemConf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES);
+        workerDiskSlots = systemConf.getFloatVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS);
+
+        if (systemConf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && HDFS_DATANODE_STORAGE_SIZE > 0) {
+          workerDiskSlots = HDFS_DATANODE_STORAGE_SIZE;
+        }
+      }
+
+      systemInfo = TajoMasterProtocol.ServerStatusProto.System.newBuilder()
+          .setAvailableProcessors(workerCpuCoreNum)
+          .setFreeMemoryMB(0)
+          .setMaxMemoryMB(0)
+          .setTotalMemoryMB(getTotalMemoryMB())
+          .build();
+    }
+
+    public void run() {
+      LOG.info("Worker Resource Heartbeat Thread start.");
+      int sendDiskInfoCount = 0;
+      int pullServerPort = 0;
+      if(context.getPullService()!= null) {
+        long startTime = System.currentTimeMillis();
+        while(true) {
+          pullServerPort = context.getPullService().getPort();
+          if(pullServerPort > 0) {
+            break;
+          }
+          //waiting while pull server init
+          try {
+            Thread.sleep(100);
+          } catch (InterruptedException e) {
+          }
+          if(System.currentTimeMillis() - startTime > 30 * 1000) {
+            LOG.fatal("Too long push server init.");
+            System.exit(0);
+          }
+        }
+      }
+
+      String hostName = null;
+      int peerRpcPort = 0;
+      int queryMasterPort = 0;
+      int clientPort = 0;
+
+      if(context.getTajoWorkerManagerService() != null) {
+        hostName = context.getTajoWorkerManagerService().getBindAddr().getHostName();
+        peerRpcPort = context.getTajoWorkerManagerService().getBindAddr().getPort();
+      }
+      if(context.getQueryMasterManagerService() != null) {
+        hostName = context.getQueryMasterManagerService().getBindAddr().getHostName();
+        queryMasterPort = context.getQueryMasterManagerService().getBindAddr().getPort();
+      }
+      if(context.getTajoWorkerClientService() != null) {
+        clientPort = context.getTajoWorkerClientService().getBindAddr().getPort();
+      }
+      if (context.getPullService() != null) {
+        pullServerPort = context.getPullService().getPort();
+      }
+
+      while(!stopped.get()) {
+        if(sendDiskInfoCount == 0 && diskDeviceInfos != null) {
+          getDiskUsageInfos();
+        }
+        TajoMasterProtocol.ServerStatusProto.JvmHeap jvmHeap =
+            TajoMasterProtocol.ServerStatusProto.JvmHeap.newBuilder()
+                .setMaxHeap(Runtime.getRuntime().maxMemory())
+                .setFreeHeap(Runtime.getRuntime().freeMemory())
+                .setTotalHeap(Runtime.getRuntime().totalMemory())
+                .build();
+
+        TajoMasterProtocol.ServerStatusProto serverStatus = TajoMasterProtocol.ServerStatusProto.newBuilder()
+            .addAllDisk(diskInfos)
+            .setRunningTaskNum(
+                context.getTaskRunnerManager() == null ? 1 : context.getTaskRunnerManager().getNumTasks())
+            .setSystem(systemInfo)
+            .setDiskSlots(workerDiskSlots)
+            .setMemoryResourceMB(workerMemoryMB)
+            .setJvmHeap(jvmHeap)
+            .setQueryMasterMode(PrimitiveProtos.BoolProto.newBuilder().setValue(context.isQueryMasterMode()))
+            .setTaskRunnerMode(PrimitiveProtos.BoolProto.newBuilder().setValue(context.isTaskRunnerMode()))
+            .build();
+
+        NodeHeartbeat heartbeatProto = NodeHeartbeat.newBuilder()
+            .setTajoWorkerHost(hostName)
+            .setTajoQueryMasterPort(queryMasterPort)
+            .setPeerRpcPort(peerRpcPort)
+            .setTajoWorkerClientPort(clientPort)
+            .setTajoWorkerHttpPort(context.getHttpPort())
+            .setTajoWorkerPullServerPort(pullServerPort)
+            .setServerStatus(serverStatus)
+            .build();
+
+        NettyClientBase rmClient = null;
+        try {
+          CallFuture<TajoMasterProtocol.TajoHeartbeatResponse> callBack =
+              new CallFuture<TajoMasterProtocol.TajoHeartbeatResponse>();
+
+          rmClient = connectionPool.getConnection(context.getResourceTrackerAddress(), TajoResourceTrackerProtocol.class, true);
+          TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService resourceTracker = rmClient.getStub();
+          resourceTracker.heartbeat(callBack.getController(), heartbeatProto, callBack);
+
+          TajoMasterProtocol.TajoHeartbeatResponse response = callBack.get(2, TimeUnit.SECONDS);
+          if(response != null) {
+            TajoMasterProtocol.ClusterResourceSummary clusterResourceSummary = response.getClusterResourceSummary();
+            if(clusterResourceSummary.getNumWorkers() > 0) {
+              context.setNumClusterNodes(clusterResourceSummary.getNumWorkers());
+            }
+            context.setClusterResource(clusterResourceSummary);
+          } else {
+            if(callBack.getController().failed()) {
+              throw new ServiceException(callBack.getController().errorText());
+            }
+          }
+        } catch (InterruptedException e) {
+          break;
+        } catch (TimeoutException te) {
+          LOG.warn("Heartbeat response is being delayed.");
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+        } finally {
+          connectionPool.releaseConnection(rmClient);
+        }
+
+        try {
+          synchronized (WorkerHeartbeatThread.this){
+            wait(10 * 1000);
+          }
+        } catch (InterruptedException e) {
+          break;
+        }
+        sendDiskInfoCount++;
+
+        if(sendDiskInfoCount > 10) {
+          sendDiskInfoCount = 0;
+        }
+      }
+
+      LOG.info("Worker Resource Heartbeat Thread stopped.");
+    }
+
+    private void getDiskUsageInfos() {
+      diskInfos.clear();
+      for(DiskDeviceInfo eachDevice: diskDeviceInfos) {
+        List<DiskMountInfo> mountInfos = eachDevice.getMountInfos();
+        if(mountInfos != null) {
+          for(DiskMountInfo eachMount: mountInfos) {
+            File eachFile = new File(eachMount.getMountPath());
+            diskInfos.add(TajoMasterProtocol.ServerStatusProto.Disk.newBuilder()
+                .setAbsolutePath(eachFile.getAbsolutePath())
+                .setTotalSpace(eachFile.getTotalSpace())
+                .setFreeSpace(eachFile.getFreeSpace())
+                .setUsableSpace(eachFile.getUsableSpace())
+                .build());
+          }
+        }
+      }
+    }
+  }
+
+  public static int getTotalMemoryMB() {
+    com.sun.management.OperatingSystemMXBean bean =
+        (com.sun.management.OperatingSystemMXBean)
+            java.lang.management.ManagementFactory.getOperatingSystemMXBean();
+    long max = bean.getTotalPhysicalMemorySize();
+    return ((int) (max / (1024 * 1024)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
new file mode 100644
index 0000000..1771255
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
@@ -0,0 +1,117 @@
+/**
+ * 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.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
+import org.apache.tajo.master.TaskRunnerLauncher;
+import org.apache.tajo.master.YarnTaskRunnerLauncherImpl;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.rm.YarnRMContainerAllocator;
+
+public class YarnResourceAllocator extends AbstractResourceAllocator {
+  private YarnRMContainerAllocator rmAllocator;
+
+  private TaskRunnerLauncher taskRunnerLauncher;
+
+  private YarnRPC yarnRPC;
+
+  private YarnClient yarnClient;
+
+  private static final Log LOG = LogFactory.getLog(YarnResourceAllocator.class.getName());
+
+  private QueryMasterTask.QueryMasterTaskContext queryTaskContext;
+
+  private TajoConf systemConf;
+
+  public YarnResourceAllocator(QueryMasterTask.QueryMasterTaskContext queryTaskContext) {
+    this.queryTaskContext = queryTaskContext;
+  }
+
+  @Override
+  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId) {
+    return new ContainerIdPBImpl(containerId);
+  }
+
+  @Override
+  public void allocateTaskWorker() {
+  }
+
+  @Override
+  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext,
+                                           int numTasks,
+                                           int memoryMBPerTask) {
+    int numClusterNodes = workerContext.getNumClusterNodes();
+
+    TajoConf conf =  (TajoConf)workerContext.getQueryMaster().getConfig();
+    int workerNum = conf.getIntVar(TajoConf.ConfVars.YARN_RM_WORKER_NUMBER_PER_NODE);
+    return numClusterNodes == 0 ? numTasks: Math.min(numTasks, numClusterNodes * workerNum);
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    systemConf = (TajoConf)conf;
+
+    yarnRPC = YarnRPC.create(systemConf);
+
+    connectYarnClient();
+
+    taskRunnerLauncher = new YarnTaskRunnerLauncherImpl(queryTaskContext, yarnRPC);
+    addService((Service) taskRunnerLauncher);
+    queryTaskContext.getDispatcher().register(TaskRunnerGroupEvent.EventType.class, taskRunnerLauncher);
+
+    rmAllocator = new YarnRMContainerAllocator(queryTaskContext);
+    addService(rmAllocator);
+    queryTaskContext.getDispatcher().register(ContainerAllocatorEventType.class, rmAllocator);
+    super.init(conf);
+  }
+
+  @Override
+  public void stop() {
+    try {
+      this.yarnClient.stop();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  private void connectYarnClient() {
+    this.yarnClient = new YarnClientImpl();
+    this.yarnClient.init(systemConf);
+    this.yarnClient.start();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/FileAccessForbiddenException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/FileAccessForbiddenException.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/FileAccessForbiddenException.java
new file mode 100644
index 0000000..6c93e4f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/FileAccessForbiddenException.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker.dataserver;
+
+import java.io.IOException;
+
+public class FileAccessForbiddenException extends IOException {
+  private static final long serialVersionUID = -3383272565826389213L;
+
+  public FileAccessForbiddenException() {
+  }
+
+  public FileAccessForbiddenException(String message) {
+    super(message);
+  }
+
+  public FileAccessForbiddenException(Throwable cause) {
+    super(cause);
+  }
+
+  public FileAccessForbiddenException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServer.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServer.java
new file mode 100644
index 0000000..523d6e1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServer.java
@@ -0,0 +1,87 @@
+/**
+ * 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.worker.dataserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.ChannelGroupFuture;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.apache.tajo.worker.dataserver.retriever.DataRetriever;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.Executors;
+
+public class HttpDataServer {
+  private final static Log LOG = LogFactory.getLog(HttpDataServer.class);
+
+  private final InetSocketAddress addr;
+  private InetSocketAddress bindAddr;
+  private ServerBootstrap bootstrap = null;
+  private ChannelFactory factory = null;
+  private ChannelGroup channelGroup = null;
+
+  public HttpDataServer(final InetSocketAddress addr, 
+      final DataRetriever retriever) {
+    this.addr = addr;
+    this.factory = new NioServerSocketChannelFactory(
+        Executors.newCachedThreadPool(), Executors.newCachedThreadPool(),
+        Runtime.getRuntime().availableProcessors() * 2);
+
+    // Configure the server.
+    this.bootstrap = new ServerBootstrap(factory);
+    // Set up the event pipeline factory.
+    this.bootstrap.setPipelineFactory(
+        new HttpDataServerPipelineFactory(retriever));    
+    this.channelGroup = new DefaultChannelGroup();
+  }
+
+  public HttpDataServer(String bindaddr, DataRetriever retriever) {
+    this(NetUtils.createSocketAddr(bindaddr), retriever);
+  }
+
+  public void start() {
+    // Bind and start to accept incoming connections.
+    Channel channel = bootstrap.bind(addr);
+    channelGroup.add(channel);    
+    this.bindAddr = (InetSocketAddress) channel.getLocalAddress();
+    LOG.info("HttpDataServer starts up ("
+        + this.bindAddr.getAddress().getHostAddress() + ":" + this.bindAddr.getPort()
+        + ")");
+  }
+  
+  public InetSocketAddress getBindAddress() {
+    return this.bindAddr;
+  }
+
+  public void stop() {
+    ChannelGroupFuture future = channelGroup.close();
+    future.awaitUninterruptibly();
+    factory.releaseExternalResources();
+
+    LOG.info("HttpDataServer shutdown ("
+        + this.bindAddr.getAddress().getHostAddress() + ":"
+        + this.bindAddr.getPort() + ")");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerHandler.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerHandler.java
new file mode 100644
index 0000000..6b9eea8
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerHandler.java
@@ -0,0 +1,199 @@
+/**
+ * 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.worker.dataserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedFile;
+import org.jboss.netty.util.CharsetUtil;
+import org.apache.tajo.worker.dataserver.retriever.DataRetriever;
+import org.apache.tajo.worker.dataserver.retriever.FileChunk;
+
+import java.io.*;
+import java.net.URLDecoder;
+
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+public class HttpDataServerHandler extends SimpleChannelUpstreamHandler {
+  private final static Log LOG = LogFactory.getLog(HttpDataServer.class);
+  private final DataRetriever retriever;
+
+  public HttpDataServerHandler(DataRetriever retriever) {
+    this.retriever = retriever;
+  }
+
+  @Override
+  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+      throws Exception {
+    HttpRequest request = (HttpRequest) e.getMessage();
+    if (request.getMethod() != GET) {
+      sendError(ctx, METHOD_NOT_ALLOWED);
+      return;
+    }
+
+    FileChunk [] file;
+    try {
+      file = retriever.handle(ctx, request);
+    } catch (FileNotFoundException fnf) {
+      LOG.error(fnf);
+      sendError(ctx, NOT_FOUND);
+      return;
+    } catch (IllegalArgumentException iae) {
+      LOG.error(iae);
+      sendError(ctx, BAD_REQUEST);
+      return;
+    } catch (FileAccessForbiddenException fafe) {
+      LOG.error(fafe);
+      sendError(ctx, FORBIDDEN);
+      return;
+    } catch (IOException ioe) {
+      LOG.error(ioe);
+      sendError(ctx, INTERNAL_SERVER_ERROR);
+      return;
+    }
+
+    // Write the content.
+    Channel ch = e.getChannel();
+    if (file == null) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
+      ch.write(response);
+      if (!isKeepAlive(request)) {
+        ch.close();
+      }
+    }  else {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      long totalSize = 0;
+      for (FileChunk chunk : file) {
+        totalSize += chunk.length();
+      }
+      setContentLength(response, totalSize);
+
+      // Write the initial line and the header.
+      ch.write(response);
+
+      ChannelFuture writeFuture = null;
+
+      for (FileChunk chunk : file) {
+        writeFuture = sendFile(ctx, ch, chunk);
+        if (writeFuture == null) {
+          sendError(ctx, NOT_FOUND);
+          return;
+        }
+      }
+
+      // Decide whether to close the connection or not.
+      if (!isKeepAlive(request)) {
+        // Close the connection when the whole content is written out.
+        writeFuture.addListener(ChannelFutureListener.CLOSE);
+      }
+    }
+  }
+
+  private ChannelFuture sendFile(ChannelHandlerContext ctx, Channel ch, FileChunk file) throws IOException {
+    RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file.getFile(), "r");
+    } catch (FileNotFoundException fnfe) {
+      return null;
+    }
+
+    ChannelFuture writeFuture;
+    if (ch.getPipeline().get(SslHandler.class) != null) {
+      // Cannot use zero-copy with HTTPS.
+      writeFuture = ch.write(new ChunkedFile(raf, file.startOffset(), file.length(), 8192));
+    } else {
+      // No encryption - use zero-copy.
+      final FileRegion region = new DefaultFileRegion(raf.getChannel(), file.startOffset(), file.length());
+      writeFuture = ch.write(region);
+      writeFuture.addListener(new ChannelFutureListener() {
+        public void operationComplete(ChannelFuture future) {
+          region.releaseExternalResources();
+        }
+      });
+    }
+
+    return writeFuture;
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+      throws Exception {
+    Channel ch = e.getChannel();
+    Throwable cause = e.getCause();
+    if (cause instanceof TooLongFrameException) {
+      sendError(ctx, BAD_REQUEST);
+      return;
+    }
+
+    cause.printStackTrace();
+    if (ch.isConnected()) {
+      sendError(ctx, INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  public static String sanitizeUri(String uri) {
+    // Decode the path.
+    try {
+      uri = URLDecoder.decode(uri, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      try {
+        uri = URLDecoder.decode(uri, "ISO-8859-1");
+      } catch (UnsupportedEncodingException e1) {
+        throw new Error();
+      }
+    }
+
+    // Convert file separators.
+    uri = uri.replace('/', File.separatorChar);
+
+    // Simplistic dumb security check.
+    // You will have to do something serious in the production environment.
+    if (uri.contains(File.separator + ".")
+        || uri.contains("." + File.separator) || uri.startsWith(".")
+        || uri.endsWith(".")) {
+      return null;
+    }
+
+    // Convert to absolute path.
+    return uri;
+  }
+
+  private void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
+    HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+    response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+    response.setContent(ChannelBuffers.copiedBuffer(
+        "Failure: " + status.toString() + "\r\n", CharsetUtil.UTF_8));
+
+    // Close the connection as soon as the error message is sent.
+    ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerPipelineFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerPipelineFactory.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerPipelineFactory.java
new file mode 100644
index 0000000..0a47f6b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpDataServerPipelineFactory.java
@@ -0,0 +1,55 @@
+/**
+ * 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.worker.dataserver;
+
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.apache.tajo.worker.dataserver.retriever.DataRetriever;
+
+import static org.jboss.netty.channel.Channels.pipeline;
+
+public class HttpDataServerPipelineFactory implements ChannelPipelineFactory {
+  private final DataRetriever ret;
+
+  public HttpDataServerPipelineFactory(DataRetriever ret) {
+    this.ret = ret;
+  }
+
+  public ChannelPipeline getPipeline() throws Exception {
+    // Create a default pipeline implementation.
+    ChannelPipeline pipeline = pipeline();
+
+    // Uncomment the following line if you want HTTPS
+    // SSLEngine engine =
+    // SecureChatSslContextFactory.getServerContext().createSSLEngine();
+    // engine.setUseClientMode(false);
+    // pipeline.addLast("ssl", new SslHandler(engine));
+
+    pipeline.addLast("decoder", new HttpRequestDecoder());
+    //pipeline.addLast("aggregator", new HttpChunkAggregator(65536));
+    pipeline.addLast("encoder", new HttpResponseEncoder());
+    pipeline.addLast("chunkedWriter", new ChunkedWriteHandler());
+    //pipeline.addLast("deflater", new HttpContentCompressor());
+    pipeline.addLast("handler", new HttpDataServerHandler(ret));
+    return pipeline;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpUtil.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpUtil.java
new file mode 100644
index 0000000..e688c39
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/HttpUtil.java
@@ -0,0 +1,69 @@
+/**
+ * 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.worker.dataserver;
+
+import com.google.common.collect.Maps;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URLEncoder;
+import java.util.Map;
+
+public class HttpUtil {
+  public static Map<String,String> getParams(URI uri) throws UnsupportedEncodingException {
+    return getParamsFromQuery(uri.getQuery());
+  }
+
+  /**
+   * It parses a query string into key/value pairs
+   *
+   * @param queryString decoded query string
+   * @return key/value pairs parsed from a given query string
+   * @throws UnsupportedEncodingException
+   */
+  public static Map<String, String> getParamsFromQuery(String queryString) throws UnsupportedEncodingException {
+    String [] queries = queryString.split("&");
+
+    Map<String,String> params = Maps.newHashMap();
+    String [] param;
+    for (String q : queries) {
+      param = q.split("=");
+      params.put(param[0], param[1]);
+    }
+
+    return params;
+  }
+
+  public static String buildQuery(Map<String,String> params) throws UnsupportedEncodingException {
+    StringBuilder sb = new StringBuilder();
+
+    boolean first = true;
+    for (Map.Entry<String,String> param : params.entrySet()) {
+      if (!first) {
+        sb.append("&");
+      }
+      sb.append(URLEncoder.encode(param.getKey(), "UTF-8")).
+          append("=").
+          append(URLEncoder.encode(param.getValue(), "UTF-8"));
+      first = false;
+    }
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
new file mode 100644
index 0000000..2ef0c4c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
@@ -0,0 +1,128 @@
+/**
+ * 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.worker.dataserver.retriever;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.QueryUnitId;
+import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.worker.dataserver.FileAccessForbiddenException;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class AdvancedDataRetriever implements DataRetriever {
+  private final Log LOG = LogFactory.getLog(AdvancedDataRetriever.class);
+  private final Map<String, RetrieverHandler> handlerMap = Maps.newConcurrentMap();
+
+  public AdvancedDataRetriever() {
+  }
+  
+  public void register(QueryUnitAttemptId id, RetrieverHandler handler) {
+    synchronized (handlerMap) {
+      if (!handlerMap.containsKey(id.toString())) {
+        handlerMap.put(id.toString(), handler);
+      }
+    } 
+  }
+  
+  public void unregister(QueryUnitAttemptId id) {
+    synchronized (handlerMap) {
+      if (handlerMap.containsKey(id.toString())) {
+        handlerMap.remove(id.toString());
+      }
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.tajo.worker.dataserver.retriever.DataRetriever#handle(org.jboss.netty.channel.ChannelHandlerContext, org.jboss.netty.handler.codec.http.HttpRequest)
+   */
+  @Override
+  public FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
+      throws IOException {
+
+    final Map<String, List<String>> params =
+      new QueryStringDecoder(request.getUri()).getParameters();
+
+    if (!params.containsKey("qid")) {
+      throw new FileNotFoundException("No such qid: " + params.containsKey("qid"));
+    }
+
+    if (params.containsKey("sid")) {
+      List<FileChunk> chunks = Lists.newArrayList();
+      List<String> qids = splitMaps(params.get("qid"));
+      for (String qid : qids) {
+        String[] ids = qid.split("_");
+        ExecutionBlockId suid = TajoIdUtils.createExecutionBlockId(params.get("sid").get(0));
+        QueryUnitId quid = new QueryUnitId(suid, Integer.parseInt(ids[0]));
+        QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid,
+            Integer.parseInt(ids[1]));
+        RetrieverHandler handler = handlerMap.get(attemptId.toString());
+        FileChunk chunk = handler.get(params);
+        chunks.add(chunk);
+      }
+      return chunks.toArray(new FileChunk[chunks.size()]);
+    } else {
+      RetrieverHandler handler = handlerMap.get(params.get("qid").get(0));
+      FileChunk chunk = handler.get(params);
+      if (chunk == null) {
+        if (params.containsKey("qid")) { // if there is no content corresponding to the query
+          return null;
+        } else { // if there is no
+          throw new FileNotFoundException("No such a file corresponding to " + params.get("qid"));
+        }
+      }
+
+      File file = chunk.getFile();
+      if (file.isHidden() || !file.exists()) {
+        throw new FileNotFoundException("No such file: " + file.getAbsolutePath());
+      }
+      if (!file.isFile()) {
+        throw new FileAccessForbiddenException(file.getAbsolutePath() + " is not file");
+      }
+
+      return new FileChunk[] {chunk};
+    }
+  }
+
+  private List<String> splitMaps(List<String> qids) {
+    if (null == qids) {
+      LOG.error("QueryUnitId is EMPTY");
+      return null;
+    }
+
+    final List<String> ret = new ArrayList<String>();
+    for (String qid : qids) {
+      Collections.addAll(ret, qid.split(","));
+    }
+    return ret;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DataRetriever.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DataRetriever.java
new file mode 100644
index 0000000..b26ba74
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DataRetriever.java
@@ -0,0 +1,29 @@
+/**
+ * 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.worker.dataserver.retriever;
+
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+
+import java.io.IOException;
+
+public interface DataRetriever {
+  FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DirectoryRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DirectoryRetriever.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DirectoryRetriever.java
new file mode 100644
index 0000000..62dabbd
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/DirectoryRetriever.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker.dataserver.retriever;
+
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.apache.tajo.worker.dataserver.FileAccessForbiddenException;
+import org.apache.tajo.worker.dataserver.HttpDataServerHandler;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class DirectoryRetriever implements DataRetriever {
+  public String baseDir;
+  
+  public DirectoryRetriever(String baseDir) {
+    this.baseDir = baseDir;
+  }
+
+  @Override
+  public FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
+      throws IOException {
+    final String path = HttpDataServerHandler.sanitizeUri(request.getUri());
+    if (path == null) {
+      throw new IllegalArgumentException("Wrong path: " +path);
+    }
+
+    File file = new File(baseDir, path);
+    if (file.isHidden() || !file.exists()) {
+      throw new FileNotFoundException("No such file: " + baseDir + "/" + path);
+    }
+    if (!file.isFile()) {
+      throw new FileAccessForbiddenException("No such file: " 
+          + baseDir + "/" + path); 
+    }
+    
+    return new FileChunk[] {new FileChunk(file, 0, file.length())};
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/FileChunk.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/FileChunk.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/FileChunk.java
new file mode 100644
index 0000000..4f11168
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/FileChunk.java
@@ -0,0 +1,51 @@
+/**
+ * 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.worker.dataserver.retriever;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+
+public class FileChunk {
+  private final File file;
+  private final long startOffset;
+  private final long length;
+
+  public FileChunk(File file, long startOffset, long length) throws FileNotFoundException {
+    this.file = file;
+    this.startOffset = startOffset;
+    this.length = length;
+  }
+
+  public File getFile() {
+    return this.file;
+  }
+
+  public long startOffset() {
+    return this.startOffset;
+  }
+
+  public long length() {
+    return this.length;
+  }
+
+  public String toString() {
+    return " (start=" + startOffset() + ", length=" + length + ") "
+        + file.getAbsolutePath();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/RetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/RetrieverHandler.java b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/RetrieverHandler.java
new file mode 100644
index 0000000..e5479cc
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/dataserver/retriever/RetrieverHandler.java
@@ -0,0 +1,33 @@
+/**
+ * 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.worker.dataserver.retriever;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+public interface RetrieverHandler {
+  /**
+   *
+   * @param kvs url-decoded key/value pairs
+   * @return a desired part of a file
+   * @throws IOException
+   */
+  public FileChunk get(Map<String, List<String>> kvs) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/proto/InternalTypes.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/InternalTypes.proto b/tajo-core/src/main/proto/InternalTypes.proto
new file mode 100644
index 0000000..1a62bc2
--- /dev/null
+++ b/tajo-core/src/main/proto/InternalTypes.proto
@@ -0,0 +1,32 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tajo";
+option java_outer_classname = "InternalTypes";
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
+
+message AvgLongProto {
+  required int64 sum = 1;
+  required int64 count = 2;
+}
+
+message AvgDoubleProto {
+  required double sum = 1;
+  required int64 count = 2;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/proto/QueryMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto
new file mode 100644
index 0000000..e12c9aa
--- /dev/null
+++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto
@@ -0,0 +1,42 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "QueryMasterProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "yarn_protos.proto";
+import "tajo_protos.proto";
+import "TajoIdProtos.proto";
+import "CatalogProtos.proto";
+import "PrimitiveProtos.proto";
+import "TajoWorkerProtocol.proto";
+
+service QueryMasterProtocolService {
+  //from Worker
+  rpc getTask(GetTaskRequestProto) returns (QueryUnitRequestProto);
+  rpc statusUpdate (TaskStatusProto) returns (BoolProto);
+  rpc ping (QueryUnitAttemptIdProto) returns (BoolProto);
+  rpc fatalError(TaskFatalErrorReport) returns (BoolProto);
+  rpc done (TaskCompletionReport) returns (BoolProto);
+
+  //from TajoMaster's QueryJobManager
+  rpc killQuery(QueryIdProto) returns (BoolProto);
+  rpc executeQuery(QueryExecutionRequestProto) returns (BoolProto);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/proto/ResourceTrackerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto
new file mode 100644
index 0000000..d46d09a
--- /dev/null
+++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "TajoResourceTrackerProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "TajoMasterProtocol.proto";
+
+message NodeHeartbeat {
+  required string tajoWorkerHost = 1;
+  required int32 peerRpcPort = 2;
+  required int32 tajoQueryMasterPort = 3;
+  optional ServerStatusProto serverStatus = 4;
+  optional int32 tajoWorkerClientPort = 5;
+  optional string statusMessage = 6;
+  optional int32 tajoWorkerPullServerPort = 7;
+  optional int32 tajoWorkerHttpPort = 8;
+}
+
+service TajoResourceTrackerProtocolService {
+  rpc heartbeat(NodeHeartbeat) returns (TajoHeartbeatResponse);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/proto/TajoMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/TajoMasterProtocol.proto b/tajo-core/src/main/proto/TajoMasterProtocol.proto
new file mode 100644
index 0000000..8fccbaf
--- /dev/null
+++ b/tajo-core/src/main/proto/TajoMasterProtocol.proto
@@ -0,0 +1,154 @@
+/**
+ * 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.
+ */
+
+//TajoWorker -> TajoMaster protocol
+
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "TajoMasterProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "yarn_protos.proto";
+import "tajo_protos.proto";
+import "TajoIdProtos.proto";
+import "CatalogProtos.proto";
+import "PrimitiveProtos.proto";
+
+message ServerStatusProto {
+    message System {
+        required int32 availableProcessors = 1;
+        required int32 freeMemoryMB = 2;
+        required int32 maxMemoryMB = 3;
+        required int32 totalMemoryMB = 4;
+    }
+    message Disk {
+        required string absolutePath = 1;
+        required int64 totalSpace = 2;
+        required int64 freeSpace = 3;
+        required int64 usableSpace = 4;
+    }
+
+    message JvmHeap {
+        required int64 maxHeap = 1;
+        required int64 totalHeap = 2;
+        required int64 freeHeap = 3;
+    }
+
+    required System system = 1;
+    required float diskSlots = 2;
+    required int32 memoryResourceMB = 3;
+    repeated Disk disk = 4;
+    required int32 runningTaskNum = 5;
+    required JvmHeap jvmHeap = 6;
+    required BoolProto queryMasterMode = 7;
+    required BoolProto taskRunnerMode = 8;
+}
+
+message TajoHeartbeat {
+  required string tajoWorkerHost = 1;
+  required int32 tajoQueryMasterPort = 2;
+  optional int32 tajoWorkerClientPort = 3;
+  optional QueryIdProto queryId = 4;
+  optional QueryState state = 5;
+  optional string statusMessage = 6;
+  optional float queryProgress = 7;
+  optional int64 queryFinishTime = 8;
+}
+
+message TajoHeartbeatResponse {
+  message ResponseCommand {
+      required string command = 1;
+      repeated string params = 2;
+  }
+  required BoolProto heartbeatResult = 1;
+  required ClusterResourceSummary clusterResourceSummary = 2;
+  optional ResponseCommand responseCommand = 3;
+}
+
+message ClusterResourceSummary {
+  required int32 numWorkers = 1;
+  required int32 totalDiskSlots = 2;
+  required int32 totalCpuCoreSlots = 3;
+  required int32 totalMemoryMB = 4;
+
+  required int32 totalAvailableDiskSlots = 5;
+  required int32 totalAvailableCpuCoreSlots = 6;
+  required int32 totalAvailableMemoryMB = 7;
+}
+
+enum ResourceRequestPriority {
+    MEMORY = 1;
+    DISK = 2;
+}
+
+message WorkerResourceAllocationRequest {
+    required QueryIdProto queryId = 1;
+    required ResourceRequestPriority resourceRequestPriority = 2;
+
+    required int32 numContainers = 3;
+
+    required int32 maxMemoryMBPerContainer = 4;
+    required int32 minMemoryMBPerContainer = 5;
+
+    required float maxDiskSlotPerContainer = 6;
+    required float minDiskSlotPerContainer = 7;
+}
+
+message WorkerResourceProto {
+    required string host = 1;
+    required int32 peerRpcPort = 2;
+    required int32 queryMasterPort = 3;
+    required int32 infoPort = 4;
+    required int32 memoryMB = 5 ;
+    required float diskSlots = 6;
+}
+
+message WorkerResourcesRequest {
+    repeated WorkerResourceProto workerResources = 1;
+}
+
+message WorkerResourceReleaseRequest {
+    required ExecutionBlockIdProto executionBlockId = 1;
+    repeated hadoop.yarn.ContainerIdProto containerIds = 2;
+}
+
+message WorkerAllocatedResource {
+    required hadoop.yarn.ContainerIdProto containerId = 1;
+    required string nodeId = 2;
+    required string workerHost = 3;
+    required int32 peerRpcPort = 4;
+    required int32 queryMasterPort = 5;
+    required int32 clientPort = 6;
+    required int32 workerPullServerPort = 7;
+
+    required int32 allocatedMemoryMB = 8;
+    required float allocatedDiskSlots = 9;
+}
+
+message WorkerResourceAllocationResponse {
+    required QueryIdProto queryId = 1;
+    repeated WorkerAllocatedResource workerAllocatedResource = 2;
+}
+
+service TajoMasterProtocolService {
+  rpc heartbeat(TajoHeartbeat) returns (TajoHeartbeatResponse);
+  rpc allocateWorkerResources(WorkerResourceAllocationRequest) returns (WorkerResourceAllocationResponse);
+  rpc releaseWorkerResource(WorkerResourceReleaseRequest) returns (BoolProto);
+  rpc stopQueryMaster(QueryIdProto) returns (BoolProto);
+  rpc getAllWorkerResource(NullProto) returns (WorkerResourcesRequest);
+}
\ No newline at end of file


[05/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
new file mode 100644
index 0000000..b56ab47
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
@@ -0,0 +1,400 @@
+/**
+ * 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.planner.global;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalOptimizer;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static junit.framework.Assert.assertNotNull;
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestBroadcastJoinPlan {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestBroadcastJoinPlan";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private Path testDir;
+
+  private TableDesc smallTable1;
+  private TableDesc smallTable2;
+  private TableDesc smallTable3;
+  private TableDesc largeTable1;
+  private TableDesc largeTable2;
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    conf = util.getConfiguration();
+    conf.setLongVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD, 500 * 1024);
+    conf.setBoolVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO, true);
+
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog = util.startCatalogCluster().getCatalog();
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    util.getMiniCatalogCluster().getCatalogServer().reloadBuiltinFunctions(TajoMaster.initBuiltinFunctions());
+
+    Schema smallTable1Schema = new Schema();
+    smallTable1Schema.addColumn("small1_id", TajoDataTypes.Type.INT4);
+    smallTable1Schema.addColumn("small1_contents", TajoDataTypes.Type.TEXT);
+    smallTable1 = makeTestData("default.small1", smallTable1Schema, 10 * 1024);
+
+    Schema smallTable2Schema = new Schema();
+    smallTable2Schema.addColumn("small2_id", TajoDataTypes.Type.INT4);
+    smallTable2Schema.addColumn("small2_contents", TajoDataTypes.Type.TEXT);
+    smallTable2 = makeTestData("default.small2", smallTable2Schema, 10 * 1024);
+
+    Schema smallTable3Schema = new Schema();
+    smallTable3Schema.addColumn("small3_id", TajoDataTypes.Type.INT4);
+    smallTable3Schema.addColumn("small3_contents", TajoDataTypes.Type.TEXT);
+    smallTable3 = makeTestData("default.small3", smallTable3Schema, 10 * 1024);
+
+    Schema largeTable1Schema = new Schema();
+    largeTable1Schema.addColumn("large1_id", TajoDataTypes.Type.INT4);
+    largeTable1Schema.addColumn("large1_contents", TajoDataTypes.Type.TEXT);
+    largeTable1 = makeTestData("default.large1", largeTable1Schema, 1024 * 1024);  //1M
+
+    Schema largeTable2Schema = new Schema();
+    largeTable2Schema.addColumn("large2_id", TajoDataTypes.Type.INT4);
+    largeTable2Schema.addColumn("large2_contents", TajoDataTypes.Type.TEXT);
+    largeTable2 = makeTestData("default.large2", largeTable2Schema, 1024 * 1024);  //1M
+
+    catalog.createTable(smallTable1);
+    catalog.createTable(smallTable2);
+    catalog.createTable(largeTable1);
+    catalog.createTable(largeTable2);
+
+    analyzer = new SQLAnalyzer();
+  }
+
+  private TableDesc makeTestData(String tableName, Schema schema, int dataSize) throws Exception {
+    TableMeta tableMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV);
+    Path dataPath = new Path(testDir, tableName + ".csv");
+
+    String contentsData = "";
+    for (int i = 0; i < 1000; i++) {
+      for (int j = 0; j < 10; j++) {
+        contentsData += j;
+      }
+    }
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(tableMeta, schema,
+        dataPath);
+    appender.init();
+    Tuple tuple = new VTuple(schema.size());
+    int writtenSize = 0;
+    int count = 0;
+    while (true) {
+      TextDatum textDatum = DatumFactory.createText(count + "_" + contentsData);
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(count), textDatum });
+      appender.addTuple(tuple);
+
+      writtenSize += textDatum.size();
+      if (writtenSize >= dataSize) {
+        break;
+      }
+    }
+
+    appender.flush();
+    appender.close();
+
+    TableDesc tableDesc = CatalogUtil.newTableDesc(tableName, schema, tableMeta, dataPath);
+    TableStats tableStats = new TableStats();
+    FileSystem fs = dataPath.getFileSystem(conf);
+    tableStats.setNumBytes(fs.getFileStatus(dataPath).getLen());
+
+    tableDesc.setStats(tableStats);
+
+    return tableDesc;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  @Test
+  public final void testBroadcastJoin() throws IOException, PlanningException {
+    String query = "select count(*) from large1 " +
+        "join small1 on large1_id = small1_id " +
+        "join small2 on small1_id = small2_id";
+
+    LogicalPlanner planner = new LogicalPlanner(catalog);
+    LogicalOptimizer optimizer = new LogicalOptimizer(conf);
+    Expr expr =  analyzer.parse(query);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+
+    optimizer.optimize(plan);
+
+    QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
+    QueryContext queryContext = new QueryContext();
+    MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+    GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
+    globalPlanner.build(masterPlan);
+
+    /*
+    |-eb_1395714781593_0000_000007 (TERMINAL)
+        |-eb_1395714781593_0000_000006 (ROOT)
+            |-eb_1395714781593_0000_000005 (LEAF)
+    */
+
+    ExecutionBlock terminalEB = masterPlan.getRoot();
+    assertEquals(1, masterPlan.getChildCount(terminalEB.getId()));
+
+    ExecutionBlock rootEB = masterPlan.getChild(terminalEB.getId(), 0);
+    assertEquals(1, masterPlan.getChildCount(rootEB.getId()));
+
+    ExecutionBlock leafEB = masterPlan.getChild(rootEB.getId(), 0);
+    assertNotNull(leafEB);
+
+    assertEquals(0, masterPlan.getChildCount(leafEB.getId()));
+    Collection<String> broadcastTables = leafEB.getBroadcastTables();
+    assertEquals(2, broadcastTables.size());
+
+    assertTrue(broadcastTables.contains("default.small1"));
+    assertTrue(broadcastTables.contains("default.small2"));
+    assertTrue(!broadcastTables.contains("default.large1"));
+
+    LogicalNode leafNode = leafEB.getPlan();
+    assertEquals(NodeType.GROUP_BY, leafNode.getType());
+
+    LogicalNode joinNode = ((GroupbyNode)leafNode).getChild();
+    assertEquals(NodeType.JOIN, joinNode.getType());
+
+    LogicalNode leftNode = ((JoinNode)joinNode).getLeftChild();
+    LogicalNode rightNode = ((JoinNode)joinNode).getRightChild();
+
+    assertEquals(NodeType.JOIN, leftNode.getType());
+    assertEquals(NodeType.SCAN, rightNode.getType());
+
+    LogicalNode lastLeftNode = ((JoinNode)leftNode).getLeftChild();
+    LogicalNode lastRightNode = ((JoinNode)leftNode).getRightChild();
+
+    assertEquals(NodeType.SCAN, lastLeftNode.getType());
+    assertEquals(NodeType.SCAN, lastRightNode.getType());
+  }
+
+  @Test
+  public final void testNotBroadcastJoinTwoLargeTable() throws IOException, PlanningException {
+    // This query is not broadcast join
+    String query = "select count(*) from large1 " +
+        "join large2 on large1_id = large2_id ";
+
+    LogicalPlanner planner = new LogicalPlanner(catalog);
+    LogicalOptimizer optimizer = new LogicalOptimizer(conf);
+    Expr expr =  analyzer.parse(query);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+
+    optimizer.optimize(plan);
+
+    QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
+    QueryContext queryContext = new QueryContext();
+    MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+    GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
+    globalPlanner.build(masterPlan);
+
+    ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan);
+    while (ebCursor.hasNext()) {
+      ExecutionBlock eb = ebCursor.nextBlock();
+      Collection<String> broadcastTables = eb.getBroadcastTables();
+      assertTrue(broadcastTables == null || broadcastTables.isEmpty());
+    }
+  }
+
+  @Test
+  public final void testTwoBroadcastJoin() throws IOException, PlanningException {
+    String query = "select count(*) from large1 " +
+        "join small1 on large1_id = small1_id " +
+        "join large2 on large1_id = large2_id " +
+        "join small2 on large2_id = small2_id";
+
+    LogicalPlanner planner = new LogicalPlanner(catalog);
+    LogicalOptimizer optimizer = new LogicalOptimizer(conf);
+    Expr expr =  analyzer.parse(query);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+
+    optimizer.optimize(plan);
+
+    QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
+    QueryContext queryContext = new QueryContext();
+    MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+    GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
+    globalPlanner.build(masterPlan);
+
+    /*
+    |-eb_1395736346625_0000_000009
+      |-eb_1395736346625_0000_000008 (GROUP-BY)
+         |-eb_1395736346625_0000_000007 (GROUP-BY, JOIN)
+           |-eb_1395736346625_0000_000006 (LEAF, JOIN)
+           |-eb_1395736346625_0000_000003 (LEAF, JOIN)
+     */
+
+    ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan);
+    int index = 0;
+    while (ebCursor.hasNext()) {
+      ExecutionBlock eb = ebCursor.nextBlock();
+      if(index == 0) {
+        Collection<String> broadcastTables = eb.getBroadcastTables();
+        assertEquals(1, broadcastTables.size());
+
+        assertTrue(!broadcastTables.contains("default.large1"));
+        assertTrue(broadcastTables.contains("default.small1"));
+      } else if(index == 1) {
+        Collection<String> broadcastTables = eb.getBroadcastTables();
+        assertEquals(1, broadcastTables.size());
+        assertTrue(!broadcastTables.contains("default.large2"));
+        assertTrue(broadcastTables.contains("default.small2"));
+      }
+      index++;
+    }
+
+    assertEquals(5, index);
+  }
+
+  @Test
+  public final void testNotBroadcastJoinSubquery() throws IOException, PlanningException {
+    // This query is not broadcast join;
+    String query = "select count(*) from large1 " +
+        "join (select * from small1) a on large1_id = a.small1_id " +
+        "join small2 on a.small1_id = small2_id";
+
+    LogicalPlanner planner = new LogicalPlanner(catalog);
+    LogicalOptimizer optimizer = new LogicalOptimizer(conf);
+    Expr expr =  analyzer.parse(query);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+
+    optimizer.optimize(plan);
+
+    QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
+    QueryContext queryContext = new QueryContext();
+    MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+    GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
+    globalPlanner.build(masterPlan);
+
+    /*
+    |-eb_1395749810370_0000_000007
+       |-eb_1395749810370_0000_000006 (GROUP-BY)
+          |-eb_1395749810370_0000_000005 (GROUP-BY, JOIN)
+             |-eb_1395749810370_0000_000004 (LEAF, SCAN, large1)
+             |-eb_1395749810370_0000_000003 (JOIN)
+                |-eb_1395749810370_0000_000002 (LEAF, SCAN, small2)
+                |-eb_1395749810370_0000_000001 (LEAF, TABLE_SUBQUERY, small1)
+     */
+
+    ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan);
+    int index = 0;
+    while (ebCursor.hasNext()) {
+      ExecutionBlock eb = ebCursor.nextBlock();
+      Collection<String> broadcastTables = eb.getBroadcastTables();
+      assertTrue(broadcastTables == null || broadcastTables.isEmpty());
+      index++;
+    }
+
+    assertEquals(7, index);
+  }
+
+  @Test
+  public final void testBroadcastJoinSubquery() throws IOException, PlanningException {
+    String query = "select count(*) from large1 " +
+        "join (select * from small1) a on large1_id = a.small1_id " +
+        "join small2 on large1_id = small2_id";
+
+    LogicalPlanner planner = new LogicalPlanner(catalog);
+    LogicalOptimizer optimizer = new LogicalOptimizer(conf);
+    Expr expr =  analyzer.parse(query);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+
+    optimizer.optimize(plan);
+
+    QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
+    QueryContext queryContext = new QueryContext();
+    MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+    GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
+    globalPlanner.build(masterPlan);
+
+    /*
+    |-eb_1395794091662_0000_000007
+       |-eb_1395794091662_0000_000006
+          |-eb_1395794091662_0000_000005 (JOIN)
+             |-eb_1395794091662_0000_000004 (LEAF, SUBQUERY)
+             |-eb_1395794091662_0000_000003 (LEAF, JOIN)
+     */
+
+    ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan);
+    int index = 0;
+    while (ebCursor.hasNext()) {
+      ExecutionBlock eb = ebCursor.nextBlock();
+      if(index == 0) {
+        //LEAF, JOIN
+        Collection<String> broadcastTables = eb.getBroadcastTables();
+        assertEquals(1, broadcastTables.size());
+
+        assertTrue(!broadcastTables.contains("default.large1"));
+        assertTrue(broadcastTables.contains("default.small2"));
+      } else if(index == 1) {
+        //LEAF, SUBQUERY
+        Collection<String> broadcastTables = eb.getBroadcastTables();
+        assertTrue(broadcastTables == null || broadcastTables.isEmpty());
+      } else if(index == 2) {
+        //JOIN
+        Collection<String> broadcastTables = eb.getBroadcastTables();
+        assertTrue(broadcastTables == null || broadcastTables.isEmpty());
+      }
+      index++;
+    }
+
+    assertEquals(5, index);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java
new file mode 100644
index 0000000..ab56bea
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java
@@ -0,0 +1,53 @@
+/**
+ * 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.planner.global;
+
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestMasterPlan {
+
+  @Test
+  public void testConnect() {
+    MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
+
+    ExecutionBlock eb1 = masterPlan.newExecutionBlock();
+    ExecutionBlock eb2 = masterPlan.newExecutionBlock();
+    ExecutionBlock eb3 = masterPlan.newExecutionBlock();
+
+    masterPlan.addConnect(eb1, eb2, TajoWorkerProtocol.ShuffleType.RANGE_SHUFFLE);
+    assertTrue(masterPlan.isConnected(eb1.getId(), eb2.getId()));
+    assertTrue(masterPlan.isReverseConnected(eb2.getId(), eb1.getId()));
+
+    masterPlan.addConnect(eb3, eb2, TajoWorkerProtocol.ShuffleType.RANGE_SHUFFLE);
+    assertTrue(masterPlan.isConnected(eb1.getId(), eb2.getId()));
+    assertTrue(masterPlan.isConnected(eb3.getId(), eb2.getId()));
+
+    assertTrue(masterPlan.isReverseConnected(eb2.getId(), eb1.getId()));
+    assertTrue(masterPlan.isReverseConnected(eb2.getId(), eb3.getId()));
+
+    masterPlan.disconnect(eb3, eb2);
+    assertFalse(masterPlan.isConnected(eb3, eb2));
+    assertFalse(masterPlan.isReverseConnected(eb2, eb3));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
new file mode 100644
index 0000000..c79796b
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
@@ -0,0 +1,217 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestBNLJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestBNLJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+
+  private static int OUTER_TUPLE_NUM = 1000;
+  private static int INNER_TUPLE_NUM = 1000;
+
+  private TableDesc employee;
+  private TableDesc people;
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema schema = new Schema();
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("memid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    appender.init();
+    Tuple tuple = new VTuple(schema.size());
+    for (int i = 0; i < OUTER_TUPLE_NUM; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+    employee = CatalogUtil.newTableDesc("default.employee", schema, employeeMeta, employeePath);
+    catalog.createTable(employee);
+
+    Schema peopleSchema = new Schema();
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path peoplePath = new Path(testDir, "people.csv");
+    appender = StorageManagerFactory.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender.init();
+    tuple = new VTuple(peopleSchema.size());
+    for (int i = 1; i < INNER_TUPLE_NUM; i += 2) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+
+    people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
+    catalog.createTable(people);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  // employee (managerId, empId, memId, deptName)
+  // people (empId, fk_memId, name, age)
+  String[] QUERIES = {
+      "select managerId, e.empId, deptName, e.memId from employee as e, people p",
+      "select managerId, e.empId, deptName, e.memId from employee as e " +
+          "inner join people as p on e.empId = p.empId and e.memId = p.fk_memId" };
+
+  @Test
+  public final void testBNLCrossJoin() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN);
+
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testBNLCrossJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof BNLJoinExec);
+
+    int i = 0;
+    exec.init();
+    while (exec.next() != null) {
+      i++;
+    }
+    exec.close();
+    assertEquals(OUTER_TUPLE_NUM * INNER_TUPLE_NUM / 2, i); // expected 10 * 5
+  }
+
+  @Test
+  public final void testBNLInnerJoin() throws IOException, PlanningException {
+    Expr context = analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),
+        context).getRootBlock().getRoot();
+
+    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(), people.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
+
+
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testBNLInnerJoin");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(),
+        merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof BNLJoinExec);
+
+    Tuple tuple;
+    int i = 1;
+    int count = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      count++;
+      assertTrue(i == tuple.get(0).asInt4());
+      assertTrue(i == tuple.get(1).asInt4());
+      assertTrue(("dept_" + i).equals(tuple.get(2).asChars()));
+      assertTrue(10 + i == tuple.get(3).asInt4());
+      i += 2;
+    }
+    exec.close();
+    assertEquals(INNER_TUPLE_NUM / 2, count);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
new file mode 100644
index 0000000..a47bde3
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
@@ -0,0 +1,210 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalOptimizer;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Random;
+import java.util.Stack;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+
+public class TestBSTIndexExec {
+
+  private TajoConf conf;
+  private Path idxPath;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private LogicalOptimizer optimizer;
+  private AbstractStorageManager sm;
+  private Schema idxSchema;
+  private TupleComparator comp;
+  private BSTIndex.BSTIndexWriter writer;
+  private HashMap<Integer , Integer> randomValues ;
+  private int rndKey = -1;
+  private FileSystem fs;
+  private TableMeta meta;
+  private Path tablePath;
+
+  private Random rnd = new Random(System.currentTimeMillis());
+
+  private TajoTestingCluster util;
+
+  @Before
+  public void setup() throws Exception {
+    this.randomValues = new HashMap<Integer, Integer>();
+    this.conf = new TajoConf();
+    util = new TajoTestingCluster();
+    util.startCatalogCluster();
+    catalog = util.getMiniCatalogCluster().getCatalog();
+
+    Path workDir = CommonTestingUtil.getTestDir();
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, workDir.toUri().toString());
+    catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    sm = StorageManagerFactory.getStorageManager(conf, workDir);
+
+    idxPath = new Path(workDir, "test.idx");
+
+    Schema schema = new Schema();
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
+
+    this.idxSchema = new Schema();
+    idxSchema.addColumn("managerid", Type.INT4);
+    SortSpec[] sortKeys = new SortSpec[1];
+    sortKeys[0] = new SortSpec(idxSchema.getColumn("managerid"), true, false);
+    this.comp = new TupleComparator(idxSchema, sortKeys);
+
+    this.writer = new BSTIndex(conf).getIndexWriter(idxPath,
+        BSTIndex.TWO_LEVEL_INDEX, this.idxSchema, this.comp);
+    writer.setLoadNum(100);
+    writer.open();
+    long offset;
+
+    meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    tablePath = StorageUtil.concatPath(workDir, "employee", "table.csv");
+    fs = tablePath.getFileSystem(conf);
+    fs.mkdirs(tablePath.getParent());
+
+    FileAppender appender = (FileAppender)StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema,
+        tablePath);
+    appender.init();
+    Tuple tuple = new VTuple(schema.size());
+    for (int i = 0; i < 10000; i++) {
+      
+      Tuple key = new VTuple(this.idxSchema.size());
+      int rndKey = rnd.nextInt(250);
+      if(this.randomValues.containsKey(rndKey)) {
+        int t = this.randomValues.remove(rndKey) + 1;
+        this.randomValues.put(rndKey, t);
+      } else {
+        this.randomValues.put(rndKey, 1);
+      }
+      
+      key.put(new Datum[] { DatumFactory.createInt4(rndKey) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(rndKey),
+          DatumFactory.createInt4(rnd.nextInt(10)),
+          DatumFactory.createText("dept_" + rnd.nextInt(10)) });
+      offset = appender.getOffset();
+      appender.addTuple(tuple);
+      writer.write(key, offset);
+    }
+    appender.flush();
+    appender.close();
+    writer.close();
+
+    TableDesc desc = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta,
+        sm.getTablePath("employee"));
+    catalog.createTable(desc);
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+    optimizer = new LogicalOptimizer(conf);
+  }
+
+  @After
+  public void tearDown() {
+    util.shutdownCatalogCluster();
+  }
+
+  @Test
+  public void testEqual() throws Exception {
+    if(conf.getBoolean("tajo.storage.manager.v2", false)) {
+      return;
+    }
+    this.rndKey = rnd.nextInt(250);
+    final String QUERY = "select * from employee where managerId = " + rndKey;
+    
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEqual");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
+    Expr expr = analyzer.parse(QUERY);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    TmpPlanner phyPlanner = new TmpPlanner(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    int tupleCount = this.randomValues.get(rndKey);
+    int counter = 0;
+    exec.init();
+    while (exec.next() != null) {
+      counter ++;
+    }
+    exec.close();
+    assertEquals(tupleCount , counter);
+  }
+
+  private class TmpPlanner extends PhysicalPlannerImpl {
+    public TmpPlanner(TajoConf conf, AbstractStorageManager sm) {
+      super(conf, sm);
+    }
+
+    @Override
+    public PhysicalExec createScanPlan(TaskAttemptContext ctx, ScanNode scanNode, Stack<LogicalNode> stack)
+        throws IOException {
+      Preconditions.checkNotNull(ctx.getTable(scanNode.getTableName()),
+          "Error: There is no table matched to %s", scanNode.getTableName());
+
+      List<FileFragment> fragments = FragmentConvertor.convert(ctx.getConf(), meta.getStoreType(),
+          ctx.getTables(scanNode.getTableName()));
+      
+      Datum[] datum = new Datum[]{DatumFactory.createInt4(rndKey)};
+
+      return new BSTIndexScanExec(ctx, sm, scanNode, fragments.get(0), idxPath, idxSchema, comp , datum);
+
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
new file mode 100644
index 0000000..ff3befe
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
@@ -0,0 +1,184 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Random;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestExternalSortExec {
+  private TajoConf conf;
+  private TajoTestingCluster util;
+  private final String TEST_PATH = "target/test-data/TestExternalSortExec";
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+
+  private final int numTuple = 100000;
+  private Random rnd = new Random(System.currentTimeMillis());
+
+  private TableDesc employee;
+
+  @Before
+  public void setUp() throws Exception {
+    this.conf = new TajoConf();
+    util = new TajoTestingCluster();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf.setVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR, testDir.toString());
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema schema = new Schema();
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    appender.enableStats();
+    appender.init();
+    Tuple tuple = new VTuple(schema.size());
+    for (int i = 0; i < numTuple; i++) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(rnd.nextInt(50)),
+          DatumFactory.createInt4(rnd.nextInt(100)),
+          DatumFactory.createText("dept_" + i),
+      });
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+
+    System.out.println(appender.getStats().getNumRows() + " rows (" + (appender.getStats().getNumBytes() / 1048576) +
+        " MB)");
+
+    employee = new TableDesc("default.employee", schema, employeeMeta, employeePath);
+    catalog.createTable(employee);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    CommonTestingUtil.cleanupTestDir(TEST_PATH);
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      "select managerId, empId from employee order by managerId, empId"
+  };
+
+  @Test
+  public final void testNext() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    
+    ProjectionExec proj = (ProjectionExec) exec;
+
+    // TODO - should be planed with user's optimization hint
+    if (!(proj.getChild() instanceof ExternalSortExec)) {
+      UnaryPhysicalExec sortExec = proj.getChild();
+      SeqScanExec scan = sortExec.getChild();
+
+      ExternalSortExec extSort = new ExternalSortExec(ctx, sm,
+          ((MemSortExec)sortExec).getPlan(), scan);
+      proj.setChild(extSort);
+    }
+
+    Tuple tuple;
+    Tuple preVal = null;
+    Tuple curVal;
+    int cnt = 0;
+    exec.init();
+    long start = System.currentTimeMillis();
+    TupleComparator comparator = new TupleComparator(proj.getSchema(),
+        new SortSpec[]{
+            new SortSpec(new Column("managerid", Type.INT4)),
+            new SortSpec(new Column("empid", Type.INT4))
+        });
+
+    while ((tuple = exec.next()) != null) {
+      curVal = tuple;
+      if (preVal != null) {
+        assertTrue("prev: " + preVal + ", but cur: " + curVal, comparator.compare(preVal, curVal) <= 0);
+      }
+      preVal = curVal;
+      cnt++;
+    }
+    long end = System.currentTimeMillis();
+    assertEquals(numTuple, cnt);
+
+    // for rescan test
+    preVal = null;
+    exec.rescan();
+    cnt = 0;
+    while ((tuple = exec.next()) != null) {
+      curVal = tuple;
+      if (preVal != null) {
+        assertTrue("prev: " + preVal + ", but cur: " + curVal, comparator.compare(preVal, curVal) <= 0);
+      }
+      preVal = curVal;
+      cnt++;
+    }
+    assertEquals(numTuple, cnt);
+    exec.close();
+    System.out.println("Sort Time: " + (end - start) + " msc");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
new file mode 100644
index 0000000..b05688d
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
@@ -0,0 +1,403 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.junit.Assert.*;
+
+public class TestFullOuterHashJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestFullOuterHashJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+  private static Session session = LocalTajoTestingUtility.createDummySession();
+
+  private TableDesc dep3;
+  private TableDesc job3;
+  private TableDesc emp3;
+  private TableDesc phone3;
+
+  private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3");
+  private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3");
+  private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3");
+  private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3");
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    //----------------- dep3 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    Schema dep3Schema = new Schema();
+    dep3Schema.addColumn("dep_id", Type.INT4);
+    dep3Schema.addColumn("dep_name", Type.TEXT);
+    dep3Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep3Path = new Path(testDir, "dep3.csv");
+    Appender appender1 = StorageManagerFactory.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    appender1.init();
+    Tuple tuple = new VTuple(dep3Schema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createText("dept_" + i),
+          DatumFactory.createInt4(1000 + i) });
+      appender1.addTuple(tuple);
+    }
+
+    appender1.flush();
+    appender1.close();
+    dep3 = CatalogUtil.newTableDesc(DEP3_NAME, dep3Schema, dep3Meta, dep3Path);
+    catalog.createTable(dep3);
+
+    //----------------- job3 ------------------------------
+    //  job_id  | job_title
+    // ----------------------
+    //   101    |  job_101
+    //   102    |  job_102
+    //   103    |  job_103
+
+    Schema job3Schema = new Schema();
+    job3Schema.addColumn("job_id", Type.INT4);
+    job3Schema.addColumn("job_title", Type.TEXT);
+
+
+    TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path job3Path = new Path(testDir, "job3.csv");
+    Appender appender2 = StorageManagerFactory.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    appender2.init();
+    Tuple tuple2 = new VTuple(job3Schema.size());
+    for (int i = 1; i < 4; i++) {
+      int x = 100 + i;
+      tuple2.put(new Datum[] { DatumFactory.createInt4(100 + i),
+          DatumFactory.createText("job_" + x) });
+      appender2.addTuple(tuple2);
+    }
+
+    appender2.flush();
+    appender2.close();
+    job3 = CatalogUtil.newTableDesc(JOB3_NAME, job3Schema, job3Meta, job3Path);
+    catalog.createTable(job3);
+
+
+
+    //---------------------emp3 --------------------
+    // emp_id  | first_name | last_name | dep_id | salary | job_id
+    // ------------------------------------------------------------
+    //  11     |  fn_11     |  ln_11    |  1     | 123    | 101
+    //  13     |  fn_13     |  ln_13    |  3     | 369    | 103
+    //  15     |  fn_15     |  ln_15    |  5     | 615    | null
+    //  17     |  fn_17     |  ln_17    |  7     | 861    | null
+    //  19     |  fn_19     |  ln_19    |  9     | 1107   | null
+    //  21     |  fn_21     |  ln_21    |  1     | 123    | 101
+    //  23     |  fn_23     |  ln_23    |  3     | 369    | 103
+
+    Schema emp3Schema = new Schema();
+    emp3Schema.addColumn("emp_id", Type.INT4);
+    emp3Schema.addColumn("first_name", Type.TEXT);
+    emp3Schema.addColumn("last_name", Type.TEXT);
+    emp3Schema.addColumn("dep_id", Type.INT4);
+    emp3Schema.addColumn("salary", Type.FLOAT4);
+    emp3Schema.addColumn("job_id", Type.INT4);
+
+
+    TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path emp3Path = new Path(testDir, "emp3.csv");
+    Appender appender3 = StorageManagerFactory.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    appender3.init();
+    Tuple tuple3 = new VTuple(emp3Schema.size());
+
+    for (int i = 1; i < 4; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+
+      int y = 20 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(20 + i),
+          DatumFactory.createText("firstname_" + y),
+          DatumFactory.createText("lastname_" + y),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+    }
+
+    for (int i = 5; i < 10; i += 2) {
+      int x= 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createNullDatum() });
+      appender3.addTuple(tuple3);
+    }
+
+    appender3.flush();
+    appender3.close();
+    emp3 = CatalogUtil.newTableDesc(EMP3_NAME, emp3Schema, emp3Meta, emp3Path);
+    catalog.createTable(emp3);
+
+    //---------------------phone3 --------------------
+    // emp_id  | phone_number
+    // -----------------------------------------------
+    // this table is empty, no rows
+
+    Schema phone3Schema = new Schema();
+    phone3Schema.addColumn("emp_id", Type.INT4);
+    phone3Schema.addColumn("phone_number", Type.TEXT);
+
+
+    TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path phone3Path = new Path(testDir, "phone3.csv");
+    Appender appender5 = StorageManagerFactory.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
+        phone3Path);
+    appender5.init();
+
+    appender5.flush();
+    appender5.close();
+    phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path);
+    catalog.createTable(phone3);
+
+
+
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      // [0] no nulls
+      "select dep3.dep_id, dep_name, emp_id, salary from dep3 full outer join emp3 on dep3.dep_id = emp3.dep_id",
+      // [1] nulls on the right operand
+      "select job3.job_id, job_title, emp_id, salary from job3 full outer join emp3 on job3.job_id=emp3.job_id",
+      // [2] nulls on the left side
+      "select job3.job_id, job_title, emp_id, salary from emp3 full outer join job3 on job3.job_id=emp3.job_id",
+      // [3] one operand is empty
+      "select emp3.emp_id, first_name, phone_number from emp3 full outer join phone3 on emp3.emp_id = phone3.emp_id"
+  };
+
+  @Test
+  public final void testFullOuterHashJoinExec0() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
+
+    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuterHashJoinExec0");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof HashFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(12, count);
+
+  }
+
+
+  @Test
+  public final void testFullOuterHashJoinExec1() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
+
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuter_HashJoinExec1");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof HashFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(8, count);
+
+  }
+
+  @Test
+  public final void testFullOuterHashJoinExec2() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[2]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
+
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuterHashJoinExec2");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof HashFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(8, count);
+
+  }
+
+
+  @Test
+  public final void testFullOuterHashJoinExec3() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[3]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
+
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuterHashJoinExec3");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(), merged,
+        workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof HashFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(7, count);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
new file mode 100644
index 0000000..0386179
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
@@ -0,0 +1,536 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.junit.Assert.*;
+
+public class TestFullOuterMergeJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestFullOuterMergeJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+  private static final Session session = LocalTajoTestingUtility.createDummySession();
+
+  private TableDesc dep3;
+  private TableDesc dep4;
+  private TableDesc job3;
+  private TableDesc emp3;
+  private TableDesc phone3;
+
+  private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3");
+  private final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4");
+  private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3");
+  private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3");
+  private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3");
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    //----------------- dep3 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    Schema dep3Schema = new Schema();
+    dep3Schema.addColumn("dep_id", Type.INT4);
+    dep3Schema.addColumn("dep_name", Type.TEXT);
+    dep3Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep3Path = new Path(testDir, "dep3.csv");
+    Appender appender1 = StorageManagerFactory.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    appender1.init();
+    Tuple tuple = new VTuple(dep3Schema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createText("dept_" + i),
+          DatumFactory.createInt4(1000 + i) });
+      appender1.addTuple(tuple);
+    }
+
+    appender1.flush();
+    appender1.close();
+    dep3 = CatalogUtil.newTableDesc(DEP3_NAME, dep3Schema, dep3Meta, dep3Path);
+    catalog.createTable(dep3);
+
+
+    //----------------- dep4 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    // 10     | dep_10    | 1010
+    Schema dep4Schema = new Schema();
+    dep4Schema.addColumn("dep_id", Type.INT4);
+    dep4Schema.addColumn("dep_name", Type.TEXT);
+    dep4Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep4Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep4Path = new Path(testDir, "dep4.csv");
+    Appender appender4 = StorageManagerFactory.getStorageManager(conf).getAppender(dep4Meta, dep4Schema, dep4Path);
+    appender4.init();
+    Tuple tuple4 = new VTuple(dep4Schema.size());
+    for (int i = 0; i < 11; i++) {
+      tuple4.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createText("dept_" + i),
+          DatumFactory.createInt4(1000 + i) });
+      appender4.addTuple(tuple4);
+    }
+
+    appender4.flush();
+    appender4.close();
+    dep4 = CatalogUtil.newTableDesc(DEP4_NAME, dep4Schema, dep4Meta, dep4Path);
+    catalog.createTable(dep4);
+
+
+
+    //----------------- job3 ------------------------------
+    //  job_id  | job_title
+    // ----------------------
+    //   101    |  job_101
+    //   102    |  job_102
+    //   103    |  job_103
+
+    Schema job3Schema = new Schema();
+    job3Schema.addColumn("job_id", Type.INT4);
+    job3Schema.addColumn("job_title", Type.TEXT);
+
+
+    TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path job3Path = new Path(testDir, "job3.csv");
+    Appender appender2 = StorageManagerFactory.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    appender2.init();
+    Tuple tuple2 = new VTuple(job3Schema.size());
+    for (int i = 1; i < 4; i++) {
+      int x = 100 + i;
+      tuple2.put(new Datum[] { DatumFactory.createInt4(100 + i),
+          DatumFactory.createText("job_" + x) });
+      appender2.addTuple(tuple2);
+    }
+
+    appender2.flush();
+    appender2.close();
+    job3 = CatalogUtil.newTableDesc(JOB3_NAME, job3Schema, job3Meta, job3Path);
+    catalog.createTable(job3);
+
+
+
+    //---------------------emp3 --------------------
+    // emp_id  | first_name | last_name | dep_id | salary | job_id
+    // ------------------------------------------------------------
+    //  11     |  fn_11     |  ln_11    |  1     | 123    | 101
+    //  13     |  fn_13     |  ln_13    |  3     | 369    | 103
+    //  15     |  fn_15     |  ln_15    |  5     | 615    | null
+    //  17     |  fn_17     |  ln_17    |  7     | 861    | null
+    //  19     |  fn_19     |  ln_19    |  9     | 1107   | null
+    //  21     |  fn_21     |  ln_21    |  1     | 123    | 101
+    //  23     |  fn_23     |  ln_23    |  3     | 369    | 103
+
+    Schema emp3Schema = new Schema();
+    emp3Schema.addColumn("emp_id", Type.INT4);
+    emp3Schema.addColumn("first_name", Type.TEXT);
+    emp3Schema.addColumn("last_name", Type.TEXT);
+    emp3Schema.addColumn("dep_id", Type.INT4);
+    emp3Schema.addColumn("salary", Type.FLOAT4);
+    emp3Schema.addColumn("job_id", Type.INT4);
+
+
+    TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path emp3Path = new Path(testDir, "emp3.csv");
+    Appender appender3 = StorageManagerFactory.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    appender3.init();
+    Tuple tuple3 = new VTuple(emp3Schema.size());
+
+    for (int i = 1; i < 4; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+
+      int y = 20 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(20 + i),
+          DatumFactory.createText("firstname_" + y),
+          DatumFactory.createText("lastname_" + y),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+    }
+
+    for (int i = 5; i < 10; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createNullDatum() });
+      appender3.addTuple(tuple3);
+    }
+
+    appender3.flush();
+    appender3.close();
+    emp3 = CatalogUtil.newTableDesc(EMP3_NAME, emp3Schema, emp3Meta, emp3Path);
+    catalog.createTable(emp3);
+
+    //---------------------phone3 --------------------
+    // emp_id  | phone_number
+    // -----------------------------------------------
+    // this table is empty, no rows
+
+    Schema phone3Schema = new Schema();
+    phone3Schema.addColumn("emp_id", Type.INT4);
+    phone3Schema.addColumn("phone_number", Type.TEXT);
+
+
+    TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path phone3Path = new Path(testDir, "phone3.csv");
+    Appender appender5 = StorageManagerFactory.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
+        phone3Path);
+    appender5.init();
+    appender5.flush();
+    appender5.close();
+    phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path);
+    catalog.createTable(phone3);
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      // [0] no nulls
+      "select dep3.dep_id, dep_name, emp_id, salary from emp3 full outer join dep3 on dep3.dep_id = emp3.dep_id",
+      // [1] nulls on the left operand
+      "select job3.job_id, job_title, emp_id, salary from emp3 full outer join job3 on job3.job_id=emp3.job_id",
+      // [2] nulls on the right side
+      "select job3.job_id, job_title, emp_id, salary from job3 full outer join emp3 on job3.job_id=emp3.job_id",
+      // [3] no nulls, right continues after left
+      "select dep4.dep_id, dep_name, emp_id, salary from emp3 full outer join dep4 on dep4.dep_id = emp3.dep_id",
+      // [4] one operand is empty
+      "select emp3.emp_id, first_name, phone_number from emp3 full outer join phone3 on emp3.emp_id = phone3.emp_id",
+      // [5] one operand is empty
+      "select emp3.emp_id, first_name, phone_number from phone3 full outer join emp3 on emp3.emp_id = phone3.emp_id",
+  };
+
+  @Test
+  public final void testFullOuterMergeJoin0() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] dep3Frags =
+        StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin0");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof MergeFullOuterJoinExec);
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(12, count);
+  }
+
+
+  @Test
+  public final void testFullOuterMergeJoin1() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] job3Frags =
+        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin1");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof MergeFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(8, count);
+  }
+
+  @Test
+  public final void testFullOuterMergeJoin2() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[2]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] job3Frags =
+        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin2");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof MergeFullOuterJoinExec);
+
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(8, count);
+  }
+
+  @Test
+  public final void testFullOuterMergeJoin3() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[3]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] dep4Frags =
+        StorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), dep4.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin3");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+
+    // if it chose the hash join WITH REVERSED ORDER, convert to merge join exec
+    assertTrue(proj.getChild() instanceof MergeFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(13, count);
+  }
+
+
+  @Test
+  public final void testFullOuterMergeJoin4() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[4]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] phone3Frags =
+        StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin4");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof MergeFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(7, count);
+  }
+
+
+  @Test
+  public final void testFullOuterMergeJoin5() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[5]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] phone3Frags =
+        StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin5");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof MergeFullOuterJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(7, count);
+  }
+
+
+
+
+
+}


[42/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java
new file mode 100644
index 0000000..60e9685
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java
@@ -0,0 +1,1550 @@
+/**
+ * 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.parser;
+
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.tree.TerminalNodeImpl;
+import org.apache.commons.lang.math.NumberUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.parser.HiveQLParser.TableAllColumnsContext;
+
+import java.math.BigInteger;
+import java.util.*;
+
+public class HiveQLAnalyzer extends HiveQLParserBaseVisitor<Expr> {
+  private static final Log LOG = LogFactory.getLog(HiveQLAnalyzer.class.getName());
+  private HiveQLParser parser;
+
+  public Expr parse(String sql) {
+    HiveQLLexer lexer = new HiveQLLexer(new ANTLRNoCaseStringStream(sql));
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    parser = new HiveQLParser(tokens);
+    parser.setBuildParseTree(true);
+
+    HiveQLParser.StatementContext context;
+    try {
+      context = parser.statement();
+    } catch (SQLParseError e) {
+      throw new SQLSyntaxError(e);
+    }
+
+    return visit(context);
+  }
+
+  @Override
+  public Expr visitStatement(HiveQLParser.StatementContext ctx) {
+    return visitExecStatement(ctx.execStatement());
+  }
+
+  @Override
+  public Expr visitQueryStatement(HiveQLParser.QueryStatementContext ctx) {
+    Expr current = null;
+
+    if (ctx.body != null) {
+      current = visitBody(ctx.body(0));
+    }
+
+    if (ctx.regular_body() != null) {
+      current = visitRegular_body(ctx.regular_body());
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitBody(HiveQLParser.BodyContext ctx) {
+
+    Expr current = null;
+    Insert insert = null;
+
+    Projection select = null;
+
+    if (ctx.insertClause() != null) {
+      insert = visitInsertClause(ctx.insertClause());
+    }
+
+    if (ctx.selectClause() != null) {
+      select = (Projection) visitSelectClause(ctx.selectClause());
+      if (ctx.selectClause().KW_DISTINCT() != null) {
+        select.setDistinct();
+      }
+
+    }
+
+    for (int i = 0; i < ctx.getParent().getChildCount(); i++) {
+      if (ctx.getParent().getChild(i) instanceof HiveQLParser.FromClauseContext) {
+        HiveQLParser.FromClauseContext fromClauseContext = (HiveQLParser.FromClauseContext) ctx.getParent().getChild(i);
+        Expr from = visitFromClause(fromClauseContext);
+        current = from;
+      }
+    }
+
+    if (ctx.whereClause() != null) {
+      Selection where = new Selection(visitWhereClause(ctx.whereClause()));
+      where.setChild(current);
+      current = where;
+    }
+
+    if (ctx.groupByClause() != null) {
+      Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
+      aggregation.setChild(current);
+      current = aggregation;
+
+      if (ctx.havingClause() != null) {
+        Expr havingCondition = visitHavingClause(ctx.havingClause());
+        Having having = new Having(havingCondition);
+        having.setChild(current);
+        current = having;
+      }
+    }
+
+    if (ctx.orderByClause() != null) {
+      Sort sort = visitOrderByClause(ctx.orderByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.clusterByClause() != null) {
+      visitClusterByClause(ctx.clusterByClause());
+    }
+
+    if (ctx.distributeByClause() != null) {
+      visitDistributeByClause(ctx.distributeByClause());
+    }
+
+    if (ctx.sortByClause() != null) {
+      Sort sort = visitSortByClause(ctx.sortByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.window_clause() != null) {
+      Expr window = visitWindow_clause(ctx.window_clause());
+    }
+
+    if (ctx.limitClause() != null) {
+      Limit limit = visitLimitClause(ctx.limitClause());
+      limit.setChild(current);
+      current = limit;
+    }
+
+    Projection projection = new Projection();
+    projection.setNamedExprs(select.getNamedExprs());
+
+    if (current != null)
+      projection.setChild(current);
+
+    if (select.isDistinct())
+      projection.setDistinct();
+
+
+    if (insert != null) {
+      insert.setSubQuery(projection);
+      current = insert;
+    } else {
+      current = projection;
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitRegular_body(HiveQLParser.Regular_bodyContext ctx) {
+    Expr current = null;
+    Insert insert = null;
+
+    if (ctx.selectStatement() != null) {
+      current = visitSelectStatement(ctx.selectStatement());
+    } else {
+      Projection select = null;
+
+      if (ctx.insertClause() != null) {
+        insert = visitInsertClause(ctx.insertClause());
+      }
+
+      if (ctx.selectClause() != null) {
+        select = (Projection) visitSelectClause(ctx.selectClause());
+        if (ctx.selectClause().KW_DISTINCT() != null) {
+          select.setDistinct();
+        }
+
+      }
+
+      if (ctx.fromClause() != null) {
+        Expr from = visitFromClause(ctx.fromClause());
+        current = from;
+      }
+
+      if (ctx.whereClause() != null) {
+        Selection where = new Selection(visitWhereClause(ctx.whereClause()));
+        where.setChild(current);
+        current = where;
+      }
+
+      if (ctx.groupByClause() != null) {
+        Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
+        aggregation.setChild(current);
+        current = aggregation;
+
+        if (ctx.havingClause() != null) {
+          Expr havingCondition = visitHavingClause(ctx.havingClause());
+          Having having = new Having(havingCondition);
+          having.setChild(current);
+          current = having;
+        }
+      }
+
+      if (ctx.orderByClause() != null) {
+        Sort sort = visitOrderByClause(ctx.orderByClause());
+        sort.setChild(current);
+        current = sort;
+      }
+
+      if (ctx.clusterByClause() != null) {
+        visitClusterByClause(ctx.clusterByClause());
+      }
+
+      if (ctx.distributeByClause() != null) {
+        visitDistributeByClause(ctx.distributeByClause());
+      }
+
+      if (ctx.sortByClause() != null) {
+        Sort sort = visitSortByClause(ctx.sortByClause());
+        sort.setChild(current);
+        current = sort;
+      }
+
+      if (ctx.window_clause() != null) {
+        Expr window = visitWindow_clause(ctx.window_clause());
+      }
+
+      if (ctx.limitClause() != null) {
+        Limit limit = visitLimitClause(ctx.limitClause());
+        limit.setChild(current);
+        current = limit;
+      }
+
+      Projection projection = new Projection();
+      projection.setNamedExprs(select.getNamedExprs());
+
+      if (current != null)
+        projection.setChild(current);
+
+      if (select.isDistinct())
+        projection.setDistinct();
+
+      if (insert != null) {
+        insert.setSubQuery(projection);
+        current = insert;
+      } else {
+        current = projection;
+      }
+
+
+    }
+    return current;
+  }
+
+  /**
+   * This method implemented for parsing union all clause.
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitQueryStatementExpression(HiveQLParser.QueryStatementExpressionContext ctx) {
+    Expr left = null, right = null, current = null;
+    if (ctx.queryStatement() != null) {
+      if (ctx.queryStatement().size() == 1)
+        return visitQueryStatement(ctx.queryStatement(0));
+
+      for (int i = 0; i < ctx.queryStatement().size(); i++) {
+        if (i == 0)
+          current = visitQueryStatement(ctx.queryStatement(i));
+        else
+          left = current;
+
+        if (i > 0) {
+          right = visitQueryStatement(ctx.queryStatement(i));
+          current = new SetOperation(OpType.Union, left, right, false);
+        }
+      }
+    }
+    return current;
+  }
+
+  @Override
+  public Expr visitSelectStatement(HiveQLParser.SelectStatementContext ctx) {
+    Expr current = null;
+
+    Projection select = (Projection) visitSelectClause(ctx.selectClause());
+
+    if (ctx.selectClause().KW_DISTINCT() != null) {
+      select.setDistinct();
+    }
+
+    Expr from = visitFromClause(ctx.fromClause());
+    current = from;
+
+    if (ctx.whereClause() != null) {
+      Selection where = new Selection(visitWhereClause(ctx.whereClause()));
+      where.setChild(current);
+      current = where;
+    }
+
+    if (ctx.groupByClause() != null) {
+      Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
+      aggregation.setChild(current);
+      current = aggregation;
+
+      if (ctx.havingClause() != null) {
+        Expr havingCondition = visitHavingClause(ctx.havingClause());
+        Having having = new Having(havingCondition);
+        having.setChild(current);
+        current = having;
+      }
+    }
+
+    if (ctx.orderByClause() != null) {
+      Sort sort = visitOrderByClause(ctx.orderByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.clusterByClause() != null) {
+      visitClusterByClause(ctx.clusterByClause());
+    }
+
+    if (ctx.distributeByClause() != null) {
+      visitDistributeByClause(ctx.distributeByClause());
+    }
+
+    if (ctx.sortByClause() != null) {
+      Sort sort = visitSortByClause(ctx.sortByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.window_clause() != null) {
+      Expr window = visitWindow_clause(ctx.window_clause());
+    }
+
+    if (ctx.limitClause() != null) {
+      Limit limit = visitLimitClause(ctx.limitClause());
+      limit.setChild(current);
+      current = limit;
+    }
+
+    Projection projection = new Projection();
+    projection.setNamedExprs(select.getNamedExprs());
+
+    if (current != null)
+      projection.setChild(current);
+
+    if (select.isDistinct())
+      projection.setDistinct();
+
+    current = projection;
+
+    return current;
+  }
+
+  @Override
+  public Expr visitFromClause(HiveQLParser.FromClauseContext ctx) {
+    return visitJoinSource(ctx.joinSource());
+  }
+
+  @Override
+  public Expr visitJoinSource(HiveQLParser.JoinSourceContext ctx) {
+    Expr[] relations = null;
+    RelationList relationList = null;
+
+    if (ctx.fromSource() != null) {
+      int fromCount = ctx.fromSource().size();
+      int uniqueJoinCount = ctx.uniqueJoinSource().size();
+
+      relations = new Expr[1];
+
+      Join current = null, parent = null;
+      JoinType type = null;
+      Expr left = null, right = null, condition = null;
+
+
+      if (fromCount == 1) {
+        relations[0] = visitFromSource(ctx.fromSource(0));
+      } else {
+        left = visitFromSource((HiveQLParser.FromSourceContext) ctx.getChild(0));
+
+        for (int i = 1; i < ctx.getChildCount(); i++) {
+          type = null;
+          right = null;
+          condition = null;
+
+          if (ctx.getChild(i) instanceof HiveQLParser.JoinTokenContext) {
+            type = getJoinType((HiveQLParser.JoinTokenContext) ctx.getChild(i));
+            if (i > 1)
+              left = parent;
+
+            if (i + 1 < ctx.getChildCount() && ctx.getChild(i + 1) instanceof HiveQLParser.FromSourceContext) {
+              right = visitFromSource((HiveQLParser.FromSourceContext) ctx.getChild(i + 1));
+            }
+
+            if (i + 3 < ctx.getChildCount() && ctx.getChild(i + 3) instanceof HiveQLParser.ExpressionContext) {
+              condition = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 3));
+            }
+
+            if (type != null) {
+              current = new Join(type);
+              current.setLeft(left);
+              current.setRight(right);
+
+              if (condition != null)
+                current.setQual(condition);
+
+              parent = current;
+            }
+          }
+
+        }
+        relations[0] = current;
+      }
+
+      //TODO: implement unique join.
+      relationList = new RelationList(relations);
+    }
+
+    return relationList;
+  }
+
+  public JoinType getJoinType(HiveQLParser.JoinTokenContext context) {
+    JoinType type = JoinType.INNER;
+
+    if (context.KW_INNER() != null) {
+      type = JoinType.INNER;
+    }
+
+    if (context.KW_LEFT() != null && context.KW_OUTER() != null) {
+      type = JoinType.LEFT_OUTER;
+    }
+
+    if (context.KW_RIGHT() != null && context.KW_OUTER() != null) {
+      type = JoinType.RIGHT_OUTER;
+    }
+
+    if (context.KW_CROSS() != null) {
+      type = JoinType.CROSS;
+    }
+
+    if (context.KW_FULL() != null) {
+      type = JoinType.FULL_OUTER;
+    }
+
+    if (context.KW_SEMI() != null) {
+      type = null;
+    }
+    return type;
+  }
+
+  @Override
+  public Expr visitFromSource(HiveQLParser.FromSourceContext ctx) {
+    Expr current = null;
+
+    if (ctx.Identifier() != null && ctx.LPAREN() != null) {
+      current = new LiteralValue(ctx.Identifier().getText(), LiteralValue.LiteralType.String);
+    }
+
+    if (ctx.tableSource() != null) {
+      current = visitTableSource(ctx.tableSource());
+    }
+
+    if (ctx.subQuerySource() != null) {
+      current = visitSubQuerySource(ctx.subQuerySource());
+
+      String tableAlias = "";
+      for (int i = 0; i < ctx.subQuerySource().getChildCount(); i++) {
+        if (ctx.subQuerySource().getChild(i) instanceof HiveQLParser.IdentifierContext) {
+          tableAlias = (ctx.subQuerySource().getChild(i)).getText();
+        }
+      }
+
+      TablePrimarySubQuery subQuery = new TablePrimarySubQuery(tableAlias, current);
+      current = subQuery;
+    }
+    // TODO: implement lateralView
+
+    return current;
+  }
+
+  @Override
+  public Expr visitSubQuerySource(HiveQLParser.SubQuerySourceContext ctx) {
+    Expr current = visitQueryStatementExpression(ctx.queryStatementExpression());
+    return current;
+  }
+
+  @Override
+  public Expr visitTableSource(HiveQLParser.TableSourceContext ctx) {
+    String tableName = "", alias = "";
+
+    if (ctx.tableName() != null)
+      tableName = ctx.tableName().getText();
+
+    if (ctx.alias != null) {
+      alias = ctx.alias.getText();
+      for (String token : HiveQLParser.tokenNames) {
+        if (token.replaceAll("'", "").equalsIgnoreCase(alias))
+          alias = "";
+      }
+    }
+
+    Relation relation = new Relation(tableName);
+    if (!alias.equals(""))
+      relation.setAlias(alias);
+
+    return relation;
+  }
+
+  @Override
+  public Expr visitSelectList(HiveQLParser.SelectListContext ctx) {
+    Expr current = null;
+    Projection projection = new Projection();
+    NamedExpr[] targets = new NamedExpr[ctx.selectItem().size()];
+    for (int i = 0; i < targets.length; i++) {
+      targets[i] = visitSelectItem(ctx.selectItem(i));
+    }
+
+    projection.setNamedExprs(targets);
+    current = projection;
+    return current;
+  }
+
+  @Override
+  public NamedExpr visitSelectItem(HiveQLParser.SelectItemContext ctx) {
+    NamedExpr target = null;
+
+    if (ctx.selectExpression() != null) {
+      target = new NamedExpr(visitSelectExpression(ctx.selectExpression()));
+    } else if (ctx.window_specification() != null) {
+      // TODO: if there is a window specification clause, we should handle it properly.
+    }
+
+    if (ctx.identifier().size() > 0 && target != null) {
+      target.setAlias(ctx.identifier(0).getText());
+    }
+    return target;
+  }
+
+  @Override
+  public Expr visitSelectExpression(HiveQLParser.SelectExpressionContext ctx) {
+    Expr current = null;
+
+    if (ctx.tableAllColumns() != null) {
+      current = visitTableAllColumns(ctx.tableAllColumns());
+    } else {
+      if (ctx.expression() != null) {
+        current = visitExpression(ctx.expression());
+      }
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitTableAllColumns(TableAllColumnsContext ctx) {
+    QualifiedAsteriskExpr target = new QualifiedAsteriskExpr();
+    if (ctx.tableName() != null) {
+      target.setQualifier(ctx.tableName().getText());
+    }
+
+    return target;
+  }
+
+  @Override
+  public Expr visitExpression(HiveQLParser.ExpressionContext ctx) {
+    Expr current = visitPrecedenceOrExpression(ctx.precedenceOrExpression());
+    return current;
+  }
+
+  @Override
+  public Expr visitPrecedenceOrExpression(HiveQLParser.PrecedenceOrExpressionContext ctx) {
+    Expr current = null, left = null, right = null;
+
+    for (int i = 0; i < ctx.precedenceAndExpression().size(); i++) {
+      if (i == 0) {
+        left = visitPrecedenceAndExpression(ctx.precedenceAndExpression(i));
+        current = left;
+      } else {
+        left = current;
+        right = visitPrecedenceAndExpression(ctx.precedenceAndExpression(i));
+        current = new BinaryOperator(OpType.Or, left, right);
+      }
+
+    }
+    return current;
+  }
+
+  /**
+   * This method parse AND expressions at WHERE clause.
+   * And this convert 'x BETWEEN y AND z' expression into 'x >= y AND x <= z' expression
+   * because Tajo doesn't provide 'BETWEEN' expression.
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitPrecedenceAndExpression(HiveQLParser.PrecedenceAndExpressionContext ctx) {
+    Expr current = null, left = null, right = null;
+
+    for (int i = 0; i < ctx.precedenceNotExpression().size(); i++) {
+      Expr min = null, max = null;
+
+      if (ctx.precedenceNotExpression(i).precedenceEqualExpression() != null) {
+        HiveQLParser.PrecedenceEqualExpressionContext expressionContext = ctx.precedenceNotExpression(i)
+            .precedenceEqualExpression();
+        if (expressionContext.KW_BETWEEN() != null) {
+
+          if (expressionContext.min != null) {
+            min = visitPrecedenceBitwiseOrExpression(expressionContext.min);
+          }
+
+          if (expressionContext.max != null) {
+            max = visitPrecedenceBitwiseOrExpression(expressionContext.max);
+          }
+        }
+      }
+
+      if (min != null && max != null) {
+        left = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
+        if (left != null) {
+          if (i == 0) {
+            BinaryOperator minOperator = new BinaryOperator(OpType.GreaterThanOrEquals, left, min);
+            BinaryOperator maxOperator = new BinaryOperator(OpType.LessThanOrEquals, left, max);
+            current = new BinaryOperator(OpType.And, minOperator, maxOperator);
+          } else {
+            BinaryOperator minOperator = new BinaryOperator(OpType.GreaterThanOrEquals, left, min);
+            current = new BinaryOperator(OpType.And, current, minOperator);
+
+            BinaryOperator maxOperator = new BinaryOperator(OpType.LessThanOrEquals, left, max);
+            current = new BinaryOperator(OpType.And, current, maxOperator);
+          }
+        }
+      } else {
+        if (i == 0) {
+          left = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
+          current = left;
+        } else {
+          left = current;
+          right = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
+          current = new BinaryOperator(OpType.And, left, right);
+        }
+      }
+    }
+    return current;
+  }
+
+  @Override
+  public Expr visitPrecedenceNotExpression(HiveQLParser.PrecedenceNotExpressionContext ctx) {
+    HiveQLParser.PrecedenceEqualExpressionContext expressionContext = ctx.precedenceEqualExpression();
+    Expr current = visitPrecedenceEqualExpression(expressionContext);
+    return current;
+  }
+
+  /**
+   * This method parse operators for equals expressions as follows:
+   * =, <>, !=, >=, >, <=, <, IN, NOT IN, LIKE, REGEXP, RLIKE
+   * <p/>
+   * In this case, this make RuntimeException>
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitPrecedenceEqualExpression(HiveQLParser.PrecedenceEqualExpressionContext ctx) {
+    Expr current = null, left = null, right = null, min = null, max = null;
+    OpType type = null;
+    boolean isNot = false, isIn = false;
+    for (int i = 0; i < ctx.getChildCount(); i++) {
+      if (ctx.getChild(i) instanceof HiveQLParser.PrecedenceBitwiseOrExpressionContext) {
+        if (i == 0) {
+          left = visitPrecedenceBitwiseOrExpression((HiveQLParser.PrecedenceBitwiseOrExpressionContext) ctx.getChild(i));
+        } else {
+          right = visitPrecedenceBitwiseOrExpression((HiveQLParser.PrecedenceBitwiseOrExpressionContext) ctx.getChild(i));
+        }
+      } else if (ctx.getChild(i) instanceof HiveQLParser.ExpressionsContext) {
+        right = visitExpressions((HiveQLParser.ExpressionsContext) ctx.getChild(i));
+      } else if (ctx.getChild(i) instanceof TerminalNodeImpl) {
+        int symbolType = ((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType();
+        switch (symbolType) {
+          case HiveQLLexer.KW_NOT:
+            isNot = true;
+            break;
+          case HiveQLLexer.KW_IN:
+            isIn = true;
+            break;
+          default:
+            break;
+        }
+      } else if (ctx.getChild(i) instanceof HiveQLParser.PrecedenceEqualOperatorContext
+          || ctx.getChild(i) instanceof HiveQLParser.PrecedenceEqualNegatableOperatorContext) {
+        String keyword = ctx.getChild(i).getText().toUpperCase();
+
+        if (keyword.equals(">")) {
+          type = OpType.GreaterThan;
+        } else if (keyword.equals("<=>")) {
+          throw new RuntimeException("Unexpected operator : <=>");
+        } else if (keyword.equals("=")) {
+          type = OpType.Equals;
+        } else if (keyword.equals("<=")) {
+          type = OpType.LessThanOrEquals;
+        } else if (keyword.equals("<")) {
+          type = OpType.LessThan;
+        } else if (keyword.equals(">=")) {
+          type = OpType.GreaterThanOrEquals;
+        } else if (keyword.equals("<>")) {
+          type = OpType.NotEquals;
+        } else if (keyword.equals("!=")) {
+          type = OpType.NotEquals;
+        } else if (keyword.equals("REGEXP")) {
+          type = OpType.Regexp;
+        } else if (keyword.equals("RLIKE")) {
+          type = OpType.Regexp;
+        } else if (keyword.equals("LIKE")) {
+          type = OpType.LikePredicate;
+        }
+      }
+    }
+
+    if (type != null && right != null) {
+      if (type.equals(OpType.LikePredicate)) {
+        PatternMatchPredicate like = new PatternMatchPredicate(OpType.LikePredicate,
+            isNot, left, right);
+        current = like;
+      } else if (type.equals(OpType.Regexp)) {
+        PatternMatchPredicate regex = new PatternMatchPredicate(OpType.Regexp, isNot, left, right);
+        current = regex;
+      } else {
+        BinaryOperator binaryOperator = new BinaryOperator(type, left, right);
+        current = binaryOperator;
+      }
+    } else if (isIn) {
+      InPredicate inPredicate = new InPredicate(left, right, isNot);
+      current = inPredicate;
+    } else {
+      current = left;
+    }
+
+    return current;
+  }
+
+  @Override
+  public ValueListExpr visitExpressions(HiveQLParser.ExpressionsContext ctx) {
+    int size = ctx.expression().size();
+    Expr[] exprs = new Expr[size];
+    for (int i = 0; i < size; i++) {
+      exprs[i] = visitExpression(ctx.expression(i));
+    }
+    return new ValueListExpr(exprs);
+  }
+
+  @Override
+  public Expr visitPrecedenceBitwiseOrExpression(HiveQLParser.PrecedenceBitwiseOrExpressionContext ctx) {
+    int expressionCount = ctx.precedenceAmpersandExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceBitwiseOrOperator(operatorIndex) != null) {
+        type = getPrecedenceBitwiseOrOperator(ctx.precedenceBitwiseOrOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i));
+        if (ctx.precedenceAmpersandExpression(i + 1) != null)
+          right = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i + 1));
+      } else {
+        parentType = getPrecedenceBitwiseOrOperator((ctx.precedenceBitwiseOrOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i - 2));
+        parentRight = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+    return current;
+  }
+
+  public OpType getPrecedenceBitwiseOrOperator(HiveQLParser.PrecedenceBitwiseOrOperatorContext ctx) {
+    OpType type = null;
+    // TODO: It needs to consider how to support.
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceAmpersandExpression(HiveQLParser.PrecedenceAmpersandExpressionContext ctx) {
+    int expressionCount = ctx.precedencePlusExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceAmpersandOperator(operatorIndex) != null) {
+        type = getPrecedenceAmpersandOperator(ctx.precedenceAmpersandOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i));
+        if (ctx.precedencePlusExpression(i + 1) != null)
+          right = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i + 1));
+      } else {
+        parentType = getPrecedenceAmpersandOperator((ctx.precedenceAmpersandOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i - 2));
+        parentRight = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+    return current;
+  }
+
+  public OpType getPrecedenceAmpersandOperator(HiveQLParser.PrecedenceAmpersandOperatorContext ctx) {
+    OpType type = null;
+    // TODO: It needs to consider how to support.
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedencePlusExpression(HiveQLParser.PrecedencePlusExpressionContext ctx) {
+    int expressionCount = ctx.precedenceStarExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedencePlusOperator(operatorIndex) != null) {
+        type = getPrecedencePlusOperator(ctx.precedencePlusOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i));
+        if (ctx.precedenceStarExpression(i + 1) != null)
+          right = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i + 1));
+      } else {
+        parentType = getPrecedencePlusOperator((ctx.precedencePlusOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i - 2));
+        parentRight = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+    return current;
+  }
+
+  public OpType getPrecedencePlusOperator(HiveQLParser.PrecedencePlusOperatorContext ctx) {
+    OpType type = null;
+
+    if (ctx.MINUS() != null) {
+      type = OpType.Minus;
+    } else if (ctx.PLUS() != null) {
+      type = OpType.Plus;
+    }
+
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceStarExpression(HiveQLParser.PrecedenceStarExpressionContext ctx) {
+    int expressionCount = ctx.precedenceBitwiseXorExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceStarOperator(operatorIndex) != null) {
+        type = getPrecedenceStarOperator(ctx.precedenceStarOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i));
+        if (ctx.precedenceBitwiseXorExpression(i + 1) != null)
+          right = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i + 1));
+      } else {
+        parentType = getPrecedenceStarOperator((ctx.precedenceStarOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i - 2));
+        parentRight = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+
+    return current;
+  }
+
+  public OpType getPrecedenceStarOperator(HiveQLParser.PrecedenceStarOperatorContext ctx) {
+    OpType type = null;
+
+    if (ctx.DIV() != null) {
+      type = OpType.Divide;
+    } else if (ctx.DIVIDE() != null) {
+      type = OpType.Divide;
+    } else if (ctx.MOD() != null) {
+      type = OpType.Modular;
+    } else if (ctx.STAR() != null) {
+      type = OpType.Multiply;
+    }
+
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceBitwiseXorExpression(HiveQLParser.PrecedenceBitwiseXorExpressionContext ctx) {
+    int expressionCount = ctx.precedenceUnarySuffixExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceBitwiseXorOperator(operatorIndex) != null) {
+        type = getPrecedenceBitwiseXorOperator(ctx.precedenceBitwiseXorOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i));
+        if (ctx.precedenceUnarySuffixExpression(i + 1) != null)
+          right = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i + 1));
+      } else {
+        parentType = getPrecedenceBitwiseXorOperator((ctx.precedenceBitwiseXorOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i - 2));
+        parentRight = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+
+    return current;
+  }
+
+  public OpType getPrecedenceBitwiseXorOperator(HiveQLParser.PrecedenceBitwiseXorOperatorContext ctx) {
+    OpType type = null;
+    // TODO: It needs to consider how to support.
+
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceUnarySuffixExpression(HiveQLParser.PrecedenceUnarySuffixExpressionContext ctx) {
+    Expr current = visitPrecedenceUnaryPrefixExpression(ctx.precedenceUnaryPrefixExpression());
+
+    if (ctx.nullCondition() != null) {
+      boolean isNot = ctx.nullCondition().KW_NOT() == null ? false : true;
+      IsNullPredicate isNullPredicate = new IsNullPredicate(isNot, (ColumnReferenceExpr) current);
+      current = isNullPredicate;
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitPrecedenceUnaryPrefixExpression(HiveQLParser.PrecedenceUnaryPrefixExpressionContext ctx) {
+    Expr current = visitPrecedenceFieldExpression(ctx.precedenceFieldExpression());
+    return current;
+  }
+
+  @Override
+  public Expr visitNullCondition(HiveQLParser.NullConditionContext ctx) {
+    return new NullLiteral();
+  }
+
+  @Override
+  public Expr visitPrecedenceFieldExpression(HiveQLParser.PrecedenceFieldExpressionContext ctx) {
+    Expr current = visitAtomExpression(ctx.atomExpression());
+
+    if (ctx.DOT().size() > 0) {
+      ColumnReferenceExpr column = new ColumnReferenceExpr(ctx.identifier(0).getText());
+      ColumnReferenceExpr table = (ColumnReferenceExpr) current;
+      column.setQualifier(table.getName());
+      current = column;
+    }
+    return current;
+  }
+
+  @Override
+  public Expr visitAtomExpression(HiveQLParser.AtomExpressionContext ctx) {
+    Expr current = null;
+
+    if (ctx.KW_NULL() != null) {
+      current = new NullLiteral();
+    }
+    if (ctx.constant() != null) {
+      current = visitConstant(ctx.constant());
+    }
+    if (ctx.function() != null) {
+      current = visitFunction(ctx.function());
+    }
+    if (ctx.castExpression() != null) {
+      current = visitCastExpression(ctx.castExpression());
+    }
+    if (ctx.caseExpression() != null) {
+      current = visitCaseExpression(ctx.caseExpression());
+    }
+    if (ctx.whenExpression() != null) {
+      current = visitWhenExpression(ctx.whenExpression());
+    }
+    if (ctx.tableOrColumn() != null) {
+      current = visitTableOrColumn(ctx.tableOrColumn());
+    } else {
+      if (ctx.LPAREN() != null && ctx.RPAREN() != null) {
+        current = visitExpression(ctx.expression());
+      }
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitTableOrColumn(HiveQLParser.TableOrColumnContext ctx) {
+    ColumnReferenceExpr columnReferenceExpr = new ColumnReferenceExpr(ctx.identifier().getText());
+    return columnReferenceExpr;
+  }
+
+  @Override
+  public Expr visitIdentifier(HiveQLParser.IdentifierContext ctx) {
+    Expr current = null;
+
+    if (ctx.nonReserved() != null) {
+      current = new LiteralValue(ctx.nonReserved().getText(), LiteralValue.LiteralType.String);
+    } else {
+      current = new LiteralValue(ctx.Identifier().getText(), LiteralValue.LiteralType.String);
+    }
+
+    return current;
+  }
+
+  @Override
+  public LiteralValue visitConstant(HiveQLParser.ConstantContext ctx) {
+    LiteralValue literalValue = null;
+
+    if (ctx.StringLiteral() != null) {
+      String value = ctx.StringLiteral().getText();
+      String strValue = "";
+      if ((value.startsWith("'") && value.endsWith("'")) || value.startsWith("\"") && value.endsWith("\"")) {
+        strValue = value.substring(1, value.length() - 1);
+      } else {
+        strValue = value;
+      }
+
+      literalValue = new LiteralValue(strValue, LiteralValue.LiteralType.String);
+    } else if (ctx.TinyintLiteral() != null) {
+      literalValue = new LiteralValue(ctx.TinyintLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Integer);
+    } else if (ctx.BigintLiteral() != null) {
+      literalValue = new LiteralValue(ctx.BigintLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Large_Integer);
+    } else if (ctx.DecimalLiteral() != null) {
+      literalValue = new LiteralValue(ctx.DecimalLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Integer);
+    } else if (ctx.Number() != null) {
+      try {
+        float floatValue = NumberUtils.createFloat(ctx.getText());
+        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(), LiteralValue.LiteralType.Unsigned_Float);
+      } catch (NumberFormatException nf) {
+      }
+
+      // TODO: double type
+
+      try {
+        BigInteger bigIntegerVallue = NumberUtils.createBigInteger(ctx.getText());
+        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(),
+            LiteralValue.LiteralType.Unsigned_Large_Integer);
+      } catch (NumberFormatException nf) {
+      }
+
+      try {
+        int intValue = NumberUtils.createInteger(ctx.getText());
+        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(), LiteralValue.LiteralType.Unsigned_Integer);
+      } catch (NumberFormatException nf) {
+      }
+
+    } else if (ctx.SmallintLiteral() != null) {
+      literalValue = new LiteralValue(ctx.SmallintLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Integer);
+    } else if (ctx.booleanValue() != null) {
+      // TODO: boolean type
+    }
+
+    return literalValue;
+  }
+
+  @Override
+  public Expr visitFunction(HiveQLParser.FunctionContext ctx) {
+    Expr current = null;
+    String signature = ctx.functionName().getText();
+
+    boolean isDistinct = false;
+    if (ctx.getChild(2) != null) {
+      if (ctx.getChild(2) instanceof TerminalNodeImpl && ctx.getChild(2).getText().equalsIgnoreCase("DISTINCT")) {
+        isDistinct = true;
+      }
+    }
+
+    if (signature.equalsIgnoreCase("MIN")
+        || signature.equalsIgnoreCase("MAX")
+        || signature.equalsIgnoreCase("SUM")
+        || signature.equalsIgnoreCase("AVG")
+        || signature.equalsIgnoreCase("COUNT")
+        ) {
+      if (ctx.selectExpression().size() > 1) {
+        throw new RuntimeException("Exactly expected one argument.");
+      }
+
+      if (ctx.selectExpression().size() == 0) {
+        CountRowsFunctionExpr countRowsFunctionExpr = new CountRowsFunctionExpr();
+        current = countRowsFunctionExpr;
+      } else {
+        GeneralSetFunctionExpr setFunctionExpr = new GeneralSetFunctionExpr(signature, isDistinct,
+            visitSelectExpression(ctx.selectExpression(0)));
+        current = setFunctionExpr;
+      }
+    } else {
+      FunctionExpr functionExpr = new FunctionExpr(signature);
+      Expr[] params = new Expr[ctx.selectExpression().size()];
+      for (int i = 0; i < ctx.selectExpression().size(); i++) {
+        params[i] = visitSelectExpression(ctx.selectExpression(i));
+      }
+      functionExpr.setParams(params);
+      current = functionExpr;
+    }
+
+
+    return current;
+  }
+
+  /**
+   * This method parse CAST expression.
+   * This returns only expression field without casting type
+   * because Tajo doesn't provide CAST expression.
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitCastExpression(HiveQLParser.CastExpressionContext ctx) {
+    DataTypeExpr castTarget = getDataTypeExpr(ctx.primitiveType());
+    Expr expr = visitExpression(ctx.expression());
+    Expr current = new CastExpr(expr, castTarget);
+    return current;
+  }
+
+  @Override
+  public Expr visitCaseExpression(HiveQLParser.CaseExpressionContext ctx) {
+    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
+    Expr condition = null, result = null;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      if (ctx.getChild(i) instanceof TerminalNodeImpl) {
+        if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_WHEN) {
+          condition = null;
+          result = null;
+
+          if (ctx.getChild(i + 1) instanceof HiveQLParser.ExpressionContext) {
+            condition = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          }
+
+          if (ctx.getChild(i + 3) instanceof HiveQLParser.ExpressionContext) {
+            result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 3));
+          }
+
+          if (condition != null && result != null) {
+            caseWhen.addWhen(condition, result);
+          }
+        } else if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_ELSE) {
+          result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          caseWhen.setElseResult(result);
+        }
+      }
+    }
+
+    return caseWhen;
+  }
+
+  @Override
+  public Expr visitWhenExpression(HiveQLParser.WhenExpressionContext ctx) {
+    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
+    Expr condition = null, result = null;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      if (ctx.getChild(i) instanceof TerminalNodeImpl) {
+        if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_WHEN) {
+          condition = null;
+          result = null;
+
+          if (ctx.getChild(i + 1) instanceof HiveQLParser.ExpressionContext) {
+            condition = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          }
+
+          if (ctx.getChild(i + 3) instanceof HiveQLParser.ExpressionContext) {
+            result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 3));
+          }
+
+          if (condition != null && result != null) {
+            caseWhen.addWhen(condition, result);
+          }
+        } else if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_ELSE) {
+          result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          caseWhen.setElseResult(result);
+        }
+      }
+    }
+
+    return caseWhen;
+  }
+
+  @Override
+  public Aggregation visitGroupByClause(HiveQLParser.GroupByClauseContext ctx) {
+    Aggregation clause = new Aggregation();
+
+    if (ctx.groupByExpression().size() > 0) {
+      int elementSize = ctx.groupByExpression().size();
+      ArrayList<Aggregation.GroupElement> groups = new ArrayList<Aggregation.GroupElement>(elementSize + 1);
+      ArrayList<Expr> ordinaryExprs = new ArrayList<Expr>();
+      int groupSize = 1;
+      groups.add(null);
+
+      for (int i = 0; i < ctx.groupByExpression().size(); i++) {
+        Expr expr = visitGroupByExpression(ctx.groupByExpression(i));
+
+        if (expr instanceof FunctionExpr) {
+          FunctionExpr function = (FunctionExpr) expr;
+
+          if (function.getSignature().equalsIgnoreCase("ROLLUP")) {
+            groupSize++;
+            groups.add(new Aggregation.GroupElement(Aggregation.GroupType.Rollup,
+                function.getParams()));
+          } else if (function.getSignature().equalsIgnoreCase("CUBE")) {
+            groupSize++;
+            groups.add(new Aggregation.GroupElement(Aggregation.GroupType.Cube, function.getParams()));
+          } else {
+            Collections.addAll(ordinaryExprs, function);
+          }
+        } else {
+          Collections.addAll(ordinaryExprs, (ColumnReferenceExpr)expr);
+        }
+      }
+
+      if (ordinaryExprs != null) {
+        groups.set(0, new Aggregation.GroupElement(Aggregation.GroupType.OrdinaryGroup, ordinaryExprs.toArray(new Expr[ordinaryExprs.size()])));
+        clause.setGroups(groups.subList(0, groupSize).toArray(new Aggregation.GroupElement[groupSize]));
+      } else if (groupSize > 1) {
+        clause.setGroups(groups.subList(1, groupSize).toArray(new Aggregation.GroupElement[groupSize - 1]));
+      }
+    }
+
+    //TODO: grouping set expression
+    return clause;
+  }
+
+  @Override
+  public Sort visitOrderByClause(HiveQLParser.OrderByClauseContext ctx) {
+    Sort clause = null;
+    Sort.SortSpec[] specs = null;
+
+    if (ctx.columnRefOrder().size() > 0) {
+      specs = new Sort.SortSpec[ctx.columnRefOrder().size()];
+      for (int i = 0; i < ctx.columnRefOrder().size(); i++) {
+        ColumnReferenceExpr column = (ColumnReferenceExpr) visitExpression(ctx.columnRefOrder().get(i).expression());
+        specs[i] = new Sort.SortSpec(column);
+        if (ctx.columnRefOrder(i).KW_DESC() != null) {
+          specs[i].setDescending();
+        }
+      }
+      clause = new Sort(specs);
+    }
+    return clause;
+
+  }
+
+  @Override
+  public Expr visitHavingClause(HiveQLParser.HavingClauseContext ctx) {
+    return visitHavingCondition(ctx.havingCondition());
+  }
+
+  @Override
+  public Expr visitClusterByClause(HiveQLParser.ClusterByClauseContext ctx) {
+    // TODO: It needs to consider how to support.
+    return null;
+  }
+
+  @Override
+  public Expr visitDistributeByClause(HiveQLParser.DistributeByClauseContext ctx) {
+    // TODO: It needs to consider how to support.
+
+    return null;
+  }
+
+  @Override
+  public Sort visitSortByClause(HiveQLParser.SortByClauseContext ctx) {
+    Sort clause = null;
+    Sort.SortSpec[] specs = null;
+
+    if (ctx.columnRefOrder().size() > 0) {
+      specs = new Sort.SortSpec[ctx.columnRefOrder().size()];
+      for (int i = 0; i < ctx.columnRefOrder().size(); i++) {
+        ColumnReferenceExpr column = (ColumnReferenceExpr) visitColumnRefOrder(ctx.columnRefOrder(i));
+        specs[i] = new Sort.SortSpec(column);
+
+        if (ctx.columnRefOrder(i).KW_DESC() != null) {
+          specs[i].setDescending();
+        }
+      }
+      clause = new Sort(specs);
+    }
+
+    return clause;
+  }
+
+  @Override
+  public Limit visitLimitClause(HiveQLParser.LimitClauseContext ctx) {
+    LiteralValue expr = new LiteralValue(ctx.Number().getText(), LiteralValue.LiteralType.Unsigned_Integer);
+    Limit limit = new Limit(expr);
+    return limit;
+  }
+
+  @Override
+  public Expr visitWindow_clause(HiveQLParser.Window_clauseContext ctx) {
+    // TODO: It needs to consider how to support.
+    return null;
+  }
+
+  @Override
+  public Insert visitInsertClause(HiveQLParser.InsertClauseContext ctx) {
+    Insert insert = new Insert();
+    if (ctx.KW_OVERWRITE() != null)
+      insert.setOverwrite();
+
+    if (ctx.tableOrPartition() != null) {
+      HiveQLParser.TableOrPartitionContext partitionContext = ctx.tableOrPartition();
+      if (partitionContext.tableName() != null) {
+        insert.setTableName(ctx.tableOrPartition().tableName().getText());
+      }
+    }
+
+    if (ctx.destination() != null) {
+      HiveQLParser.DestinationContext destination = ctx.destination();
+      if (destination.KW_DIRECTORY() != null) {
+        String location = destination.StringLiteral().getText();
+        location = location.replaceAll("\\'", "");
+        insert.setLocation(location);
+      } else if (destination.KW_TABLE() != null) {
+        if (destination.tableOrPartition() != null) {
+          HiveQLParser.TableOrPartitionContext partitionContext = destination.tableOrPartition();
+          if (partitionContext.tableName() != null) {
+            insert.setTableName(partitionContext.tableName().getText());
+          }
+        }
+
+        if (destination.tableFileFormat() != null) {
+          if (destination.tableFileFormat().KW_RCFILE() != null) {
+            insert.setStorageType("rcfile");
+          } else if (destination.tableFileFormat().KW_TEXTFILE() != null) {
+            insert.setStorageType("csv");
+          }
+
+        }
+      }
+    }
+
+    return insert;
+  }
+
+  @Override
+  public Expr visitCreateTableStatement(HiveQLParser.CreateTableStatementContext ctx) {
+    CreateTable createTable = null;
+    Map<String, String> params = new HashMap<String, String>();
+
+    if (ctx.name != null) {
+      createTable = new CreateTable(ctx.name.getText(), ctx.ifNotExists() != null);
+      if (ctx.KW_EXTERNAL() != null) {
+        createTable.setExternal();
+      }
+
+      if (ctx.tableFileFormat() != null) {
+        if (ctx.tableFileFormat().KW_RCFILE() != null) {
+          createTable.setStorageType("rcfile");
+        } else if (ctx.tableFileFormat().KW_TEXTFILE() != null) {
+          createTable.setStorageType("csv");
+        }
+      }
+
+      if (ctx.tableRowFormat() != null) {
+        if (ctx.tableRowFormat().rowFormatDelimited() != null) {
+          String delimiter = ctx.tableRowFormat().rowFormatDelimited().tableRowFormatFieldIdentifier().getChild(3)
+              .getText().replaceAll("'", "");
+          params.put("csvfile.delimiter", SQLAnalyzer.escapeDelimiter(delimiter));
+        }
+      }
+
+      if (ctx.tableLocation() != null) {
+        String location = ctx.tableLocation().StringLiteral().getText();
+        location = location.replaceAll("'", "");
+        createTable.setLocation(location);
+
+      }
+
+      if (ctx.columnNameTypeList() != null) {
+        List<HiveQLParser.ColumnNameTypeContext> list = ctx.columnNameTypeList().columnNameType();
+
+        ColumnDefinition[] columns = new ColumnDefinition[list.size()];
+
+        for (int i = 0; i < list.size(); i++) {
+          HiveQLParser.ColumnNameTypeContext eachColumn = list.get(i);
+          String type = null;
+          if (eachColumn.colType().type() != null) {
+            if (eachColumn.colType().type().primitiveType() != null) {
+              HiveQLParser.PrimitiveTypeContext primitiveType = eachColumn.colType().type().primitiveType();
+              type = getDataTypeExpr(primitiveType).getTypeName();
+              columns[i] = new ColumnDefinition(eachColumn.colName.Identifier().getText(), type);
+            }
+          }
+        }
+        if (columns != null) {
+          createTable.setTableElements(columns);
+        }
+
+        if (!params.isEmpty()) {
+          createTable.setParams(params);
+        }
+      }
+    }
+
+    return createTable;
+  }
+
+
+  private DataTypeExpr getDataTypeExpr(HiveQLParser.PrimitiveTypeContext primitiveType) {
+    DataTypeExpr typeDefinition = null;
+
+    if (primitiveType.KW_STRING() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.TEXT.name());
+    } else if (primitiveType.KW_TINYINT() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.INT1.name());
+    } else if (primitiveType.KW_SMALLINT() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.INT2.name());
+    } else if (primitiveType.KW_INT() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.INT4.name());
+    } else if (primitiveType.KW_BIGINT() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.INT8.name());
+    } else if (primitiveType.KW_FLOAT() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.FLOAT4.name());
+    } else if (primitiveType.KW_DOUBLE() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.FLOAT8.name());
+    } else if (primitiveType.KW_DECIMAL() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.NUMERIC.name());
+    } else if (primitiveType.KW_BOOLEAN() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.BOOLEAN.name());
+    } else if (primitiveType.KW_DATE() != null) {
+    } else if (primitiveType.KW_DATETIME() != null) {
+      //TODO
+    } else if (primitiveType.KW_TIMESTAMP() != null) {
+      typeDefinition = new DataTypeExpr(TajoDataTypes.Type.TIMESTAMP.name());
+    }
+
+    return typeDefinition;
+  }
+
+
+  @Override
+  public Expr visitDropTableStatement(HiveQLParser.DropTableStatementContext ctx) {
+    DropTable dropTable = new DropTable(ctx.tableName().getText(), false, ctx.ifExists() != null);
+    return dropTable;
+  }
+
+  /**
+   * This class provides and implementation for a case insensitive token checker
+   * for the lexical analysis part of antlr. By converting the token stream into
+   * upper case at the time when lexical rules are checked, this class ensures that the
+   * lexical rules need to just match the token with upper case letters as opposed to
+   * combination of upper case and lower case characteres. This is purely used for matching lexical
+   * rules. The actual token text is stored in the same way as the user input without
+   * actually converting it into an upper case. The token values are generated by the consume()
+   * function of the super class ANTLRStringStream. The LA() function is the lookahead funtion
+   * and is purely used for matching lexical rules. This also means that the grammar will only
+   * accept capitalized tokens in case it is run from other tools like antlrworks which
+   * do not have the ANTLRNoCaseStringStream implementation.
+   */
+  public class ANTLRNoCaseStringStream extends ANTLRInputStream {
+
+    public ANTLRNoCaseStringStream(String input) {
+      super(input);
+    }
+
+    @Override
+    public int LA(int i) {
+
+      int returnChar = super.LA(i);
+      if (returnChar == CharStream.EOF) {
+        return returnChar;
+      } else if (returnChar == 0) {
+        return returnChar;
+      }
+
+      return Character.toUpperCase((char) returnChar);
+    }
+  }
+}
\ No newline at end of file


[30/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortBasedColPartitionStoreExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortBasedColPartitionStoreExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortBasedColPartitionStoreExec.java
new file mode 100644
index 0000000..8c55d7f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortBasedColPartitionStoreExec.java
@@ -0,0 +1,149 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.planner.logical.StoreTableNode;
+import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.StorageManagerFactory;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+/**
+ * It stores a sorted data set into a number of partition files. It assumes that input tuples are sorted in an
+ * ascending or descending order of partition columns.
+ */
+public class SortBasedColPartitionStoreExec extends ColPartitionStoreExec {
+  private static Log LOG = LogFactory.getLog(SortBasedColPartitionStoreExec.class);
+
+  private Tuple currentKey;
+  private Tuple prevKey;
+
+  private Appender appender;
+  private TableStats aggregated;
+
+  public SortBasedColPartitionStoreExec(TaskAttemptContext context, StoreTableNode plan, PhysicalExec child)
+      throws IOException {
+    super(context, plan, child);
+  }
+
+  public void init() throws IOException {
+    super.init();
+
+    currentKey = new VTuple(keyNum);
+    aggregated = new TableStats();
+  }
+
+  private Appender getAppender(String partition) throws IOException {
+    Path dataFile = getDataFile(partition);
+    FileSystem fs = dataFile.getFileSystem(context.getConf());
+
+    if (fs.exists(dataFile.getParent())) {
+      LOG.info("Path " + dataFile.getParent() + " already exists!");
+    } else {
+      fs.mkdirs(dataFile.getParent());
+      LOG.info("Add subpartition path directory :" + dataFile.getParent());
+    }
+
+    if (fs.exists(dataFile)) {
+      LOG.info("File " + dataFile + " already exists!");
+      FileStatus status = fs.getFileStatus(dataFile);
+      LOG.info("File size: " + status.getLen());
+    }
+
+    appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta, outSchema, dataFile);
+    appender.enableStats();
+    appender.init();
+
+    return appender;
+  }
+
+  private void fillKeyTuple(Tuple inTuple, Tuple keyTuple) {
+    for (int i = 0; i < keyIds.length; i++) {
+      keyTuple.put(i, inTuple.get(keyIds[i]));
+    }
+  }
+
+  private String getSubdirectory(Tuple keyTuple) {
+    StringBuilder sb = new StringBuilder();
+
+    for(int i = 0; i < keyIds.length; i++) {
+      Datum datum = keyTuple.get(i);
+      if(i > 0) {
+        sb.append("/");
+      }
+      sb.append(keyNames[i]).append("=");
+      sb.append(datum.asChars());
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple;
+    while((tuple = child.next()) != null) {
+
+      fillKeyTuple(tuple, currentKey);
+
+      if (prevKey == null) {
+        appender = getAppender(getSubdirectory(currentKey));
+        prevKey = new VTuple(currentKey);
+      } else {
+        if (!prevKey.equals(currentKey)) {
+          appender.close();
+          StatisticsUtil.aggregateTableStat(aggregated, appender.getStats());
+
+          appender = getAppender(getSubdirectory(currentKey));
+          prevKey = new VTuple(currentKey);
+        }
+      }
+
+      appender.addTuple(tuple);
+    }
+
+    return null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (appender != null) {
+      appender.close();
+      StatisticsUtil.aggregateTableStat(aggregated, appender.getStats());
+      context.setResultStats(aggregated);
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    // nothing to do
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortExec.java
new file mode 100644
index 0000000..a4a8d37
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortExec.java
@@ -0,0 +1,51 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+
+import java.io.IOException;
+import java.util.Comparator;
+
+public abstract class SortExec extends UnaryPhysicalExec {
+  private final Comparator<Tuple> comparator;
+  private final SortSpec [] sortSpecs;
+
+  public SortExec(TaskAttemptContext context, Schema inSchema,
+                  Schema outSchema, PhysicalExec child, SortSpec [] sortSpecs) {
+    super(context, inSchema, outSchema, child);
+    this.sortSpecs = sortSpecs;
+    this.comparator = new TupleComparator(inSchema, sortSpecs);
+  }
+
+  public SortSpec[] getSortSpecs() {
+    return sortSpecs;
+  }
+
+  public Comparator<Tuple> getComparator() {
+    return comparator;
+  }
+
+  @Override
+  abstract public Tuple next() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
new file mode 100644
index 0000000..1f927a6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
@@ -0,0 +1,102 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.engine.planner.logical.InsertNode;
+import org.apache.tajo.engine.planner.logical.PersistentStoreNode;
+import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.StorageManagerFactory;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+/**
+ * This is a physical executor to store a table part into a specified storage.
+ */
+public class StoreTableExec extends UnaryPhysicalExec {
+  private PersistentStoreNode plan;
+  private Appender appender;
+  private Tuple tuple;
+
+  public StoreTableExec(TaskAttemptContext context, PersistentStoreNode plan, PhysicalExec child) throws IOException {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+    this.plan = plan;
+  }
+
+  public void init() throws IOException {
+    super.init();
+
+    TableMeta meta;
+    if (plan.hasOptions()) {
+      meta = CatalogUtil.newTableMeta(plan.getStorageType(), plan.getOptions());
+    } else {
+      meta = CatalogUtil.newTableMeta(plan.getStorageType());
+    }
+
+    if (plan instanceof InsertNode) {
+      InsertNode createTableNode = (InsertNode) plan;
+      appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta,
+          createTableNode.getTableSchema(), context.getOutputPath());
+    } else {
+      appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta, outSchema,
+          context.getOutputPath());
+    }
+
+    appender.enableStats();
+    appender.init();
+  }
+
+  /* (non-Javadoc)
+   * @see PhysicalExec#next()
+   */
+  @Override
+  public Tuple next() throws IOException {
+    while((tuple = child.next()) != null) {
+      appender.addTuple(tuple);
+    }
+        
+    return null;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    // nothing to do
+  }
+
+  public void close() throws IOException {
+    super.close();
+
+    if(appender != null){
+      appender.flush();
+      appender.close();
+      // Collect statistics data
+      context.setResultStats(appender.getStats());
+      context.addShuffleFileOutput(0, context.getTaskId().toString());
+    }
+
+    appender = null;
+    plan = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/TunnelExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/TunnelExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/TunnelExec.java
new file mode 100644
index 0000000..fffcc39
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/TunnelExec.java
@@ -0,0 +1,42 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public class TunnelExec extends UnaryPhysicalExec {
+
+  public TunnelExec (final TaskAttemptContext context,
+                     final Schema outputSchema,
+                     final PhysicalExec child) {
+    super(context, outputSchema, outputSchema, child);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    return child.next();
+  }
+  @Override
+  public void rescan() throws IOException {   
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnaryPhysicalExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnaryPhysicalExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnaryPhysicalExec.java
new file mode 100644
index 0000000..ab67d7b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnaryPhysicalExec.java
@@ -0,0 +1,96 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public abstract class UnaryPhysicalExec extends PhysicalExec {
+  protected PhysicalExec child;
+  protected float progress;
+  protected TableStats inputStats;
+
+  public UnaryPhysicalExec(TaskAttemptContext context,
+                           Schema inSchema, Schema outSchema,
+                           PhysicalExec child) {
+    super(context, inSchema, outSchema);
+    this.child = child;
+  }
+
+  public <T extends PhysicalExec> T getChild() {
+    return (T) this.child;
+  }
+
+  @VisibleForTesting
+  public void setChild(PhysicalExec child){
+    this.child = child;
+  }
+
+  public void init() throws IOException {
+    progress = 0.0f;
+    if (child != null) {
+      child.init();
+    }
+  }
+
+  public void rescan() throws IOException {
+    progress = 0.0f;
+    if (child != null) {
+      child.rescan();
+    }
+  }
+
+  public void close() throws IOException {
+    progress = 1.0f;
+    if (child != null) {
+      child.close();
+      try {
+        TableStats stat = child.getInputStats();
+        if (stat != null) {
+          inputStats = (TableStats)(stat.clone());
+        }
+      } catch (CloneNotSupportedException e) {
+        e.printStackTrace();
+      }
+      child = null;
+    }
+  }
+
+  @Override
+  public float getProgress() {
+    if (child != null) {
+      return child.getProgress();
+    } else {
+      return progress;
+    }
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    if (child != null) {
+      return child.getInputStats();
+    } else {
+      return inputStats;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnionExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnionExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnionExec.java
new file mode 100644
index 0000000..497c6d3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/UnionExec.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * 
+ */
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.engine.exception.InvalidQueryException;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public class UnionExec extends BinaryPhysicalExec {
+  private boolean nextOuter = true;
+  private Tuple tuple;
+
+  public UnionExec(TaskAttemptContext context, PhysicalExec outer, PhysicalExec inner) {
+    super(context, outer.getSchema(), inner.getSchema(), outer, inner);
+    if (!outer.getSchema().equals(inner.getSchema())) {
+      throw new InvalidQueryException(
+          "The both schemas are not same");
+    }
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (nextOuter) {
+      tuple = leftChild.next();
+      if (tuple == null) {
+       nextOuter = false; 
+      } else {
+        return tuple;
+      }
+    }
+    
+    return rightChild.next();
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+
+    nextOuter = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/BasicQueryRewriteEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/BasicQueryRewriteEngine.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/BasicQueryRewriteEngine.java
new file mode 100644
index 0000000..3b4b712
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/BasicQueryRewriteEngine.java
@@ -0,0 +1,70 @@
+/**
+ * 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.planner.rewrite;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.PlanningException;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * This is a basic query rewrite rule engine. This rewrite rule engine
+ * rewrites a logical plan with various query rewrite rules.
+ */
+public class BasicQueryRewriteEngine implements QueryRewriteEngine {
+  /** class logger */
+  private Log LOG = LogFactory.getLog(BasicQueryRewriteEngine.class);
+
+  /** a map for query rewrite rules  */
+  private Map<String, RewriteRule> rewriteRules = new LinkedHashMap<String, RewriteRule>();
+
+  /**
+   * Add a query rewrite rule to this engine.
+   *
+   * @param rule The rule to be added to this engine.
+   */
+  public void addRewriteRule(RewriteRule rule) {
+    if (!rewriteRules.containsKey(rule.getName())) {
+      rewriteRules.put(rule.getName(), rule);
+    }
+  }
+
+  /**
+   * Rewrite a logical plan with all query rewrite rules added to this engine.
+   *
+   * @param plan The plan to be rewritten with all query rewrite rule.
+   * @return The rewritten plan.
+   */
+  public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
+    RewriteRule rule;
+    for (Entry<String, RewriteRule> rewriteRule : rewriteRules.entrySet()) {
+      rule = rewriteRule.getValue();
+      if (rule.isEligible(plan)) {
+        plan = rule.rewrite(plan);
+        LOG.info("The rule \"" + rule.getName() + " \" rewrites the query.");
+      }
+    }
+
+    return plan;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java
new file mode 100644
index 0000000..63b426f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java
@@ -0,0 +1,323 @@
+/**
+ * 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.planner.rewrite;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.exception.InvalidQueryException;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+public class FilterPushDownRule extends BasicLogicalPlanVisitor<Set<EvalNode>, LogicalNode> implements RewriteRule {
+  private static final String NAME = "FilterPushDown";
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public boolean isEligible(LogicalPlan plan) {
+    for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+      if (block.hasNode(NodeType.SELECTION) || block.hasNode(NodeType.JOIN)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
+    for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+      this.visit(new HashSet<EvalNode>(), plan, block, block.getRoot(), new Stack<LogicalNode>());
+    }
+
+    return plan;
+  }
+
+  @Override
+  public LogicalNode visitFilter(Set<EvalNode> cnf, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 SelectionNode selNode, Stack<LogicalNode> stack) throws PlanningException {
+    cnf.addAll(Sets.newHashSet(AlgebraicUtil.toConjunctiveNormalFormArray(selNode.getQual())));
+
+    stack.push(selNode);
+    visit(cnf, plan, block, selNode.getChild(), stack);
+    stack.pop();
+
+    if(cnf.size() == 0) { // remove the selection operator if there is no search condition after selection push.
+      LogicalNode node = stack.peek();
+      if (node instanceof UnaryNode) {
+        UnaryNode unary = (UnaryNode) node;
+        unary.setChild(selNode.getChild());
+      } else {
+        throw new InvalidQueryException("Unexpected Logical Query Plan");
+      }
+    } else { // if there remain search conditions
+
+      // check if it can be evaluated here
+      Set<EvalNode> matched = TUtil.newHashSet();
+      for (EvalNode eachEval : cnf) {
+        if (LogicalPlanner.checkIfBeEvaluatedAtThis(eachEval, selNode)) {
+          matched.add(eachEval);
+        }
+      }
+
+      // if there are search conditions which can be evaluated here, push down them and remove them from cnf.
+      if (matched.size() > 0) {
+        selNode.setQual(AlgebraicUtil.createSingletonExprFromCNF(matched.toArray(new EvalNode[matched.size()])));
+        cnf.removeAll(matched);
+      }
+    }
+
+    return selNode;
+  }
+
+  private boolean isOuterJoin(JoinType joinType) {
+    return joinType == JoinType.LEFT_OUTER || joinType == JoinType.RIGHT_OUTER || joinType==JoinType.FULL_OUTER;
+  }
+
+  @Override
+  public LogicalNode visitJoin(Set<EvalNode> cnf, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode joinNode,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    LogicalNode left = joinNode.getRightChild();
+    LogicalNode right = joinNode.getLeftChild();
+
+    // here we should stop selection pushdown on the null supplying side(s) of an outer join
+    // get the two operands of the join operation as well as the join type
+    JoinType joinType = joinNode.getJoinType();
+    EvalNode joinQual = joinNode.getJoinQual();
+    if (joinQual != null && isOuterJoin(joinType)) {
+
+      // if both are fields
+      if (joinQual.getLeftExpr().getType() == EvalType.FIELD && joinQual.getRightExpr().getType() == EvalType.FIELD) {
+
+        String leftTableName = ((FieldEval) joinQual.getLeftExpr()).getQualifier();
+        String rightTableName = ((FieldEval) joinQual.getRightExpr()).getQualifier();
+        List<String> nullSuppliers = Lists.newArrayList();
+        Set<String> leftTableSet = Sets.newHashSet(PlannerUtil.getRelationLineageWithinQueryBlock(plan,
+            joinNode.getLeftChild()));
+        Set<String> rightTableSet = Sets.newHashSet(PlannerUtil.getRelationLineageWithinQueryBlock(plan,
+            joinNode.getRightChild()));
+
+        // some verification
+        if (joinType == JoinType.FULL_OUTER) {
+          nullSuppliers.add(leftTableName);
+          nullSuppliers.add(rightTableName);
+
+          // verify that these null suppliers are indeed in the left and right sets
+          if (!rightTableSet.contains(nullSuppliers.get(0)) && !leftTableSet.contains(nullSuppliers.get(0))) {
+            throw new InvalidQueryException("Incorrect Logical Query Plan with regard to outer join");
+          }
+          if (!rightTableSet.contains(nullSuppliers.get(1)) && !leftTableSet.contains(nullSuppliers.get(1))) {
+            throw new InvalidQueryException("Incorrect Logical Query Plan with regard to outer join");
+          }
+
+        } else if (joinType == JoinType.LEFT_OUTER) {
+          nullSuppliers.add(((RelationNode)joinNode.getRightChild()).getCanonicalName());
+          //verify that this null supplier is indeed in the right sub-tree
+          if (!rightTableSet.contains(nullSuppliers.get(0))) {
+            throw new InvalidQueryException("Incorrect Logical Query Plan with regard to outer join");
+          }
+        } else if (joinType == JoinType.RIGHT_OUTER) {
+          if (((RelationNode)joinNode.getRightChild()).getCanonicalName().equals(rightTableName)) {
+            nullSuppliers.add(leftTableName);
+          } else {
+            nullSuppliers.add(rightTableName);
+          }
+
+          // verify that this null supplier is indeed in the left sub-tree
+          if (!leftTableSet.contains(nullSuppliers.get(0))) {
+            throw new InvalidQueryException("Incorrect Logical Query Plan with regard to outer join");
+          }
+        }
+
+        // retain in this outer join node's JoinQual those selection predicates
+        // related to the outer join's null supplier(s)
+        List<EvalNode> matched2 = Lists.newArrayList();
+        for (EvalNode eval : cnf) {
+
+          Set<Column> columnRefs = EvalTreeUtil.findUniqueColumns(eval);
+          Set<String> tableNames = Sets.newHashSet();
+          // getting distinct table references
+          for (Column col : columnRefs) {
+            if (!tableNames.contains(col.getQualifier())) {
+              tableNames.add(col.getQualifier());
+            }
+          }
+
+          //if the predicate involves any of the null suppliers
+          boolean shouldKeep=false;
+          Iterator<String> it2 = nullSuppliers.iterator();
+          while(it2.hasNext()){
+            if(tableNames.contains(it2.next()) == true) {
+              shouldKeep = true;
+            }
+          }
+
+          if(shouldKeep == true) {
+            matched2.add(eval);
+          }
+
+        }
+
+        //merge the retained predicates and establish them in the current outer join node. Then remove them from the cnf
+        EvalNode qual2 = null;
+        if (matched2.size() > 1) {
+          // merged into one eval tree
+          qual2 = AlgebraicUtil.createSingletonExprFromCNF(
+              matched2.toArray(new EvalNode[matched2.size()]));
+        } else if (matched2.size() == 1) {
+          // if the number of matched expr is one
+          qual2 = matched2.get(0);
+        }
+
+        if (qual2 != null) {
+          EvalNode conjQual2 = AlgebraicUtil.createSingletonExprFromCNF(joinNode.getJoinQual(), qual2);
+          joinNode.setJoinQual(conjQual2);
+          cnf.removeAll(matched2);
+        } // for the remaining cnf, push it as usual
+      }
+    }
+
+    if (joinNode.hasJoinQual()) {
+      cnf.addAll(Sets.newHashSet(AlgebraicUtil.toConjunctiveNormalFormArray(joinNode.getJoinQual())));
+    }
+
+    visit(cnf, plan, block, left, stack);
+    visit(cnf, plan, block, right, stack);
+
+    List<EvalNode> matched = Lists.newArrayList();
+    for (EvalNode eval : cnf) {
+      if (LogicalPlanner.checkIfBeEvaluatedAtJoin(block, eval, joinNode, stack.peek().getType() != NodeType.JOIN)) {
+        matched.add(eval);
+      }
+    }
+
+    EvalNode qual = null;
+    if (matched.size() > 1) {
+      // merged into one eval tree
+      qual = AlgebraicUtil.createSingletonExprFromCNF(
+          matched.toArray(new EvalNode[matched.size()]));
+    } else if (matched.size() == 1) {
+      // if the number of matched expr is one
+      qual = matched.get(0);
+    }
+
+    if (qual != null) {
+      joinNode.setJoinQual(qual);
+
+      if (joinNode.getJoinType() == JoinType.CROSS) {
+        joinNode.setJoinType(JoinType.INNER);
+      }
+      cnf.removeAll(matched);
+    }
+
+    return joinNode;
+  }
+
+  @Override
+  public LogicalNode visitTableSubQuery(Set<EvalNode> cnf, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                        TableSubQueryNode node, Stack<LogicalNode> stack) throws PlanningException {
+    List<EvalNode> matched = Lists.newArrayList();
+    for (EvalNode eval : cnf) {
+      if (LogicalPlanner.checkIfBeEvaluatedAtRelation(block, eval, node)) {
+        matched.add(eval);
+      }
+    }
+
+    Map<String, String> columnMap = new HashMap<String, String>();
+    for (int i = 0; i < node.getInSchema().size(); i++) {
+      LogicalNode childNode = node.getSubQuery();
+      if (childNode.getOutSchema().getColumn(i).hasQualifier()) {
+      columnMap.put(node.getInSchema().getColumn(i).getQualifiedName(),
+          childNode.getOutSchema().getColumn(i).getQualifiedName());
+      } else {
+        NamedExprsManager namedExprsMgr = plan.getBlock(node.getSubQuery()).getNamedExprsManager();
+        columnMap.put(node.getInSchema().getColumn(i).getQualifiedName(),
+          namedExprsMgr.getOriginalName(childNode.getOutSchema().getColumn(i).getQualifiedName()));
+      }
+    }
+
+    Set<EvalNode> transformed = new HashSet<EvalNode>();
+
+    // Rename from upper block's one to lower block's one
+    for (EvalNode matchedEval : matched) {
+      EvalNode copy;
+      try {
+        copy = (EvalNode) matchedEval.clone();
+      } catch (CloneNotSupportedException e) {
+        throw new PlanningException(e);
+      }
+
+      Set<Column> columns = EvalTreeUtil.findUniqueColumns(copy);
+      for (Column c : columns) {
+        if (columnMap.containsKey(c.getQualifiedName())) {
+          EvalTreeUtil.changeColumnRef(copy, c.getQualifiedName(), columnMap.get(c.getQualifiedName()));
+        } else {
+          throw new PlanningException(
+              "Invalid Filter PushDown on SubQuery: No such a corresponding column '"
+                  + c.getQualifiedName());
+        }
+      }
+
+      transformed.add(copy);
+    }
+
+    visit(transformed, plan, plan.getBlock(node.getSubQuery()));
+
+    cnf.removeAll(matched);
+
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitScan(Set<EvalNode> cnf, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode scanNode,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    List<EvalNode> matched = Lists.newArrayList();
+    for (EvalNode eval : cnf) {
+      if (LogicalPlanner.checkIfBeEvaluatedAtRelation(block, eval, scanNode)) {
+        matched.add(eval);
+      }
+    }
+
+    EvalNode qual = null;
+    if (matched.size() > 1) {
+      // merged into one eval tree
+      qual = AlgebraicUtil.createSingletonExprFromCNF(
+          matched.toArray(new EvalNode[matched.size()]));
+    } else if (matched.size() == 1) {
+      // if the number of matched expr is one
+      qual = matched.get(0);
+    }
+
+    if (qual != null) { // if a matched qual exists
+      scanNode.setQual(qual);
+    }
+
+    cnf.removeAll(matched);
+
+    return scanNode;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/PartitionedTableRewriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/PartitionedTableRewriter.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/PartitionedTableRewriter.java
new file mode 100644
index 0000000..6e78c18
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/PartitionedTableRewriter.java
@@ -0,0 +1,373 @@
+/**
+ * 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.planner.rewrite;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.*;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.planner.BasicLogicalPlanVisitor;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.Stack;
+
+public class PartitionedTableRewriter implements RewriteRule {
+  private static final Log LOG = LogFactory.getLog(PartitionedTableRewriter.class);
+
+  private static final String NAME = "Partitioned Table Rewriter";
+  private final Rewriter rewriter = new Rewriter();
+
+  private final TajoConf systemConf;
+
+  public PartitionedTableRewriter(TajoConf conf) {
+    systemConf = conf;
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public boolean isEligible(LogicalPlan plan) {
+    for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+      for (RelationNode relation : block.getRelations()) {
+        if (relation.getType() == NodeType.SCAN) {
+          TableDesc table = ((ScanNode)relation).getTableDesc();
+          if (table.hasPartition()) {
+            return true;
+          }
+        }
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
+    boolean containsPartitionedTables;
+    for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+      containsPartitionedTables = false;
+      for (RelationNode relation : block.getRelations()) {
+        if (relation.getType() == NodeType.SCAN) {
+          TableDesc table = ((ScanNode)relation).getTableDesc();
+          if (table.hasPartition()) {
+            containsPartitionedTables = true;
+          }
+        }
+      }
+      if (containsPartitionedTables) {
+        rewriter.visit(block, plan, block, block.getRoot(), new Stack<LogicalNode>());
+      }
+    }
+    return plan;
+  }
+
+  private static class PartitionPathFilter implements PathFilter {
+    private Schema schema;
+    private EvalNode partitionFilter;
+
+
+    public PartitionPathFilter(Schema schema, EvalNode partitionFilter) {
+      this.schema = schema;
+      this.partitionFilter = partitionFilter;
+    }
+
+    @Override
+    public boolean accept(Path path) {
+      Tuple tuple = TupleUtil.buildTupleFromPartitionPath(schema, path, true);
+      if (tuple == null) { // if it is a file or not acceptable file
+        return false;
+      }
+
+      return partitionFilter.eval(schema, tuple).asBool();
+    }
+
+    @Override
+    public String toString() {
+      return partitionFilter.toString();
+    }
+  }
+
+  /**
+   * It assumes that each conjunctive form corresponds to one column.
+   *
+   * @param partitionColumns
+   * @param conjunctiveForms search condition corresponding to partition columns.
+   *                         If it is NULL, it means that there is no search condition for this table.
+   * @param tablePath
+   * @return
+   * @throws IOException
+   */
+  private Path [] findFilteredPaths(Schema partitionColumns, EvalNode [] conjunctiveForms, Path tablePath)
+      throws IOException {
+
+    FileSystem fs = tablePath.getFileSystem(systemConf);
+
+    PathFilter [] filters;
+    if (conjunctiveForms == null) {
+      filters = buildAllAcceptingPathFilters(partitionColumns);
+    } else {
+      filters = buildPathFiltersForAllLevels(partitionColumns, conjunctiveForms);
+    }
+
+    // loop from one to the number of partition columns
+    Path [] filteredPaths = toPathArray(fs.listStatus(tablePath, filters[0]));
+
+    for (int i = 1; i < partitionColumns.size(); i++) {
+      // Get all file status matched to a ith level path filter.
+      filteredPaths = toPathArray(fs.listStatus(filteredPaths, filters[i]));
+    }
+
+    LOG.info("Filtered directory or files: " + filteredPaths.length);
+    return filteredPaths;
+  }
+
+  /**
+   * Build path filters for all levels with a list of filter conditions.
+   *
+   * For example, consider you have a partitioned table for three columns (i.e., col1, col2, col3).
+   * Then, this methods will create three path filters for (col1), (col1, col2), (col1, col2, col3).
+   *
+   * Corresponding filter conditions will be placed on each path filter,
+   * If there is no corresponding expression for certain column,
+   * The condition will be filled with a true value.
+   *
+   * Assume that an user gives a condition WHERE col1 ='A' and col3 = 'C'.
+   * There is no filter condition corresponding to col2.
+   * Then, the path filter conditions are corresponding to the followings:
+   *
+   * The first path filter: col1 = 'A'
+   * The second path filter: col1 = 'A' AND col2 IS NOT NULL
+   * The third path filter: col1 = 'A' AND col2 IS NOT NULL AND col3 = 'C'
+   *
+   * 'IS NOT NULL' predicate is always true against the partition path.
+   *
+   * @param partitionColumns
+   * @param conjunctiveForms
+   * @return
+   */
+  private static PathFilter [] buildPathFiltersForAllLevels(Schema partitionColumns,
+                                                     EvalNode [] conjunctiveForms) {
+    // Building partition path filters for all levels
+    Column target;
+    PathFilter [] filters = new PathFilter[partitionColumns.size()];
+    List<EvalNode> accumulatedFilters = Lists.newArrayList();
+    for (int i = 0; i < partitionColumns.size(); i++) { // loop from one to level
+      target = partitionColumns.getColumn(i);
+
+      for (EvalNode expr : conjunctiveForms) {
+        if (EvalTreeUtil.findUniqueColumns(expr).contains(target)) {
+          // Accumulate one qual per level
+          accumulatedFilters.add(expr);
+        }
+      }
+
+      if (accumulatedFilters.size() < (i + 1)) {
+        accumulatedFilters.add(new IsNullEval(true, new FieldEval(target)));
+      }
+
+      EvalNode filterPerLevel = AlgebraicUtil.createSingletonExprFromCNF(
+          accumulatedFilters.toArray(new EvalNode[accumulatedFilters.size()]));
+      filters[i] = new PartitionPathFilter(partitionColumns, filterPerLevel);
+    }
+    return filters;
+  }
+
+  /**
+   * Build an array of path filters for all levels with all accepting filter condition.
+   * @param partitionColumns The partition columns schema
+   * @return The array of path filter, accpeting all partition paths.
+   */
+  private static PathFilter [] buildAllAcceptingPathFilters(Schema partitionColumns) {
+    Column target;
+    PathFilter [] filters = new PathFilter[partitionColumns.size()];
+    List<EvalNode> accumulatedFilters = Lists.newArrayList();
+    for (int i = 0; i < partitionColumns.size(); i++) { // loop from one to level
+      target = partitionColumns.getColumn(i);
+      accumulatedFilters.add(new IsNullEval(true, new FieldEval(target)));
+
+      EvalNode filterPerLevel = AlgebraicUtil.createSingletonExprFromCNF(
+          accumulatedFilters.toArray(new EvalNode[accumulatedFilters.size()]));
+      filters[i] = new PartitionPathFilter(partitionColumns, filterPerLevel);
+    }
+    return filters;
+  }
+
+  private static Path [] toPathArray(FileStatus[] fileStatuses) {
+    Path [] paths = new Path[fileStatuses.length];
+    for (int j = 0; j < fileStatuses.length; j++) {
+      paths[j] = fileStatuses[j].getPath();
+    }
+    return paths;
+  }
+
+  private Path [] findFilteredPartitionPaths(ScanNode scanNode) throws IOException {
+    TableDesc table = scanNode.getTableDesc();
+    PartitionMethodDesc partitionDesc = scanNode.getTableDesc().getPartitionMethod();
+
+    Schema paritionValuesSchema = new Schema();
+    for (Column column : partitionDesc.getExpressionSchema().getColumns()) {
+      paritionValuesSchema.addColumn(column);
+    }
+
+    Set<EvalNode> indexablePredicateSet = Sets.newHashSet();
+
+    // if a query statement has a search condition, try to find indexable predicates
+    if (scanNode.hasQual()) {
+      EvalNode [] conjunctiveForms = AlgebraicUtil.toConjunctiveNormalFormArray(scanNode.getQual());
+      Set<EvalNode> remainExprs = Sets.newHashSet(conjunctiveForms);
+
+      // add qualifier to schema for qual
+      paritionValuesSchema.setQualifier(scanNode.getCanonicalName());
+      for (Column column : paritionValuesSchema.getColumns()) {
+        for (EvalNode simpleExpr : conjunctiveForms) {
+          if (checkIfIndexablePredicateOnTargetColumn(simpleExpr, column)) {
+            indexablePredicateSet.add(simpleExpr);
+          }
+        }
+      }
+
+      // Partitions which are not matched to the partition filter conditions are pruned immediately.
+      // So, the partition filter conditions are not necessary later, and they are removed from
+      // original search condition for simplicity and efficiency.
+      remainExprs.removeAll(indexablePredicateSet);
+      if (remainExprs.isEmpty()) {
+        scanNode.setQual(null);
+      } else {
+        scanNode.setQual(
+            AlgebraicUtil.createSingletonExprFromCNF(remainExprs.toArray(new EvalNode[remainExprs.size()])));
+      }
+    }
+
+    if (indexablePredicateSet.size() > 0) { // There are at least one indexable predicates
+      return findFilteredPaths(paritionValuesSchema,
+          indexablePredicateSet.toArray(new EvalNode[indexablePredicateSet.size()]), table.getPath());
+    } else { // otherwise, we will get all partition paths.
+      return findFilteredPaths(paritionValuesSchema, null, table.getPath());
+    }
+  }
+
+  private boolean checkIfIndexablePredicateOnTargetColumn(EvalNode evalNode, Column targetColumn) {
+    if (checkIfIndexablePredicate(evalNode) || checkIfDisjunctiveButOneVariable(evalNode)) {
+      Set<Column> variables = EvalTreeUtil.findUniqueColumns(evalNode);
+      // if it contains only single variable matched to a target column
+      return variables.size() == 1 && variables.contains(targetColumn);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Check if an expression consists of one variable and one constant and
+   * the expression is a comparison operator.
+   *
+   * @param evalNode The expression to be checked
+   * @return true if an expression consists of one variable and one constant
+   * and the expression is a comparison operator. Other, false.
+   */
+  private boolean checkIfIndexablePredicate(EvalNode evalNode) {
+    // TODO - LIKE with a trailing wild-card character and IN with an array can be indexable
+    return AlgebraicUtil.containSingleVar(evalNode) && AlgebraicUtil.isIndexableOperator(evalNode);
+  }
+
+  /**
+   *
+   * @param evalNode The expression to be checked
+   * @return true if an disjunctive expression, consisting of indexable expressions
+   */
+  private boolean checkIfDisjunctiveButOneVariable(EvalNode evalNode) {
+    if (evalNode.getType() == EvalType.OR) {
+      boolean indexable =
+          checkIfIndexablePredicate(evalNode.getLeftExpr()) &&
+              checkIfIndexablePredicate(evalNode.getRightExpr());
+
+      boolean sameVariable =
+          EvalTreeUtil.findUniqueColumns(evalNode.getLeftExpr())
+          .equals(EvalTreeUtil.findUniqueColumns(evalNode.getRightExpr()));
+
+      return indexable && sameVariable;
+    } else {
+      return false;
+    }
+  }
+
+  private void updateTableStat(PartitionedTableScanNode scanNode) throws PlanningException {
+    if (scanNode.getInputPaths().length > 0) {
+      try {
+        FileSystem fs = scanNode.getInputPaths()[0].getFileSystem(systemConf);
+        long totalVolume = 0;
+
+        for (Path input : scanNode.getInputPaths()) {
+          ContentSummary summary = fs.getContentSummary(input);
+          totalVolume += summary.getLength();
+          totalVolume += summary.getFileCount();
+        }
+        scanNode.getTableDesc().getStats().setNumBytes(totalVolume);
+      } catch (IOException e) {
+        throw new PlanningException(e);
+      }
+    }
+  }
+
+  private final class Rewriter extends BasicLogicalPlanVisitor<Object, Object> {
+    @Override
+    public Object visitScan(Object object, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode scanNode,
+                            Stack<LogicalNode> stack) throws PlanningException {
+
+      TableDesc table = scanNode.getTableDesc();
+      if (!table.hasPartition()) {
+        return null;
+      }
+
+      try {
+        Path [] filteredPaths = findFilteredPartitionPaths(scanNode);
+        plan.addHistory("PartitionTableRewriter chooses " + filteredPaths.length + " of partitions");
+        PartitionedTableScanNode rewrittenScanNode = plan.createNode(PartitionedTableScanNode.class);
+        rewrittenScanNode.init(scanNode, filteredPaths);
+        updateTableStat(rewrittenScanNode);
+
+        // if it is topmost node, set it as the rootnode of this block.
+        if (stack.empty()) {
+          block.setRoot(rewrittenScanNode);
+        } else {
+          PlannerUtil.replaceNode(plan, stack.peek(), scanNode, rewrittenScanNode);
+        }
+      } catch (IOException e) {
+        throw new PlanningException("Partitioned Table Rewrite Failed: \n" + e.getMessage());
+      }
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java
new file mode 100644
index 0000000..668ed68
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java
@@ -0,0 +1,966 @@
+/**
+ * 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.planner.rewrite;
+
+import com.google.common.collect.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+/**
+ * ProjectionPushDownRule deploys expressions in a selection list to proper
+ * {@link org.apache.tajo.engine.planner.logical.Projectable}
+ * nodes. In this process, the expressions are usually pushed down into as lower as possible.
+ * It also enables scanners to read only necessary columns.
+ */
+public class ProjectionPushDownRule extends
+    BasicLogicalPlanVisitor<ProjectionPushDownRule.Context, LogicalNode> implements RewriteRule {
+  /** Class Logger */
+  private final Log LOG = LogFactory.getLog(ProjectionPushDownRule.class);
+  private static final String name = "ProjectionPushDown";
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public boolean isEligible(LogicalPlan plan) {
+    LogicalNode toBeOptimized = plan.getRootBlock().getRoot();
+
+    if (PlannerUtil.checkIfDDLPlan(toBeOptimized) || !plan.getRootBlock().hasTableExpression()) {
+      LOG.info("This query skips the logical optimization step.");
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
+    LogicalPlan.QueryBlock rootBlock = plan.getRootBlock();
+
+    LogicalPlan.QueryBlock topmostBlock = rootBlock;
+
+    Stack<LogicalNode> stack = new Stack<LogicalNode>();
+    Context context = new Context(plan);
+    visit(context, plan, topmostBlock, topmostBlock.getRoot(), stack);
+
+    return plan;
+  }
+
+  /**
+   * <h2>What is TargetListManager?</h2>
+   * It manages all expressions used in a query block, and their reference names.
+   * TargetListManager provides a way to find an expression by a reference name.
+   * It keeps a set of expressions, and one or more reference names can point to
+   * the same expression.
+   *
+   * Also, TargetListManager keeps the evaluation state of each expression.
+   * The evaluation state is a boolean state to indicate whether the expression
+   * was evaluated in descendant node or not. If an expression is evaluated,
+   * the evaluation state is changed to TRUE. It also means that
+   * the expression can be referred by an column reference instead of evaluating the expression.
+   *
+   * Consider an example query:
+   *
+   * SELECT sum(l_orderkey + 1) from lineitem where l_partkey > 1;
+   *
+   * In this case, an expression sum(l_orderkey + 1) is divided into two sub expressions:
+   * <ul>
+   *  <li>$1 <- l_orderkey + 1</li>
+   *  <li>$2 <- sum($1)</li>
+   * </ul>
+   *
+   * <code>$1</code> is a simple arithmetic operation, and $2 is an aggregation function.
+   * <code>$1</code> is evaluated in ScanNode because it's just a simple arithmetic operation.
+   * So, the evaluation state of l_orderkey + 1 initially
+   * is false, but the state will be true after ScanNode.
+   *
+   * In contrast, sum($1) is evaluated at GroupbyNode. So, its evaluation state is changed
+   * after GroupByNode.
+   *
+   * <h2>Why is TargetListManager necessary?</h2>
+   *
+   * Expressions used in a query block can be divided into various categories according to
+   * the possible {@link Projectable} nodes. Their references become available depending on
+   * the Projectable node at which expressions are evaluated. It manages the expressions and
+   * references for optimized places of expressions. It performs duplicated removal and enables
+   * common expressions to be shared with two or more Projectable nodes. It also helps Projectable
+   * nodes to find correct column references.
+   */
+  public static class TargetListManager {
+    private Integer seqId = 0;
+
+    /**
+     * Why should we use LinkedHashMap for those maps ?
+     *
+     * These maps are mainly by the target list of each projectable node
+     * (i.e., ProjectionNode, GroupbyNode, JoinNode, and ScanNode).
+     * The projection node removal occurs only when the projection node's output
+     * schema and its child's output schema are equivalent to each other.
+     *
+     * If we keep the inserted order of all expressions. It would make the possibility
+     * of projection node removal higher.
+     **/
+
+    /** A Map: Name -> Id */
+    private LinkedHashMap<String, Integer> nameToIdBiMap;
+    /** Map: Id <-> EvalNode */
+    private BiMap<Integer, EvalNode> idToEvalBiMap;
+    /** Map: Id -> Names */
+    private LinkedHashMap<Integer, List<String>> idToNamesMap;
+    /** Map: Name -> Boolean */
+    private LinkedHashMap<String, Boolean> evaluationStateMap;
+
+    private LogicalPlan plan;
+
+    public TargetListManager(LogicalPlan plan) {
+      this.plan = plan;
+      nameToIdBiMap = Maps.newLinkedHashMap();
+      idToEvalBiMap = HashBiMap.create();
+      idToNamesMap = Maps.newLinkedHashMap();
+      evaluationStateMap = Maps.newLinkedHashMap();
+    }
+
+    private int getNextSeqId() {
+      return seqId++;
+    }
+
+    /**
+     * Add an expression with a specified name, which is usually an alias.
+     * Later, you can refer this expression by the specified name.
+     */
+    private String add(String specifiedName, EvalNode evalNode) throws PlanningException {
+
+      // if a name already exists, it only just keeps an actual
+      // expression instead of a column reference.
+      if (nameToIdBiMap.containsKey(specifiedName)) {
+        int refId = nameToIdBiMap.get(specifiedName);
+        EvalNode found = idToEvalBiMap.get(refId);
+        if (found != null && !evalNode.equals(found)) {
+          if (found.getType() != EvalType.FIELD && evalNode.getType() != EvalType.FIELD) {
+            throw new PlanningException("Duplicate alias: " + evalNode);
+          }
+          if (found.getType() == EvalType.FIELD) {
+            idToEvalBiMap.forcePut(refId, evalNode);
+          }
+        }
+      }
+
+      int refId;
+      if (idToEvalBiMap.inverse().containsKey(evalNode)) {
+        refId = idToEvalBiMap.inverse().get(evalNode);
+      } else {
+        refId = getNextSeqId();
+        idToEvalBiMap.put(refId, evalNode);
+      }
+
+      nameToIdBiMap.put(specifiedName, refId);
+      TUtil.putToNestedList(idToNamesMap, refId, specifiedName);
+      evaluationStateMap.put(specifiedName, false);
+
+      for (Column column : EvalTreeUtil.findUniqueColumns(evalNode)) {
+        add(new FieldEval(column));
+      }
+
+      return specifiedName;
+    }
+
+    /**
+     * Adds an expression without any name. It returns an automatically
+     * generated name. It can be also used for referring this expression.
+     */
+    public String add(EvalNode evalNode) throws PlanningException {
+      String name;
+
+      if (idToEvalBiMap.inverse().containsKey(evalNode)) {
+        int refId = idToEvalBiMap.inverse().get(evalNode);
+        return getPrimaryName(refId);
+      }
+
+      if (evalNode.getType() == EvalType.FIELD) {
+        FieldEval fieldEval = (FieldEval) evalNode;
+        name = fieldEval.getName();
+      } else {
+        name = plan.generateUniqueColumnName(evalNode);
+      }
+
+      return add(name, evalNode);
+    }
+
+    public Collection<String> getNames() {
+      return nameToIdBiMap.keySet();
+    }
+
+    public String add(Target target) throws PlanningException {
+      return add(target.getCanonicalName(), target.getEvalTree());
+    }
+
+    /**
+     * Each expression can have one or more names.
+     * We call a name added with an expression firstly as the primary name.
+     * It has a special meaning. Since duplicated expression in logical planning are removed,
+     * the primary name is only used for each expression during logical planning.
+     *
+     * @param refId The identifier of an expression
+     * @param name The name to check if it is the primary name.
+     * @return True if this name is the primary added name. Otherwise, False.
+     */
+    private boolean isPrimaryName(int refId, String name) {
+      if (idToNamesMap.get(refId).size() > 0) {
+        return getPrimaryName(refId).equals(name);
+      } else {
+        return false;
+      }
+    }
+
+    private String getPrimaryName(int refId) {
+      return idToNamesMap.get(refId).get(0);
+    }
+
+    public Target getTarget(String name) {
+      if (!nameToIdBiMap.containsKey(name)) {
+        throw new RuntimeException("No Such target name: " + name);
+      }
+      int id = nameToIdBiMap.get(name);
+      EvalNode evalNode = idToEvalBiMap.get(id);
+
+      // if it is a constant value, just returns a constant because it can be evaluated everywhere.
+      if (evalNode.getType() == EvalType.CONST) {
+        return new Target(evalNode, name);
+      }
+
+      // if a name is not the primary name, it means that its expression may be already evaluated and
+      // can just refer a value. Consider an example as follows:
+      //
+      // select l_orderkey + 1 as total1, l_orderkey + 1 as total2 from lineitem
+      //
+      // In this case, total2 will meet the following condition. Then, total2 can
+      // just refer the result of total1 rather than calculating l_orderkey + 1.
+      if (!isPrimaryName(id, name) && isEvaluated(getPrimaryName(id))) {
+        evalNode = new FieldEval(getPrimaryName(id), evalNode.getValueType());
+      }
+
+      // if it is a column reference itself, just returns a column reference without any alias.
+      if (evalNode.getType() == EvalType.FIELD && evalNode.getName().equals(name)) {
+        return new Target((FieldEval)evalNode);
+      } else { // otherwise, it returns an expression.
+        return new Target(evalNode, name);
+      }
+    }
+
+    public boolean isEvaluated(String name) {
+      if (!nameToIdBiMap.containsKey(name)) {
+        throw new RuntimeException("No Such target name: " + name);
+      }
+      return evaluationStateMap.get(name);
+    }
+
+    public void markAsEvaluated(Target target) {
+      int refId = nameToIdBiMap.get(target.getCanonicalName());
+      EvalNode evalNode = target.getEvalTree();
+      if (!idToNamesMap.containsKey(refId)) {
+        throw new RuntimeException("No such eval: " + evalNode);
+      }
+      evaluationStateMap.put(target.getCanonicalName(), true);
+    }
+
+    public Iterator<Target> getFilteredTargets(Set<String> required) {
+      return new FilteredTargetIterator(required);
+    }
+
+    class FilteredTargetIterator implements Iterator<Target> {
+      List<Target> filtered = TUtil.newList();
+
+      public FilteredTargetIterator(Set<String> required) {
+        for (String name : nameToIdBiMap.keySet()) {
+          if (required.contains(name)) {
+            filtered.add(getTarget(name));
+          }
+        }
+      }
+
+      @Override
+      public boolean hasNext() {
+        return false;
+      }
+
+      @Override
+      public Target next() {
+        return null;
+      }
+
+      @Override
+      public void remove() {
+      }
+    }
+
+    public String toString() {
+      int evaluated = 0;
+      for (Boolean flag: evaluationStateMap.values()) {
+        if (flag) {
+          evaluated++;
+        }
+      }
+      return "eval=" + evaluationStateMap.size() + ", evaluated=" + evaluated;
+    }
+  }
+
+  static class Context {
+    TargetListManager targetListMgr;
+    Set<String> requiredSet;
+
+    public Context(LogicalPlan plan) {
+      requiredSet = new LinkedHashSet<String>();
+      targetListMgr = new TargetListManager(plan);
+    }
+
+    public Context(LogicalPlan plan, Collection<String> requiredSet) {
+      this.requiredSet = new LinkedHashSet<String>(requiredSet);
+      targetListMgr = new TargetListManager(plan);
+    }
+
+    public Context(Context upperContext) {
+      this.requiredSet = new LinkedHashSet<String>(upperContext.requiredSet);
+      targetListMgr = upperContext.targetListMgr;
+    }
+
+    public String addExpr(Target target) throws PlanningException {
+      String reference = targetListMgr.add(target);
+      addNecessaryReferences(target.getEvalTree());
+      return reference;
+    }
+
+    public String addExpr(EvalNode evalNode) throws PlanningException {
+      String reference = targetListMgr.add(evalNode);
+      addNecessaryReferences(evalNode);
+      return reference;
+    }
+
+    private void addNecessaryReferences(EvalNode evalNode) {
+      for (Column column : EvalTreeUtil.findUniqueColumns(evalNode)) {
+        requiredSet.add(column.getQualifiedName());
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "required=" + requiredSet.size() + "," + targetListMgr.toString();
+    }
+  }
+
+  @Override
+  public LogicalNode visitRoot(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, LogicalRootNode node,
+                          Stack<LogicalNode> stack) throws PlanningException {
+    LogicalNode child = super.visitRoot(context, plan, block, node, stack);
+    node.setInSchema(child.getOutSchema());
+    node.setOutSchema(child.getOutSchema());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitProjection(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                     ProjectionNode node, Stack<LogicalNode> stack) throws PlanningException {
+    Context newContext = new Context(context);
+    Target [] targets = node.getTargets();
+    int targetNum = targets.length;
+    String [] referenceNames = new String[targetNum];
+    for (int i = 0; i < targetNum; i++) {
+      referenceNames[i] = newContext.addExpr(targets[i]);
+    }
+
+    LogicalNode child = super.visitProjection(newContext, plan, block, node, stack);
+
+    node.setInSchema(child.getOutSchema());
+
+    int evaluationCount = 0;
+    List<Target> finalTargets = TUtil.newList();
+    for (String referenceName : referenceNames) {
+      Target target = context.targetListMgr.getTarget(referenceName);
+
+      if (context.targetListMgr.isEvaluated(referenceName)) {
+        finalTargets.add(new Target(new FieldEval(target.getNamedColumn())));
+      } else if (LogicalPlanner.checkIfBeEvaluatedAtThis(target.getEvalTree(), node)) {
+        finalTargets.add(target);
+        context.targetListMgr.markAsEvaluated(target);
+        evaluationCount++;
+      }
+    }
+
+    node.setTargets(finalTargets.toArray(new Target[finalTargets.size()]));
+    LogicalPlanner.verifyProjectedFields(block, node);
+
+    // Removing ProjectionNode
+    // TODO - Consider INSERT and CTAS statement, and then remove the check of stack.empty.
+    if (evaluationCount == 0 && PlannerUtil.targetToSchema(finalTargets).equals(child.getOutSchema())) {
+      if (stack.empty()) {
+        // if it is topmost, set it as the root of this block.
+        block.setRoot(child);
+      } else {
+        LogicalNode parentNode = stack.peek();
+        switch (parentNode.getType()) {
+        case ROOT:
+          LogicalRootNode rootNode = (LogicalRootNode) parentNode;
+          rootNode.setChild(child);
+          rootNode.setInSchema(child.getOutSchema());
+          rootNode.setOutSchema(child.getOutSchema());
+          break;
+        case TABLE_SUBQUERY:
+          TableSubQueryNode tableSubQueryNode = (TableSubQueryNode) parentNode;
+          tableSubQueryNode.setSubQuery(child);
+          break;
+        case STORE:
+          StoreTableNode storeTableNode = (StoreTableNode) parentNode;
+          storeTableNode.setChild(child);
+          storeTableNode.setInSchema(child.getOutSchema());
+          break;
+        case INSERT:
+          InsertNode insertNode = (InsertNode) parentNode;
+          insertNode.setSubQuery(child);
+          break;
+        case CREATE_TABLE:
+          CreateTableNode createTableNode = (CreateTableNode) parentNode;
+          createTableNode.setChild(child);
+          createTableNode.setInSchema(child.getOutSchema());
+          break;
+        default:
+          throw new PlanningException("Unexpected Parent Node: " + parentNode.getType());
+        }
+        plan.addHistory("ProjectionNode is eliminated.");
+      }
+
+      return child;
+
+    } else {
+      return node;
+    }
+  }
+
+  public LogicalNode visitLimit(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, LimitNode node,
+                           Stack<LogicalNode> stack) throws PlanningException {
+    LogicalNode child = super.visitLimit(context, plan, block, node, stack);
+
+    node.setInSchema(child.getOutSchema());
+    node.setOutSchema(child.getOutSchema());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitSort(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                               SortNode node, Stack<LogicalNode> stack) throws PlanningException {
+    Context newContext = new Context(context);
+
+    final int sortKeyNum = node.getSortKeys().length;
+    String [] keyNames = new String[sortKeyNum];
+    for (int i = 0; i < sortKeyNum; i++) {
+      SortSpec sortSpec = node.getSortKeys()[i];
+      keyNames[i] = newContext.addExpr(new FieldEval(sortSpec.getSortKey()));
+    }
+
+    LogicalNode child = super.visitSort(newContext, plan, block, node, stack);
+
+    // it rewrite sortkeys. This rewrite sets right column names and eliminates duplicated sort keys.
+    List<SortSpec> sortSpecs = new ArrayList<SortSpec>();
+    for (int i = 0; i < keyNames.length; i++) {
+      String sortKey = keyNames[i];
+      Target target = context.targetListMgr.getTarget(sortKey);
+      if (context.targetListMgr.isEvaluated(sortKey)) {
+        Column c = target.getNamedColumn();
+        SortSpec sortSpec = new SortSpec(c, node.getSortKeys()[i].isAscending(), node.getSortKeys()[i].isNullFirst());
+        if (!sortSpecs.contains(sortSpec)) {
+          sortSpecs.add(sortSpec);
+        }
+      } else {
+        if (target.getEvalTree().getType() == EvalType.FIELD) {
+          Column c = ((FieldEval)target.getEvalTree()).getColumnRef();
+          SortSpec sortSpec = new SortSpec(c, node.getSortKeys()[i].isAscending(), node.getSortKeys()[i].isNullFirst());
+          if (!sortSpecs.contains(sortSpec)) {
+            sortSpecs.add(sortSpec);
+          }
+        }
+      }
+    }
+    node.setSortSpecs(sortSpecs.toArray(new SortSpec[sortSpecs.size()]));
+
+    node.setInSchema(child.getOutSchema());
+    node.setOutSchema(child.getOutSchema());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitHaving(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, HavingNode node,
+                            Stack<LogicalNode> stack) throws PlanningException {
+    Context newContext = new Context(context);
+    String referenceName = newContext.targetListMgr.add(node.getQual());
+    newContext.addNecessaryReferences(node.getQual());
+
+    LogicalNode child = super.visitHaving(newContext, plan, block, node, stack);
+
+    node.setInSchema(child.getOutSchema());
+    node.setOutSchema(child.getOutSchema());
+
+    Target target = context.targetListMgr.getTarget(referenceName);
+    if (newContext.targetListMgr.isEvaluated(referenceName)) {
+      node.setQual(new FieldEval(target.getNamedColumn()));
+    } else {
+      node.setQual(target.getEvalTree());
+      newContext.targetListMgr.markAsEvaluated(target);
+    }
+
+    return node;
+  }
+
+  public LogicalNode visitGroupBy(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, GroupbyNode node,
+                             Stack<LogicalNode> stack) throws PlanningException {
+    Context newContext = new Context(context);
+
+    // Getting grouping key names
+    final int groupingKeyNum = node.getGroupingColumns().length;
+    String [] groupingKeyNames = null;
+    if (groupingKeyNum > 0) {
+      groupingKeyNames = new String[groupingKeyNum];
+      for (int i = 0; i < groupingKeyNum; i++) {
+        FieldEval fieldEval = new FieldEval(node.getGroupingColumns()[i]);
+        groupingKeyNames[i] = newContext.addExpr(fieldEval);
+      }
+    }
+
+    // Getting eval names
+
+    final String [] aggEvalNames;
+    if (node.hasAggFunctions()) {
+      final int evalNum = node.getAggFunctions().length;
+      aggEvalNames = new String[evalNum];
+      for (int evalIdx = 0, targetIdx = groupingKeyNum; targetIdx < node.getTargets().length; evalIdx++, targetIdx++) {
+        Target target = node.getTargets()[targetIdx];
+        EvalNode evalNode = node.getAggFunctions()[evalIdx];
+        aggEvalNames[evalIdx] = newContext.addExpr(new Target(evalNode, target.getCanonicalName()));
+      }
+    } else {
+      aggEvalNames = null;
+    }
+
+    // visit a child node
+    LogicalNode child = super.visitGroupBy(newContext, plan, block, node, stack);
+
+    node.setInSchema(child.getOutSchema());
+
+    List<Target> targets = Lists.newArrayList();
+    if (groupingKeyNum > 0 && groupingKeyNames != null) {
+      // Restoring grouping key columns
+      final List<Column> groupingColumns = new ArrayList<Column>();
+      for (int i = 0; i < groupingKeyNum; i++) {
+        String groupingKey = groupingKeyNames[i];
+
+        Target target = context.targetListMgr.getTarget(groupingKey);
+
+        // it rewrite grouping keys.
+        // This rewrite sets right column names and eliminates duplicated grouping keys.
+        if (context.targetListMgr.isEvaluated(groupingKey)) {
+          Column c = target.getNamedColumn();
+          if (!groupingColumns.contains(c)) {
+            groupingColumns.add(c);
+            targets.add(new Target(new FieldEval(target.getNamedColumn())));
+          }
+        } else {
+          if (target.getEvalTree().getType() == EvalType.FIELD) {
+            Column c = ((FieldEval)target.getEvalTree()).getColumnRef();
+            if (!groupingColumns.contains(c)) {
+              groupingColumns.add(c);
+              targets.add(target);
+              context.targetListMgr.markAsEvaluated(target);
+            }
+          } else {
+            throw new PlanningException("Cannot evaluate this expression in grouping keys: " + target.getEvalTree());
+          }
+        }
+      }
+
+      node.setGroupingColumns(groupingColumns.toArray(new Column[groupingColumns.size()]));
+    }
+
+    // Getting projected targets
+    if (node.hasAggFunctions() && aggEvalNames != null) {
+      AggregationFunctionCallEval [] aggEvals = new AggregationFunctionCallEval[aggEvalNames.length];
+      int i = 0;
+      for (Iterator<String> it = getFilteredReferences(aggEvalNames, TUtil.newList(aggEvalNames)); it.hasNext();) {
+
+        String referenceName = it.next();
+        Target target = context.targetListMgr.getTarget(referenceName);
+
+        if (LogicalPlanner.checkIfBeEvaluatedAtGroupBy(target.getEvalTree(), node)) {
+          aggEvals[i++] = target.getEvalTree();
+          context.targetListMgr.markAsEvaluated(target);
+        }
+      }
+      if (aggEvals.length > 0) {
+        node.setAggFunctions(aggEvals);
+      }
+    }
+    Target [] finalTargets = buildGroupByTarget(node, targets, aggEvalNames);
+    node.setTargets(finalTargets);
+
+    LogicalPlanner.verifyProjectedFields(block, node);
+
+    return node;
+  }
+
+  public static Target [] buildGroupByTarget(GroupbyNode groupbyNode, @Nullable List<Target> groupingKeyTargets,
+                                             String [] aggEvalNames) {
+    final int groupingKeyNum =
+        groupingKeyTargets == null ? groupbyNode.getGroupingColumns().length : groupingKeyTargets.size();
+    final int aggrFuncNum = aggEvalNames != null ? aggEvalNames.length : 0;
+    EvalNode [] aggEvalNodes = groupbyNode.getAggFunctions();
+    Target [] targets = new Target[groupingKeyNum + aggrFuncNum];
+
+    if (groupingKeyTargets != null) {
+      for (int groupingKeyIdx = 0; groupingKeyIdx < groupingKeyNum; groupingKeyIdx++) {
+        targets[groupingKeyIdx] = groupingKeyTargets.get(groupingKeyIdx);
+      }
+    } else {
+      for (int groupingKeyIdx = 0; groupingKeyIdx < groupingKeyNum; groupingKeyIdx++) {
+        targets[groupingKeyIdx] = new Target(new FieldEval(groupbyNode.getGroupingColumns()[groupingKeyIdx]));
+      }
+    }
+
+    if (aggEvalNames != null) {
+      for (int aggrFuncIdx = 0, targetIdx = groupingKeyNum; aggrFuncIdx < aggrFuncNum; aggrFuncIdx++, targetIdx++) {
+        targets[targetIdx] =
+            new Target(new FieldEval(aggEvalNames[aggrFuncIdx], aggEvalNodes[aggrFuncIdx].getValueType()));
+      }
+    }
+
+    return targets;
+  }
+
+  public LogicalNode visitFilter(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 SelectionNode node, Stack<LogicalNode> stack) throws PlanningException {
+    Context newContext = new Context(context);
+    String referenceName = newContext.targetListMgr.add(node.getQual());
+    newContext.addNecessaryReferences(node.getQual());
+
+    LogicalNode child = super.visitFilter(newContext, plan, block, node, stack);
+
+    node.setInSchema(child.getOutSchema());
+    node.setOutSchema(child.getOutSchema());
+
+    Target target = context.targetListMgr.getTarget(referenceName);
+    if (newContext.targetListMgr.isEvaluated(referenceName)) {
+      node.setQual(new FieldEval(target.getNamedColumn()));
+    } else {
+      node.setQual(target.getEvalTree());
+      newContext.targetListMgr.markAsEvaluated(target);
+    }
+
+    return node;
+  }
+
+  public LogicalNode visitJoin(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode node,
+                          Stack<LogicalNode> stack) throws PlanningException {
+    Context newContext = new Context(context);
+
+    String joinQualReference = null;
+    if (node.hasJoinQual()) {
+      joinQualReference = newContext.addExpr(node.getJoinQual());
+      newContext.addNecessaryReferences(node.getJoinQual());
+    }
+
+    String [] referenceNames = null;
+    if (node.hasTargets()) {
+      referenceNames = new String[node.getTargets().length];
+      int i = 0;
+      for (Iterator<Target> it = getFilteredTarget(node.getTargets(), context.requiredSet); it.hasNext();) {
+        Target target = it.next();
+        referenceNames[i++] = newContext.addExpr(target);
+      }
+    }
+
+    stack.push(node);
+    LogicalNode left = visit(newContext, plan, block, node.getLeftChild(), stack);
+    LogicalNode right = visit(newContext, plan, block, node.getRightChild(), stack);
+    stack.pop();
+
+    Schema merged = SchemaUtil.merge(left.getOutSchema(), right.getOutSchema());
+
+    node.setInSchema(merged);
+
+    if (node.hasJoinQual()) {
+      Target target = context.targetListMgr.getTarget(joinQualReference);
+      if (newContext.targetListMgr.isEvaluated(joinQualReference)) {
+        throw new PlanningException("Join condition must be evaluated in the proper Join Node: " + joinQualReference);
+      } else {
+        node.setJoinQual(target.getEvalTree());
+        newContext.targetListMgr.markAsEvaluated(target);
+      }
+    }
+
+    LinkedHashSet<Target> projectedTargets = Sets.newLinkedHashSet();
+    for (Iterator<String> it = getFilteredReferences(context.targetListMgr.getNames(),
+        context.requiredSet); it.hasNext();) {
+      String referenceName = it.next();
+      Target target = context.targetListMgr.getTarget(referenceName);
+
+      if (context.targetListMgr.isEvaluated(referenceName)) {
+        Target fieldReference = new Target(new FieldEval(target.getNamedColumn()));
+        if (LogicalPlanner.checkIfBeEvaluatedAtJoin(block, fieldReference.getEvalTree(), node,
+            stack.peek().getType() != NodeType.JOIN)) {
+          projectedTargets.add(fieldReference);
+        }
+      } else if (LogicalPlanner.checkIfBeEvaluatedAtJoin(block, target.getEvalTree(), node,
+          stack.peek().getType() != NodeType.JOIN)) {
+        projectedTargets.add(target);
+        context.targetListMgr.markAsEvaluated(target);
+      }
+    }
+
+    node.setTargets(projectedTargets.toArray(new Target[projectedTargets.size()]));
+    LogicalPlanner.verifyProjectedFields(block, node);
+    return node;
+  }
+
+  static Iterator<String> getFilteredReferences(Collection<String> targetNames, Set<String> required) {
+    return new FilteredStringsIterator(targetNames, required);
+  }
+
+  static Iterator<String> getFilteredReferences(String [] targetNames, Collection<String> required) {
+    return new FilteredStringsIterator(targetNames, required);
+  }
+
+  static class FilteredStringsIterator implements Iterator<String> {
+    Iterator<String> iterator;
+
+    FilteredStringsIterator(Collection<String> targetNames, Collection<String> required) {
+      List<String> filtered = TUtil.newList();
+      for (String name : targetNames) {
+        if (required.contains(name)) {
+          filtered.add(name);
+        }
+      }
+
+      iterator = filtered.iterator();
+    }
+
+    FilteredStringsIterator(String [] targetNames, Collection<String> required) {
+      this(TUtil.newList(targetNames), required);
+    }
+
+    @Override
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    @Override
+    public String next() {
+      return iterator.next();
+    }
+
+    @Override
+    public void remove() {
+    }
+  }
+
+  static Iterator<Target> getFilteredTarget(Target[] targets, Set<String> required) {
+    return new FilteredIterator(targets, required);
+  }
+
+  static class FilteredIterator implements Iterator<Target> {
+    Iterator<Target> iterator;
+
+    FilteredIterator(Target [] targets, Set<String> requiredReferences) {
+      List<Target> filtered = TUtil.newList();
+      Map<String, Target> targetSet = new HashMap<String, Target>();
+      for (Target t : targets) {
+        // Only should keep an raw target instead of field reference.
+        if (targetSet.containsKey(t.getCanonicalName())) {
+          Target targetInSet = targetSet.get(t.getCanonicalName());
+          EvalNode evalNode = targetInSet.getEvalTree();
+          if (evalNode.getType() == EvalType.FIELD && t.getEvalTree().getType() != EvalType.FIELD) {
+            targetSet.put(t.getCanonicalName(), t);
+          }
+        } else {
+          targetSet.put(t.getCanonicalName(), t);
+        }
+      }
+
+      for (String name : requiredReferences) {
+        if (targetSet.containsKey(name)) {
+          filtered.add(targetSet.get(name));
+        }
+      }
+
+      iterator = filtered.iterator();
+    }
+    @Override
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    @Override
+    public Target next() {
+      return iterator.next();
+    }
+
+    @Override
+    public void remove() {
+    }
+  }
+
+  @Override
+  public LogicalNode visitUnion(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, UnionNode node,
+                           Stack<LogicalNode> stack) throws PlanningException {
+
+    LogicalPlan.QueryBlock leftBlock = plan.getBlock(node.getLeftChild());
+    LogicalPlan.QueryBlock rightBlock = plan.getBlock(node.getRightChild());
+
+    Context leftContext = new Context(plan, PlannerUtil.toQualifiedFieldNames(context.requiredSet,
+        leftBlock.getName()));
+    Context rightContext = new Context(plan, PlannerUtil.toQualifiedFieldNames(context.requiredSet,
+        rightBlock.getName()));
+
+    stack.push(node);
+    visit(leftContext, plan, leftBlock, leftBlock.getRoot(), new Stack<LogicalNode>());
+    visit(rightContext, plan, rightBlock, rightBlock.getRoot(), new Stack<LogicalNode>());
+    stack.pop();
+    return node;
+  }
+
+  public LogicalNode visitScan(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode node,
+                          Stack<LogicalNode> stack) throws PlanningException {
+
+    Context newContext = new Context(context);
+
+    Target [] targets;
+    if (node.hasTargets()) {
+      targets = node.getTargets();
+    } else {
+      targets = PlannerUtil.schemaToTargets(node.getTableSchema());
+    }
+
+    LinkedHashSet<Target> projectedTargets = Sets.newLinkedHashSet();
+    for (Iterator<Target> it = getFilteredTarget(targets, newContext.requiredSet); it.hasNext();) {
+      Target target = it.next();
+      newContext.addExpr(target);
+    }
+
+    for (Iterator<Target> it = getFilteredTarget(targets, context.requiredSet); it.hasNext();) {
+      Target target = it.next();
+
+      if (LogicalPlanner.checkIfBeEvaluatedAtRelation(block, target.getEvalTree(), node)) {
+        projectedTargets.add(target);
+        newContext.targetListMgr.markAsEvaluated(target);
+      }
+    }
+
+    node.setTargets(projectedTargets.toArray(new Target[projectedTargets.size()]));
+    LogicalPlanner.verifyProjectedFields(block, node);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitPartitionedTableScan(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                               PartitionedTableScanNode node, Stack<LogicalNode> stack)
+      throws PlanningException {
+
+    Context newContext = new Context(context);
+
+    Target [] targets;
+    if (node.hasTargets()) {
+      targets = node.getTargets();
+    } else {
+      targets = PlannerUtil.schemaToTargets(node.getOutSchema());
+    }
+
+    LinkedHashSet<Target> projectedTargets = Sets.newLinkedHashSet();
+    for (Iterator<Target> it = getFilteredTarget(targets, newContext.requiredSet); it.hasNext();) {
+      Target target = it.next();
+      newContext.addExpr(target);
+    }
+
+    for (Iterator<Target> it = getFilteredTarget(targets, context.requiredSet); it.hasNext();) {
+      Target target = it.next();
+
+      if (LogicalPlanner.checkIfBeEvaluatedAtRelation(block, target.getEvalTree(), node)) {
+        projectedTargets.add(target);
+        newContext.targetListMgr.markAsEvaluated(target);
+      }
+    }
+
+    node.setTargets(projectedTargets.toArray(new Target[projectedTargets.size()]));
+    LogicalPlanner.verifyProjectedFields(block, node);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitTableSubQuery(Context upperContext, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                   TableSubQueryNode node, Stack<LogicalNode> stack) throws PlanningException {
+    Context childContext = new Context(plan, upperContext.requiredSet);
+    stack.push(node);
+    LogicalNode child = super.visitTableSubQuery(childContext, plan, block, node, stack);
+    node.setSubQuery(child);
+    stack.pop();
+
+    Context newContext = new Context(upperContext);
+
+    Target [] targets;
+    if (node.hasTargets()) {
+      targets = node.getTargets();
+    } else {
+      targets = PlannerUtil.schemaToTargets(node.getOutSchema());
+    }
+
+    LinkedHashSet<Target> projectedTargets = Sets.newLinkedHashSet();
+    for (Iterator<Target> it = getFilteredTarget(targets, newContext.requiredSet); it.hasNext();) {
+      Target target = it.next();
+      childContext.addExpr(target);
+    }
+
+    for (Iterator<Target> it = getFilteredTarget(targets, upperContext.requiredSet); it.hasNext();) {
+      Target target = it.next();
+
+      if (LogicalPlanner.checkIfBeEvaluatedAtRelation(block, target.getEvalTree(), node)) {
+        projectedTargets.add(target);
+        childContext.targetListMgr.markAsEvaluated(target);
+      }
+    }
+
+    node.setTargets(projectedTargets.toArray(new Target[projectedTargets.size()]));
+    LogicalPlanner.verifyProjectedFields(block, node);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitInsert(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, InsertNode node,
+                            Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return node;
+  }
+}


[49/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4 b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4
new file mode 100644
index 0000000..6a85695
--- /dev/null
+++ b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4
@@ -0,0 +1,2067 @@
+/**
+   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.
+*/
+parser grammar HiveQLParser;
+
+options
+{
+tokenVocab=HiveQLLexer;
+language=Java;
+}
+
+
+// Package headers
+@header {
+import java.util.*;
+}
+
+
+@members {
+  Stack msgs = new Stack<String>();
+}
+
+@rulecatch {
+catch (RecognitionException e) {
+ reportError(e);
+  throw e;
+}
+}
+
+//----------------------- Rules for parsing selectClause -----------------------------
+// select a,b,c ...
+selectClause
+@init { msgs.push("select clause"); }
+@after { msgs.pop(); }
+    :
+    KW_SELECT hintClause? (((KW_ALL | dist=KW_DISTINCT)? selectList)
+                          | (transform=KW_TRANSFORM selectTrfmClause))
+    |
+    trfmClause
+    ;
+
+selectList
+@init { msgs.push("select list"); }
+@after { msgs.pop(); }
+    :
+    selectItem ( COMMA  selectItem )* 
+    ;
+
+selectTrfmClause
+@init { msgs.push("transform clause"); }
+@after { msgs.pop(); }
+    :
+    LPAREN selectExpressionList RPAREN
+    inSerde=rowFormat inRec=recordWriter
+    KW_USING StringLiteral
+    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
+    outSerde=rowFormat outRec=recordReader
+    ;
+
+hintClause
+@init { msgs.push("hint clause"); }
+@after { msgs.pop(); }
+    :
+    DIVIDE STAR PLUS hintList STAR DIVIDE 
+    ;
+
+hintList
+@init { msgs.push("hint list"); }
+@after { msgs.pop(); }
+    :
+    hintItem (COMMA hintItem)* 
+    ;
+
+hintItem
+@init { msgs.push("hint item"); }
+@after { msgs.pop(); }
+    :
+    hintName (LPAREN hintArgs RPAREN)? 
+    ;
+
+hintName
+@init { msgs.push("hint name"); }
+@after { msgs.pop(); }
+    :
+    KW_MAPJOIN 
+    | KW_STREAMTABLE 
+    | KW_HOLD_DDLTIME 
+    ;
+
+hintArgs
+@init { msgs.push("hint arguments"); }
+@after { msgs.pop(); }
+    :
+    hintArgName (COMMA hintArgName)* 
+    ;
+
+hintArgName
+@init { msgs.push("hint argument name"); }
+@after { msgs.pop(); }
+    :
+    identifier
+    ;
+
+selectItem
+@init { msgs.push("selection target"); }
+@after { msgs.pop(); }
+    :
+    ( selectExpression (KW_OVER ws=window_specification )?
+      ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))?
+    ) 
+    ;
+
+trfmClause
+@init { msgs.push("transform clause"); }
+@after { msgs.pop(); }
+    :
+    (   KW_MAP    selectExpressionList
+      | KW_REDUCE selectExpressionList )
+    inSerde=rowFormat inRec=recordWriter
+    KW_USING StringLiteral
+    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
+    outSerde=rowFormat outRec=recordReader
+    ;
+
+selectExpression
+@init { msgs.push("select expression"); }
+@after { msgs.pop(); }
+    :
+    expression | tableAllColumns
+    ;
+
+selectExpressionList
+@init { msgs.push("select expression list"); }
+@after { msgs.pop(); }
+    :
+    selectExpression (COMMA selectExpression)* 
+    ;
+
+
+//---------------------- Rules for windowing clauses -------------------------------
+window_clause 
+@init { msgs.push("window_clause"); }
+@after { msgs.pop(); } 
+:
+  KW_WINDOW window_defn (COMMA window_defn)* 
+;  
+
+window_defn 
+@init { msgs.push("window_defn"); }
+@after { msgs.pop(); } 
+:
+  Identifier KW_AS window_specification 
+;  
+
+window_specification 
+@init { msgs.push("window_specification"); }
+@after { msgs.pop(); } 
+:
+  (Identifier | ( LPAREN Identifier? partitioningSpec? window_frame? RPAREN)) 
+;
+
+window_frame :
+ window_range_expression |
+ window_value_expression
+;
+
+window_range_expression 
+@init { msgs.push("window_range_expression"); }
+@after { msgs.pop(); } 
+:
+ KW_ROWS sb=window_frame_start_boundary 
+ KW_ROWS KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary 
+;
+
+window_value_expression 
+@init { msgs.push("window_value_expression"); }
+@after { msgs.pop(); } 
+:
+ KW_RANGE sb=window_frame_start_boundary 
+ KW_RANGE KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary 
+;
+
+window_frame_start_boundary 
+@init { msgs.push("windowframestartboundary"); }
+@after { msgs.pop(); } 
+:
+  KW_UNBOUNDED KW_PRECEDING  
+  KW_CURRENT KW_ROW  
+  Number KW_PRECEDING 
+;
+
+window_frame_boundary 
+@init { msgs.push("windowframeboundary"); }
+@after { msgs.pop(); } 
+:
+  KW_UNBOUNDED (r=KW_PRECEDING|r=KW_FOLLOWING)  
+  KW_CURRENT KW_ROW  
+  Number (d=KW_PRECEDING | d=KW_FOLLOWING ) 
+;   
+
+
+tableAllColumns
+    : STAR
+    | tableName DOT STAR
+    ;
+
+// (table|column)
+tableOrColumn
+@init { msgs.push("table or column identifier"); }
+@after { msgs.pop(); }
+    :
+    identifier 
+    ;
+
+expressionList
+@init { msgs.push("expression list"); }
+@after { msgs.pop(); }
+    :
+    expression (COMMA expression)* 
+    ;
+
+aliasList
+@init { msgs.push("alias list"); }
+@after { msgs.pop(); }
+    :
+    identifier (COMMA identifier)* 
+    ;
+
+
+//----------------------- Rules for parsing fromClause ------------------------------
+// from [col1, col2, col3] table1, [col4, col5] table2
+fromClause
+@init { msgs.push("from clause"); }
+@after { msgs.pop(); }
+    :
+    KW_FROM joinSource 
+    ;
+
+joinSource
+@init { msgs.push("join source"); }
+@after { msgs.pop(); }
+    : fromSource ( joinToken fromSource (KW_ON expression)? 
+    )*
+    | uniqueJoinToken uniqueJoinSource (COMMA uniqueJoinSource)+
+    ;
+
+uniqueJoinSource
+@init { msgs.push("join source"); }
+@after { msgs.pop(); }
+    : KW_PRESERVE? fromSource uniqueJoinExpr
+    ;
+
+uniqueJoinExpr
+@init { msgs.push("unique join expression list"); }
+@after { msgs.pop(); }
+    : LPAREN e1+=expression (COMMA e1+=expression)* RPAREN
+    ;
+
+uniqueJoinToken
+@init { msgs.push("unique join"); }
+@after { msgs.pop(); }
+    : KW_UNIQUEJOIN 
+;
+
+joinToken
+@init { msgs.push("join type specifier"); }
+@after { msgs.pop(); }
+    :
+      KW_JOIN                    
+    | KW_INNER  KW_JOIN            
+    | KW_CROSS KW_JOIN            
+    | KW_LEFT  KW_OUTER KW_JOIN   
+    | KW_RIGHT KW_OUTER KW_JOIN  
+    | KW_FULL  KW_OUTER KW_JOIN  
+    | KW_LEFT  KW_SEMI  KW_JOIN  
+    ;
+
+lateralView
+@init {msgs.push("lateral view"); }
+@after {msgs.pop(); }
+	:
+	KW_LATERAL KW_VIEW function tableAlias KW_AS identifier (COMMA identifier)* 
+	;
+
+tableAlias
+@init {msgs.push("table alias"); }
+@after {msgs.pop(); }
+    :
+    identifier 
+    ;
+
+fromSource
+@init { msgs.push("from source"); }
+@after { msgs.pop(); }
+    :
+    ((Identifier LPAREN) | tableSource | subQuerySource) (lateralView)*
+    ;
+
+tableBucketSample
+@init { msgs.push("table bucket sample specification"); }
+@after { msgs.pop(); }
+    :
+    KW_TABLESAMPLE LPAREN KW_BUCKET (numerator=Number) KW_OUT KW_OF (denominator=Number) (KW_ON expr+=expression (COMMA expr+=expression)*)? RPAREN 
+    ;
+
+splitSample
+@init { msgs.push("table split sample specification"); }
+@after { msgs.pop(); }
+    :
+    KW_TABLESAMPLE LPAREN  (numerator=Number) (percent=KW_PERCENT|KW_ROWS) RPAREN
+    |
+    KW_TABLESAMPLE LPAREN  (numerator=ByteLengthLiteral) RPAREN
+    ;
+
+tableSample
+@init { msgs.push("table sample specification"); }
+@after { msgs.pop(); }
+    :
+    tableBucketSample |
+    splitSample
+    ;
+
+tableSource
+@init { msgs.push("table source"); }
+@after { msgs.pop(); }
+    : tabname=tableName (ts=tableSample)? (alias=identifier)?
+    ;
+
+tableName
+@init { msgs.push("table name"); }
+@after { msgs.pop(); }
+    :
+    db=identifier DOT tab=identifier
+    |
+    tab=identifier
+    ;
+
+viewName
+@init { msgs.push("view name"); }
+@after { msgs.pop(); }
+    :
+    (db=identifier DOT)? view=identifier
+    ;
+
+subQuerySource
+@init { msgs.push("subquery source"); }
+@after { msgs.pop(); }
+    :
+    LPAREN queryStatementExpression RPAREN identifier 
+    ;
+
+//---------------------- Rules for parsing PTF clauses -----------------------------
+partitioningSpec
+@init { msgs.push("partitioningSpec clause"); }
+@after { msgs.pop(); } 
+   :
+   partitionByClause orderByClause? 
+   orderByClause 
+   distributeByClause sortByClause? 
+   sortByClause 
+   clusterByClause 
+   ;
+
+partitionTableFunctionSource
+@init { msgs.push("partitionTableFunctionSource clause"); }
+@after { msgs.pop(); } 
+   :
+   subQuerySource |
+   tableSource |
+   partitionedTableFunction
+   ;
+
+partitionedTableFunction
+@init { msgs.push("ptf clause"); }
+@after { msgs.pop(); } 
+   :
+   name=Identifier
+   LPAREN KW_ON ptfsrc=partitionTableFunctionSource partitioningSpec?
+     ((Identifier LPAREN expression RPAREN ) )? 
+   RPAREN alias=Identifier? 
+   ;
+
+//----------------------- Rules for parsing whereClause -----------------------------
+// where a=b and ...
+whereClause
+@init { msgs.push("where clause"); }
+@after { msgs.pop(); }
+    :
+    KW_WHERE searchCondition 
+    ;
+
+searchCondition
+@init { msgs.push("search condition"); }
+@after { msgs.pop(); }
+    :
+    expression
+    ;
+
+//-----------------------------------------------------------------------------------
+
+
+// group by a,b
+groupByClause
+@init { msgs.push("group by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_GROUP KW_BY
+    groupByExpression
+    ( COMMA groupByExpression )*
+    ((rollup=KW_WITH KW_ROLLUP) | (cube=KW_WITH KW_CUBE)) ?
+    (sets=KW_GROUPING KW_SETS 
+    LPAREN groupingSetExpression ( COMMA groupingSetExpression)*  RPAREN ) ?
+    ;
+
+groupingSetExpression
+@init {msgs.push("grouping set expression"); }
+@after {msgs.pop(); }
+   :
+   groupByExpression
+   |
+   LPAREN 
+   groupByExpression (COMMA groupByExpression)*
+   RPAREN
+   |
+   LPAREN
+   RPAREN
+   ;
+
+
+groupByExpression
+@init { msgs.push("group by expression"); }
+@after { msgs.pop(); }
+    :
+    expression
+    ;
+
+havingClause
+@init { msgs.push("having clause"); }
+@after { msgs.pop(); }
+    :
+    KW_HAVING havingCondition 
+    ;
+
+havingCondition
+@init { msgs.push("having condition"); }
+@after { msgs.pop(); }
+    :
+    expression
+    ;
+
+// order by a,b
+orderByClause
+@init { msgs.push("order by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_ORDER KW_BY
+    LPAREN columnRefOrder
+    ( COMMA columnRefOrder)* RPAREN 
+    |
+    KW_ORDER KW_BY
+    columnRefOrder
+    ( COMMA columnRefOrder)* 
+    ;
+
+clusterByClause
+@init { msgs.push("cluster by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_CLUSTER KW_BY
+    LPAREN expression (COMMA expression)* RPAREN
+    |
+    KW_CLUSTER KW_BY
+    expression
+    ((COMMA))*
+    ;
+
+partitionByClause
+@init  { msgs.push("partition by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_PARTITION KW_BY
+    LPAREN expression (COMMA expression)* RPAREN
+    |
+    KW_PARTITION KW_BY
+    expression ((COMMA))*
+    ;
+
+distributeByClause
+@init { msgs.push("distribute by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_DISTRIBUTE KW_BY
+    LPAREN expression (COMMA expression)* RPAREN
+    |
+    KW_DISTRIBUTE KW_BY
+    expression ((COMMA))*
+    ;
+
+sortByClause
+@init { msgs.push("sort by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_SORT KW_BY
+    LPAREN columnRefOrder
+    ( COMMA columnRefOrder)* RPAREN 
+    |
+    KW_SORT KW_BY
+    columnRefOrder
+    ( (COMMA))*
+    ;
+
+// fun(par1, par2, par3)
+function
+@init { msgs.push("function specification"); }
+@after { msgs.pop(); }
+    :
+    functionName
+    LPAREN
+      (
+        (star=STAR)
+        | (dist=KW_DISTINCT)? (selectExpression (COMMA selectExpression)*)?
+      )
+    RPAREN 
+    ;
+
+functionName
+@init { msgs.push("function name"); }
+@after { msgs.pop(); }
+    : // Keyword IF is also a function name
+    KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE | identifier
+    ;
+
+castExpression
+@init { msgs.push("cast expression"); }
+@after { msgs.pop(); }
+    :
+    KW_CAST
+    LPAREN
+          expression
+          KW_AS
+          primitiveType
+    RPAREN 
+    ;
+
+caseExpression
+@init { msgs.push("case expression"); }
+@after { msgs.pop(); }
+    :
+    KW_CASE expression
+    (KW_WHEN expression KW_THEN expression)+
+    (KW_ELSE expression)?
+    KW_END 
+    ;
+
+whenExpression
+@init { msgs.push("case expression"); }
+@after { msgs.pop(); }
+    :
+    KW_CASE
+     ( KW_WHEN expression KW_THEN expression)+
+    (KW_ELSE expression)?
+    KW_END 
+    ;
+
+constant
+@init { msgs.push("constant"); }
+@after { msgs.pop(); }
+    :
+    Number
+    | StringLiteral
+    | stringLiteralSequence
+    | BigintLiteral
+    | SmallintLiteral
+    | TinyintLiteral
+    | DecimalLiteral
+    | charSetStringLiteral
+    | booleanValue
+    ;
+
+stringLiteralSequence
+    :
+    StringLiteral StringLiteral+ 
+    ;
+
+charSetStringLiteral
+@init { msgs.push("character string literal"); }
+@after { msgs.pop(); }
+    :
+    csName=CharSetName csLiteral=CharSetLiteral 
+    ;
+
+expression
+@init { msgs.push("expression specification"); }
+@after { msgs.pop(); }
+    :
+    precedenceOrExpression
+    ;
+
+atomExpression
+    :
+    KW_NULL 
+    | constant
+    | function
+    | castExpression
+    | caseExpression
+    | whenExpression
+    | tableOrColumn
+    | LPAREN expression RPAREN
+    ;
+
+
+precedenceFieldExpression
+    :
+    atomExpression ((LSQUARE expression RSQUARE) | (DOT identifier))*
+    ;
+
+precedenceUnaryOperator
+    :
+    PLUS | MINUS | TILDE
+    ;
+
+nullCondition
+    :
+    KW_NULL     
+    | KW_NOT KW_NULL     
+    ;
+
+precedenceUnaryPrefixExpression
+    :
+    (precedenceUnaryOperator)* precedenceFieldExpression
+    ;
+
+precedenceUnarySuffixExpression
+    : precedenceUnaryPrefixExpression (a=KW_IS nullCondition)?
+    ;
+
+
+precedenceBitwiseXorOperator
+    :
+    BITWISEXOR
+    ;
+
+precedenceBitwiseXorExpression
+    :
+    precedenceUnarySuffixExpression (precedenceBitwiseXorOperator precedenceUnarySuffixExpression)*
+    ;
+
+
+precedenceStarOperator
+    :
+    STAR | DIVIDE | MOD | DIV
+    ;
+
+precedenceStarExpression
+    :
+    precedenceBitwiseXorExpression (precedenceStarOperator precedenceBitwiseXorExpression)*
+    ;
+
+
+precedencePlusOperator
+    :
+    PLUS | MINUS
+    ;
+
+precedencePlusExpression
+    :
+    precedenceStarExpression (precedencePlusOperator precedenceStarExpression)*
+    ;
+
+
+precedenceAmpersandOperator
+    :
+    AMPERSAND
+    ;
+
+precedenceAmpersandExpression
+    :
+    precedencePlusExpression (precedenceAmpersandOperator precedencePlusExpression)*
+    ;
+
+
+precedenceBitwiseOrOperator
+    :
+    BITWISEOR
+    ;
+
+precedenceBitwiseOrExpression
+    :
+    precedenceAmpersandExpression (precedenceBitwiseOrOperator precedenceAmpersandExpression)*
+    ;
+
+
+// Equal operators supporting NOT prefix
+precedenceEqualNegatableOperator
+    :
+    KW_LIKE | KW_RLIKE | KW_REGEXP
+    ;
+
+precedenceEqualOperator
+    :
+    precedenceEqualNegatableOperator | EQUAL | EQUAL_NS | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN
+    ;
+
+precedenceEqualExpression
+    :
+    (left=precedenceBitwiseOrExpression     
+    )
+    (
+       (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression) 
+    | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression)
+    | (KW_NOT KW_IN expressions) 
+    | (KW_IN expressions) 
+    | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) ) 
+    | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
+    )*
+    ;
+
+expressions
+    :
+    LPAREN expression (COMMA expression)* RPAREN 
+    ;
+
+precedenceNotOperator
+    :
+    KW_NOT
+    ;
+
+precedenceNotExpression
+    :
+    (precedenceNotOperator)* precedenceEqualExpression
+    ;
+
+
+precedenceAndOperator
+    :
+    KW_AND
+    ;
+
+precedenceAndExpression
+    :
+    precedenceNotExpression (precedenceAndOperator precedenceNotExpression)*
+    ;
+
+
+precedenceOrOperator
+    :
+    KW_OR
+    ;
+
+precedenceOrExpression
+    :
+    precedenceAndExpression (precedenceOrOperator precedenceAndExpression)*
+    ;
+
+
+booleanValue
+    :
+    KW_TRUE | KW_FALSE
+    ;
+
+tableOrPartition
+   :
+   tableName partitionSpec? 
+   ;
+
+partitionSpec
+    :
+    KW_PARTITION
+     LPAREN partitionVal (COMMA  partitionVal )* RPAREN 
+    ;
+
+partitionVal
+    :
+    identifier (EQUAL constant)? 
+    ;
+
+dropPartitionSpec
+    :
+    KW_PARTITION
+     LPAREN dropPartitionVal (COMMA  dropPartitionVal )* RPAREN 
+    ;
+
+dropPartitionVal
+    :
+    identifier dropPartitionOperator constant 
+    ;
+
+dropPartitionOperator
+    :
+    EQUAL | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN
+    ;
+
+sysFuncNames
+    :
+      KW_AND
+    | KW_OR
+    | KW_NOT
+    | KW_LIKE
+    | KW_IF
+    | KW_CASE
+    | KW_WHEN
+    | KW_TINYINT
+    | KW_SMALLINT
+    | KW_INT
+    | KW_BIGINT
+    | KW_FLOAT
+    | KW_DOUBLE
+    | KW_BOOLEAN
+    | KW_STRING
+    | KW_BINARY
+    | KW_ARRAY
+    | KW_MAP
+    | KW_STRUCT
+    | KW_UNIONTYPE
+    | EQUAL
+    | EQUAL_NS
+    | NOTEQUAL
+    | LESSTHANOREQUALTO
+    | LESSTHAN
+    | GREATERTHANOREQUALTO
+    | GREATERTHAN
+    | DIVIDE
+    | PLUS
+    | MINUS
+    | STAR
+    | MOD
+    | DIV
+    | AMPERSAND
+    | TILDE
+    | BITWISEOR
+    | BITWISEXOR
+    | KW_RLIKE
+    | KW_REGEXP
+    | KW_IN
+    | KW_BETWEEN
+    ;
+
+descFuncNames
+    :
+      sysFuncNames
+    | StringLiteral
+    | identifier
+    ;
+
+identifier
+    :
+    Identifier
+    | nonReserved 
+    ;
+    
+nonReserved
+    :
+    KW_TRUE | KW_FALSE | KW_LIKE | KW_EXISTS | KW_ASC | KW_DESC | KW_ORDER | KW_GROUP | KW_BY | KW_AS | KW_INSERT | KW_OVERWRITE | KW_OUTER | KW_LEFT | KW_RIGHT | KW_FULL | KW_PARTITION | KW_PARTITIONS | KW_TABLE | KW_TABLES | KW_COLUMNS | KW_INDEX | KW_INDEXES | KW_REBUILD | KW_FUNCTIONS | KW_SHOW | KW_MSCK | KW_REPAIR | KW_DIRECTORY | KW_LOCAL | KW_USING | KW_CLUSTER | KW_DISTRIBUTE | KW_SORT | KW_UNION | KW_LOAD | KW_EXPORT | KW_IMPORT | KW_DATA | KW_INPATH | KW_IS | KW_NULL | KW_CREATE | KW_EXTERNAL | KW_ALTER | KW_CHANGE | KW_FIRST | KW_AFTER | KW_DESCRIBE | KW_DROP | KW_RENAME | KW_IGNORE | KW_PROTECTION | KW_TO | KW_COMMENT | KW_BOOLEAN | KW_TINYINT | KW_SMALLINT | KW_INT | KW_BIGINT | KW_FLOAT | KW_DOUBLE | KW_DATE | KW_DATETIME | KW_TIMESTAMP | KW_DECIMAL | KW_STRING | KW_ARRAY | KW_STRUCT | KW_UNIONTYPE | KW_PARTITIONED | KW_CLUSTERED | KW_SORTED | KW_INTO | KW_BUCKETS | KW_ROW | KW_ROWS | KW_FORMAT | KW_DELIMITED | KW_FIELDS | KW_TERMINATED | KW_ESCAPED | KW_COLLECTION | 
 KW_ITEMS | KW_KEYS | KW_KEY_TYPE | KW_LINES | KW_STORED | KW_FILEFORMAT | KW_SEQUENCEFILE | KW_TEXTFILE | KW_RCFILE | KW_ORCFILE | KW_INPUTFORMAT | KW_OUTPUTFORMAT | KW_INPUTDRIVER | KW_OUTPUTDRIVER | KW_OFFLINE | KW_ENABLE | KW_DISABLE | KW_READONLY | KW_NO_DROP | KW_LOCATION | KW_BUCKET | KW_OUT | KW_OF | KW_PERCENT | KW_ADD | KW_REPLACE | KW_RLIKE | KW_REGEXP | KW_TEMPORARY | KW_EXPLAIN | KW_FORMATTED | KW_PRETTY | KW_DEPENDENCY | KW_SERDE | KW_WITH | KW_DEFERRED | KW_SERDEPROPERTIES | KW_DBPROPERTIES | KW_LIMIT | KW_SET | KW_UNSET | KW_TBLPROPERTIES | KW_IDXPROPERTIES | KW_VALUE_TYPE | KW_ELEM_TYPE | KW_MAPJOIN | KW_STREAMTABLE | KW_HOLD_DDLTIME | KW_CLUSTERSTATUS | KW_UTC | KW_UTCTIMESTAMP | KW_LONG | KW_DELETE | KW_PLUS | KW_MINUS | KW_FETCH | KW_INTERSECT | KW_VIEW | KW_IN | KW_DATABASES | KW_MATERIALIZED | KW_SCHEMA | KW_SCHEMAS | KW_GRANT | KW_REVOKE | KW_SSL | KW_UNDO | KW_LOCK | KW_LOCKS | KW_UNLOCK | KW_SHARED | KW_EXCLUSIVE | KW_PROCEDURE | KW_UNSIGNED | KW_WHILE | KW_R
 EAD | KW_READS | KW_PURGE | KW_RANGE | KW_ANALYZE | KW_BEFORE | KW_BETWEEN | KW_BOTH | KW_BINARY | KW_CONTINUE | KW_CURSOR | KW_TRIGGER | KW_RECORDREADER | KW_RECORDWRITER | KW_SEMI | KW_LATERAL | KW_TOUCH | KW_ARCHIVE | KW_UNARCHIVE | KW_COMPUTE | KW_STATISTICS | KW_USE | KW_OPTION | KW_CONCATENATE | KW_SHOW_DATABASE | KW_UPDATE | KW_RESTRICT | KW_CASCADE | KW_SKEWED | KW_ROLLUP | KW_CUBE | KW_DIRECTORIES | KW_FOR | KW_GROUPING | KW_SETS | KW_TRUNCATE | KW_NOSCAN | KW_USER | KW_ROLE | KW_INNER
+    ;
+
+//-----------------------------------------------------------------------------------
+
+// starting rule
+statement
+	: explainStatement EOF
+	| execStatement EOF
+	;
+
+explainStatement
+@init { msgs.push("explain statement"); }
+@after { msgs.pop(); }
+	: KW_EXPLAIN (explainOptions=KW_EXTENDED|explainOptions=KW_FORMATTED|explainOptions=KW_DEPENDENCY)? execStatement
+	;
+
+execStatement
+@init { msgs.push("statement"); }
+@after { msgs.pop(); }
+    : queryStatementExpression
+    | loadStatement
+    | exportStatement
+    | importStatement
+    | ddlStatement
+    ;
+
+loadStatement
+@init { msgs.push("load statement"); }
+@after { msgs.pop(); }
+    : KW_LOAD KW_DATA (islocal=KW_LOCAL)? KW_INPATH (path=StringLiteral) (isoverwrite=KW_OVERWRITE)? KW_INTO KW_TABLE (tab=tableOrPartition)
+    ;
+
+exportStatement
+@init { msgs.push("export statement"); }
+@after { msgs.pop(); }
+    : KW_EXPORT KW_TABLE (tab=tableOrPartition) KW_TO (path=StringLiteral)
+    ;
+
+importStatement
+@init { msgs.push("import statement"); }
+@after { msgs.pop(); }
+	: KW_IMPORT ((ext=KW_EXTERNAL)? KW_TABLE (tab=tableOrPartition))? KW_FROM (path=StringLiteral) tableLocation?
+    ;
+
+ddlStatement
+@init { msgs.push("ddl statement"); }
+@after { msgs.pop(); }
+    : createDatabaseStatement
+    | switchDatabaseStatement
+    | dropDatabaseStatement
+    | createTableStatement
+    | dropTableStatement
+    | truncateTableStatement
+    | alterStatement
+    | descStatement
+    | showStatement
+    | metastoreCheck
+    | createViewStatement
+    | dropViewStatement
+    | createFunctionStatement
+    | createIndexStatement
+    | dropIndexStatement
+    | dropFunctionStatement
+    | analyzeStatement
+    | lockStatement
+    | unlockStatement
+    | createRoleStatement
+    | dropRoleStatement
+    | grantPrivileges
+    | revokePrivileges
+    | showGrants
+    | showRoleGrants
+    | grantRole
+    | revokeRole
+    ;
+
+ifExists
+@init { msgs.push("if exists clause"); }
+@after { msgs.pop(); }
+    : KW_IF KW_EXISTS
+    ;
+
+restrictOrCascade
+@init { msgs.push("restrict or cascade clause"); }
+@after { msgs.pop(); }
+    : KW_RESTRICT
+    | KW_CASCADE
+    ;
+
+ifNotExists
+@init { msgs.push("if not exists clause"); }
+@after { msgs.pop(); }
+    : KW_IF KW_NOT KW_EXISTS
+    ;
+
+storedAsDirs
+@init { msgs.push("stored as directories"); }
+@after { msgs.pop(); }
+    : KW_STORED KW_AS KW_DIRECTORIES
+    ;
+
+orReplace
+@init { msgs.push("or replace clause"); }
+@after { msgs.pop(); }
+    : KW_OR KW_REPLACE
+    ;
+
+ignoreProtection
+@init { msgs.push("ignore protection clause"); }
+@after { msgs.pop(); }
+        : KW_IGNORE KW_PROTECTION
+        ;
+
+createDatabaseStatement
+@init { msgs.push("create database statement"); }
+@after { msgs.pop(); }
+    : KW_CREATE (KW_DATABASE|KW_SCHEMA)
+        ifNotExists?
+        name=identifier
+        databaseComment?
+        dbLocation?
+        (KW_WITH KW_DBPROPERTIES dbprops=dbProperties)?
+    ;
+
+dbLocation
+@init { msgs.push("database location specification"); }
+@after { msgs.pop(); }
+    :
+      KW_LOCATION locn=StringLiteral 
+    ;
+
+dbProperties
+@init { msgs.push("dbproperties"); }
+@after { msgs.pop(); }
+    :
+      LPAREN dbPropertiesList RPAREN 
+    ;
+
+dbPropertiesList
+@init { msgs.push("database properties list"); }
+@after { msgs.pop(); }
+    :
+      keyValueProperty (COMMA keyValueProperty)* 
+    ;
+
+
+switchDatabaseStatement
+@init { msgs.push("switch database statement"); }
+@after { msgs.pop(); }
+    : KW_USE identifier
+    ;
+
+dropDatabaseStatement
+@init { msgs.push("drop database statement"); }
+@after { msgs.pop(); }
+    : KW_DROP (KW_DATABASE|KW_SCHEMA) ifExists? identifier restrictOrCascade?
+    ;
+
+databaseComment
+@init { msgs.push("database's comment"); }
+@after { msgs.pop(); }
+    : KW_COMMENT comment=StringLiteral
+    ;
+
+createTableStatement
+@init { msgs.push("create table statement"); }
+@after { msgs.pop(); }
+    : KW_CREATE (ext=KW_EXTERNAL)? KW_TABLE ifNotExists? name=tableName
+      (  like=KW_LIKE likeName=tableName
+         tableLocation?
+         tablePropertiesPrefixed?
+       | (LPAREN columnNameTypeList RPAREN)?
+         tableComment?
+         tablePartition?
+         tableBuckets?
+         tableSkewed?
+         tableRowFormat?
+         tableFileFormat?
+         tableLocation?
+         tablePropertiesPrefixed?
+         (KW_AS selectStatement)?
+      )
+    ;
+
+truncateTableStatement
+@init { msgs.push("truncate table statement"); }
+@after { msgs.pop(); }
+    : KW_TRUNCATE KW_TABLE tablePartitionPrefix 
+;
+
+createIndexStatement
+@init { msgs.push("create index statement");}
+@after {msgs.pop();}
+    : KW_CREATE KW_INDEX indexName=identifier
+      KW_ON KW_TABLE tab=tableName LPAREN indexedCols=columnNameList RPAREN
+      KW_AS typeName=StringLiteral
+      autoRebuild?
+      indexPropertiesPrefixed?
+      indexTblName?
+      tableRowFormat?
+      tableFileFormat?
+      tableLocation?
+      tablePropertiesPrefixed?
+      indexComment?
+    ;
+
+indexComment
+@init { msgs.push("comment on an index");}
+@after {msgs.pop();}
+        :
+                KW_COMMENT comment=StringLiteral  
+        ;
+
+autoRebuild
+@init { msgs.push("auto rebuild index");}
+@after {msgs.pop();}
+    : KW_WITH KW_DEFERRED KW_REBUILD
+    ;
+
+indexTblName
+@init { msgs.push("index table name");}
+@after {msgs.pop();}
+    : KW_IN KW_TABLE indexTbl=tableName
+    ;
+
+indexPropertiesPrefixed
+@init { msgs.push("table properties with prefix"); }
+@after { msgs.pop(); }
+    :
+        KW_IDXPROPERTIES indexProperties
+    ;
+
+indexProperties
+@init { msgs.push("index properties"); }
+@after { msgs.pop(); }
+    :
+      LPAREN indexPropertiesList RPAREN 
+    ;
+
+indexPropertiesList
+@init { msgs.push("index properties list"); }
+@after { msgs.pop(); }
+    :
+      keyValueProperty (COMMA keyValueProperty)* 
+    ;
+
+dropIndexStatement
+@init { msgs.push("drop index statement");}
+@after {msgs.pop();}
+    : KW_DROP KW_INDEX ifExists? indexName=identifier KW_ON tab=tableName
+    ;
+
+dropTableStatement
+@init { msgs.push("drop statement"); }
+@after { msgs.pop(); }
+    : KW_DROP KW_TABLE ifExists? tableName 
+    ;
+
+alterStatement
+@init { msgs.push("alter statement"); }
+@after { msgs.pop(); }
+    : 
+    KW_ALTER
+        (
+            KW_TABLE alterTableStatementSuffix
+        |
+            KW_VIEW alterViewStatementSuffix
+        |
+            KW_INDEX alterIndexStatementSuffix
+        |
+            KW_DATABASE alterDatabaseStatementSuffix
+        )
+    ;
+
+alterTableStatementSuffix
+@init { msgs.push("alter table statement"); }
+@after { msgs.pop(); }
+    : alterStatementSuffixRename
+    | alterStatementSuffixAddCol
+    | alterStatementSuffixRenameCol
+    | alterStatementSuffixDropPartitions
+    | alterStatementSuffixAddPartitions
+    | alterStatementSuffixTouch
+    | alterStatementSuffixArchive
+    | alterStatementSuffixUnArchive
+    | alterStatementSuffixProperties
+    | alterTblPartitionStatement
+    | alterStatementSuffixSkewedby
+    ;
+
+alterViewStatementSuffix
+@init { msgs.push("alter view statement"); }
+@after { msgs.pop(); }
+    : alterViewSuffixProperties
+    | alterStatementSuffixRename
+    | alterStatementSuffixAddPartitions
+    | alterStatementSuffixDropPartitions
+    | name=tableName KW_AS selectStatement
+    ;
+
+alterIndexStatementSuffix
+@init { msgs.push("alter index statement"); }
+@after { msgs.pop(); }
+    : indexName=identifier
+      (KW_ON tableNameId=identifier)
+      partitionSpec?
+    (
+      KW_REBUILD
+    |
+      KW_SET KW_IDXPROPERTIES
+      indexProperties
+    )
+    ;
+
+alterDatabaseStatementSuffix
+@init { msgs.push("alter database statement"); }
+@after { msgs.pop(); }
+    : alterDatabaseSuffixProperties
+    ;
+
+alterDatabaseSuffixProperties
+@init { msgs.push("alter database properties statement"); }
+@after { msgs.pop(); }
+    : name=identifier KW_SET KW_DBPROPERTIES dbProperties
+    ;
+
+alterStatementSuffixRename
+@init { msgs.push("rename statement"); }
+@after { msgs.pop(); }
+    : oldName=identifier KW_RENAME KW_TO newName=identifier
+    ;
+
+alterStatementSuffixAddCol
+@init { msgs.push("add column statement"); }
+@after { msgs.pop(); }
+    : identifier (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN
+    ;
+
+alterStatementSuffixRenameCol
+@init { msgs.push("rename column name"); }
+@after { msgs.pop(); }
+    : identifier KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition?
+    ;
+
+alterStatementChangeColPosition
+    : first=KW_FIRST|KW_AFTER afterCol=identifier
+    ;
+
+alterStatementSuffixAddPartitions
+@init { msgs.push("add partition statement"); }
+@after { msgs.pop(); }
+    : identifier KW_ADD ifNotExists? partitionSpec partitionLocation? (partitionSpec partitionLocation?)*
+    ;
+
+alterStatementSuffixTouch
+@init { msgs.push("touch statement"); }
+@after { msgs.pop(); }
+    : identifier KW_TOUCH (partitionSpec)*
+    ;
+
+alterStatementSuffixArchive
+@init { msgs.push("archive statement"); }
+@after { msgs.pop(); }
+    : identifier KW_ARCHIVE (partitionSpec)*
+    ;
+
+alterStatementSuffixUnArchive
+@init { msgs.push("unarchive statement"); }
+@after { msgs.pop(); }
+    : identifier KW_UNARCHIVE (partitionSpec)*
+    ;
+
+partitionLocation
+@init { msgs.push("partition location"); }
+@after { msgs.pop(); }
+    :
+      KW_LOCATION locn=StringLiteral 
+    ;
+
+alterStatementSuffixDropPartitions
+@init { msgs.push("drop partition statement"); }
+@after { msgs.pop(); }
+    : identifier KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
+    ;
+
+alterStatementSuffixProperties
+@init { msgs.push("alter properties statement"); }
+@after { msgs.pop(); }
+    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
+    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    ;
+
+alterViewSuffixProperties
+@init { msgs.push("alter view properties statement"); }
+@after { msgs.pop(); }
+    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
+    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    ;
+
+alterStatementSuffixSerdeProperties
+@init { msgs.push("alter serdes statement"); }
+@after { msgs.pop(); }
+    : KW_SET KW_SERDE serdeName=StringLiteral (KW_WITH KW_SERDEPROPERTIES tableProperties)?
+    | KW_SET KW_SERDEPROPERTIES tableProperties
+    ;
+
+tablePartitionPrefix
+@init {msgs.push("table partition prefix");}
+@after {msgs.pop();}
+  :name=identifier partitionSpec?
+  ;
+
+alterTblPartitionStatement
+@init {msgs.push("alter table partition statement");}
+@after {msgs.pop();}
+  : tablePartitionPrefix alterTblPartitionStatementSuffix
+  |Identifier KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
+  ;
+
+alterTblPartitionStatementSuffix
+@init {msgs.push("alter table partition statement suffix");}
+@after {msgs.pop();}
+  : alterStatementSuffixFileFormat
+  | alterStatementSuffixLocation
+  | alterStatementSuffixProtectMode
+  | alterStatementSuffixMergeFiles
+  | alterStatementSuffixSerdeProperties
+  | alterStatementSuffixRenamePart
+  | alterStatementSuffixBucketNum
+  | alterTblPartitionStatementSuffixSkewedLocation
+  | alterStatementSuffixClusterbySortby
+  ;
+
+alterStatementSuffixFileFormat
+@init {msgs.push("alter fileformat statement"); }
+@after {msgs.pop();}
+	: KW_SET KW_FILEFORMAT fileFormat
+	;
+
+alterStatementSuffixClusterbySortby
+@init {msgs.push("alter partition cluster by sort by statement");}
+@after {msgs.pop();}
+  : KW_NOT KW_CLUSTERED 
+  | KW_NOT KW_SORTED 
+  | tableBuckets 
+  ;
+
+alterTblPartitionStatementSuffixSkewedLocation
+@init {msgs.push("alter partition skewed location");}
+@after {msgs.pop();}
+  : KW_SET KW_SKEWED KW_LOCATION skewedLocations
+  ;
+  
+skewedLocations
+@init { msgs.push("skewed locations"); }
+@after { msgs.pop(); }
+    :
+      LPAREN skewedLocationsList RPAREN 
+    ;
+
+skewedLocationsList
+@init { msgs.push("skewed locations list"); }
+@after { msgs.pop(); }
+    :
+      skewedLocationMap (COMMA skewedLocationMap)* 
+    ;
+
+skewedLocationMap
+@init { msgs.push("specifying skewed location map"); }
+@after { msgs.pop(); }
+    :
+      key=skewedValueLocationElement EQUAL value=StringLiteral 
+    ;
+
+alterStatementSuffixLocation
+@init {msgs.push("alter location");}
+@after {msgs.pop();}
+  : KW_SET KW_LOCATION newLoc=StringLiteral
+  ;
+
+	
+alterStatementSuffixSkewedby
+@init {msgs.push("alter skewed by statement");}
+@after{msgs.pop();}
+	:name=identifier tableSkewed
+	|
+	name=identifier KW_NOT KW_SKEWED
+	|
+	name=identifier KW_NOT storedAsDirs
+	;
+
+alterStatementSuffixProtectMode
+@init { msgs.push("alter partition protect mode statement"); }
+@after { msgs.pop(); }
+    : alterProtectMode
+    ;
+
+alterStatementSuffixRenamePart
+@init { msgs.push("alter table rename partition statement"); }
+@after { msgs.pop(); }
+    : KW_RENAME KW_TO partitionSpec
+    ;
+
+alterStatementSuffixMergeFiles
+@init { msgs.push(""); }
+@after { msgs.pop(); }
+    : KW_CONCATENATE
+    ;
+
+alterProtectMode
+@init { msgs.push("protect mode specification enable"); }
+@after { msgs.pop(); }
+    : KW_ENABLE alterProtectModeMode  
+    | KW_DISABLE alterProtectModeMode  
+    ;
+
+alterProtectModeMode
+@init { msgs.push("protect mode specification enable"); }
+@after { msgs.pop(); }
+    : KW_OFFLINE  
+    | KW_NO_DROP KW_CASCADE? 
+    | KW_READONLY  
+    ;
+
+alterStatementSuffixBucketNum
+@init { msgs.push(""); }
+@after { msgs.pop(); }
+    : KW_INTO num=Number KW_BUCKETS
+    ;
+
+fileFormat
+@init { msgs.push("file format specification"); }
+@after { msgs.pop(); }
+    : KW_SEQUENCEFILE  
+    | KW_TEXTFILE  
+    | KW_RCFILE  
+    | KW_ORCFILE 
+    | KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
+    | genericSpec=identifier 
+    ;
+
+tabTypeExpr
+@init { msgs.push("specifying table types"); }
+@after { msgs.pop(); }
+
+   : 
+   identifier (DOT (KW_ELEM_TYPE | KW_KEY_TYPE | KW_VALUE_TYPE | identifier))*
+   ;
+
+descTabTypeExpr
+@init { msgs.push("specifying describe table types"); }
+@after { msgs.pop(); }
+
+   : 
+   identifier (DOT (KW_ELEM_TYPE | KW_KEY_TYPE | KW_VALUE_TYPE | identifier))* identifier?
+   ;
+
+partTypeExpr
+@init { msgs.push("specifying table partitions"); }
+@after { msgs.pop(); }
+    :  tabTypeExpr partitionSpec? 
+    ;
+
+descPartTypeExpr
+@init { msgs.push("specifying describe table partitions"); }
+@after { msgs.pop(); }
+    :  descTabTypeExpr partitionSpec? 
+    ;
+
+descStatement
+@init { msgs.push("describe statement"); }
+@after { msgs.pop(); }
+    : (KW_DESCRIBE|KW_DESC) (descOptions=KW_FORMATTED|descOptions=KW_EXTENDED|descOptions=KW_PRETTY)? (parttype=descPartTypeExpr) 
+    | (KW_DESCRIBE|KW_DESC) KW_FUNCTION KW_EXTENDED? (name=descFuncNames) 
+    | (KW_DESCRIBE|KW_DESC) KW_DATABASE KW_EXTENDED? (dbName=identifier) 
+    ;
+
+analyzeStatement
+@init { msgs.push("analyze statement"); }
+@after { msgs.pop(); }
+    : KW_ANALYZE KW_TABLE (parttype=tableOrPartition) KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN) | (KW_FOR KW_COLUMNS statsColumnName=columnNameList))? 
+    ;
+
+showStatement
+@init { msgs.push("show statement"); }
+@after { msgs.pop(); }
+    : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? 
+    | KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  
+    | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tabname=tableName ((KW_FROM|KW_IN) db_name=identifier)? 
+    | KW_SHOW KW_FUNCTIONS showStmtIdentifier?  
+    | KW_SHOW KW_PARTITIONS identifier partitionSpec? 
+    | KW_SHOW KW_CREATE KW_TABLE tabName=tableName 
+    | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec?
+    | KW_SHOW KW_TBLPROPERTIES tblName=identifier (LPAREN prptyName=StringLiteral RPAREN)? 
+    | KW_SHOW KW_LOCKS (parttype=partTypeExpr)? (isExtended=KW_EXTENDED)? 
+    | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
+    ;
+
+lockStatement
+@init { msgs.push("lock statement"); }
+@after { msgs.pop(); }
+    : KW_LOCK KW_TABLE tableName partitionSpec? lockMode 
+    ;
+
+lockMode
+@init { msgs.push("lock mode"); }
+@after { msgs.pop(); }
+    : KW_SHARED | KW_EXCLUSIVE
+    ;
+
+unlockStatement
+@init { msgs.push("unlock statement"); }
+@after { msgs.pop(); }
+    : KW_UNLOCK KW_TABLE tableName partitionSpec?  
+    ;
+
+createRoleStatement
+@init { msgs.push("create role"); }
+@after { msgs.pop(); }
+    : KW_CREATE KW_ROLE roleName=identifier
+    ;
+
+dropRoleStatement
+@init {msgs.push("drop role");}
+@after {msgs.pop();}
+    : KW_DROP KW_ROLE roleName=identifier
+    ;
+
+grantPrivileges
+@init {msgs.push("grant privileges");}
+@after {msgs.pop();}
+    : KW_GRANT privList=privilegeList
+      privilegeObject?
+      KW_TO principalSpecification
+      (KW_WITH withOption)?
+    ;
+
+revokePrivileges
+@init {msgs.push("revoke privileges");}
+@afer {msgs.pop();}
+    : KW_REVOKE privilegeList privilegeObject? KW_FROM principalSpecification
+    ;
+
+grantRole
+@init {msgs.push("grant role");}
+@after {msgs.pop();}
+    : KW_GRANT KW_ROLE identifier (COMMA identifier)* KW_TO principalSpecification
+    ;
+
+revokeRole
+@init {msgs.push("revoke role");}
+@after {msgs.pop();}
+    : KW_REVOKE KW_ROLE identifier (COMMA identifier)* KW_FROM principalSpecification
+    ;
+
+showRoleGrants
+@init {msgs.push("show role grants");}
+@after {msgs.pop();}
+    : KW_SHOW KW_ROLE KW_GRANT principalName
+    ;
+
+showGrants
+@init {msgs.push("show grants");}
+@after {msgs.pop();}
+    : KW_SHOW KW_GRANT principalName privilegeIncludeColObject?
+    ;
+
+privilegeIncludeColObject
+@init {msgs.push("privilege object including columns");}
+@after {msgs.pop();}
+    : KW_ON (table=KW_TABLE|KW_DATABASE) identifier (LPAREN cols=columnNameList RPAREN)? partitionSpec?
+    ;
+
+privilegeObject
+@init {msgs.push("privilege subject");}
+@after {msgs.pop();}
+    : KW_ON (table=KW_TABLE|KW_DATABASE) identifier partitionSpec?
+    ;
+
+privilegeList
+@init {msgs.push("grant privilege list");}
+@after {msgs.pop();}
+    : privlegeDef (COMMA privlegeDef)*
+    ;
+
+privlegeDef
+@init {msgs.push("grant privilege");}
+@after {msgs.pop();}
+    : privilegeType (LPAREN cols=columnNameList RPAREN)?
+    ;
+
+privilegeType
+@init {msgs.push("privilege type");}
+@after {msgs.pop();}
+    : KW_ALL 
+    | KW_ALTER 
+    | KW_UPDATE 
+    | KW_CREATE 
+    | KW_DROP 
+    | KW_INDEX 
+    | KW_LOCK 
+    | KW_SELECT 
+    | KW_SHOW_DATABASE 
+    ;
+
+principalSpecification
+@init { msgs.push("user/group/role name list"); }
+@after { msgs.pop(); }
+    : principalName (COMMA principalName)* 
+    ;
+
+principalName
+@init {msgs.push("user|group|role name");}
+@after {msgs.pop();}
+    : KW_USER identifier 
+    | KW_GROUP identifier 
+    | KW_ROLE identifier 
+    ;
+
+withOption
+@init {msgs.push("grant with option");}
+@after {msgs.pop();}
+    : KW_GRANT KW_OPTION
+    ;
+
+metastoreCheck
+@init { msgs.push("metastore check statement"); }
+@after { msgs.pop(); }
+    : KW_MSCK (repair=KW_REPAIR)? (KW_TABLE table=identifier partitionSpec? (COMMA partitionSpec)*)?
+    ;
+
+createFunctionStatement
+@init { msgs.push("create function statement"); }
+@after { msgs.pop(); }
+    : KW_CREATE KW_TEMPORARY KW_FUNCTION identifier KW_AS StringLiteral
+    ;
+
+dropFunctionStatement
+@init { msgs.push("drop temporary function statement"); }
+@after { msgs.pop(); }
+    : KW_DROP KW_TEMPORARY KW_FUNCTION ifExists? identifier
+    ;
+
+createViewStatement
+@init {
+    msgs.push("create view statement");
+}
+@after { msgs.pop(); }
+    : KW_CREATE (orReplace)? KW_VIEW (ifNotExists)? name=tableName
+        (LPAREN columnNameCommentList RPAREN)? tableComment? viewPartition?
+        tablePropertiesPrefixed?
+        KW_AS
+        selectStatement
+    ;
+
+viewPartition
+@init { msgs.push("view partition specification"); }
+@after { msgs.pop(); }
+    : KW_PARTITIONED KW_ON LPAREN columnNameList RPAREN
+    ;
+
+dropViewStatement
+@init { msgs.push("drop view statement"); }
+@after { msgs.pop(); }
+    : KW_DROP KW_VIEW ifExists? viewName 
+    ;
+
+showStmtIdentifier
+@init { msgs.push("identifier for show statement"); }
+@after { msgs.pop(); }
+    : identifier
+    | StringLiteral
+    ;
+
+tableComment
+@init { msgs.push("table's comment"); }
+@after { msgs.pop(); }
+    :
+      KW_COMMENT comment=StringLiteral  
+    ;
+
+tablePartition
+@init { msgs.push("table partition specification"); }
+@after { msgs.pop(); }
+    : KW_PARTITIONED KW_BY LPAREN columnNameTypeList RPAREN
+    ;
+
+tableBuckets
+@init { msgs.push("table buckets specification"); }
+@after { msgs.pop(); }
+    :
+      KW_CLUSTERED KW_BY LPAREN bucketCols=columnNameList RPAREN (KW_SORTED KW_BY LPAREN sortCols=columnNameOrderList RPAREN)? KW_INTO num=Number KW_BUCKETS
+    ;
+
+tableSkewed
+@init { msgs.push("table skewed specification"); }
+@after { msgs.pop(); }
+    :
+     KW_SKEWED KW_BY LPAREN skewedCols=columnNameList RPAREN KW_ON LPAREN (skewedValues=skewedValueElement) RPAREN (storedAsDirs)?
+    ;
+
+rowFormat
+@init { msgs.push("serde specification"); }
+@after { msgs.pop(); }
+    : rowFormatSerde 
+    | rowFormatDelimited 
+    ;
+
+recordReader
+@init { msgs.push("record reader specification"); }
+@after { msgs.pop(); }
+    : KW_RECORDREADER StringLiteral 
+    ;
+
+recordWriter
+@init { msgs.push("record writer specification"); }
+@after { msgs.pop(); }
+    : KW_RECORDWRITER StringLiteral 
+    ;
+
+rowFormatSerde
+@init { msgs.push("serde format specification"); }
+@after { msgs.pop(); }
+    : KW_ROW KW_FORMAT KW_SERDE name=StringLiteral (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
+    ;
+
+rowFormatDelimited
+@init { msgs.push("serde properties specification"); }
+@after { msgs.pop(); }
+    :
+      KW_ROW KW_FORMAT KW_DELIMITED tableRowFormatFieldIdentifier? tableRowFormatCollItemsIdentifier? tableRowFormatMapKeysIdentifier? tableRowFormatLinesIdentifier?
+    ;
+
+tableRowFormat
+@init { msgs.push("table row format specification"); }
+@after { msgs.pop(); }
+    :
+      rowFormatDelimited
+    | rowFormatSerde
+    ;
+
+tablePropertiesPrefixed
+@init { msgs.push("table properties with prefix"); }
+@after { msgs.pop(); }
+    :
+        KW_TBLPROPERTIES tableProperties
+    ;
+
+tableProperties
+@init { msgs.push("table properties"); }
+@after { msgs.pop(); }
+    :
+      LPAREN tablePropertiesList RPAREN 
+    ;
+
+tablePropertiesList
+@init { msgs.push("table properties list"); }
+@after { msgs.pop(); }
+    :
+      keyValueProperty (COMMA keyValueProperty)* 
+    |
+      keyProperty (COMMA keyProperty)* 
+    ;
+
+keyValueProperty
+@init { msgs.push("specifying key/value property"); }
+@after { msgs.pop(); }
+    :
+      key=StringLiteral EQUAL value=StringLiteral 
+    ;
+
+keyProperty
+@init { msgs.push("specifying key property"); }
+@after { msgs.pop(); }
+    :
+      key=StringLiteral 
+    ;
+
+tableRowFormatFieldIdentifier
+@init { msgs.push("table row format's field separator"); }
+@after { msgs.pop(); }
+    :
+      KW_FIELDS KW_TERMINATED KW_BY fldIdnt=StringLiteral (KW_ESCAPED KW_BY fldEscape=StringLiteral)?
+    ;
+
+tableRowFormatCollItemsIdentifier
+@init { msgs.push("table row format's column separator"); }
+@after { msgs.pop(); }
+    :
+      KW_COLLECTION KW_ITEMS KW_TERMINATED KW_BY collIdnt=StringLiteral
+    ;
+
+tableRowFormatMapKeysIdentifier
+@init { msgs.push("table row format's map key separator"); }
+@after { msgs.pop(); }
+    :
+      KW_MAP KW_KEYS KW_TERMINATED KW_BY mapKeysIdnt=StringLiteral
+    ;
+
+tableRowFormatLinesIdentifier
+@init { msgs.push("table row format's line separator"); }
+@after { msgs.pop(); }
+    :
+      KW_LINES KW_TERMINATED KW_BY linesIdnt=StringLiteral
+    ;
+
+tableFileFormat
+@init { msgs.push("table file format specification"); }
+@after { msgs.pop(); }
+    :
+      KW_STORED KW_AS KW_SEQUENCEFILE  
+      | KW_STORED KW_AS KW_TEXTFILE  
+      | KW_STORED KW_AS KW_RCFILE  
+      | KW_STORED KW_AS KW_ORCFILE 
+      | KW_STORED KW_AS KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
+      | KW_STORED KW_BY storageHandler=StringLiteral
+         (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
+      | KW_STORED KW_AS genericSpec=identifier
+    ;
+
+tableLocation
+@init { msgs.push("table location specification"); }
+@after { msgs.pop(); }
+    :
+      KW_LOCATION locn=StringLiteral 
+    ;
+
+columnNameTypeList
+@init { msgs.push("column name type list"); }
+@after { msgs.pop(); }
+    : columnNameType (COMMA columnNameType)* 
+    ;
+
+columnNameColonTypeList
+@init { msgs.push("column name type list"); }
+@after { msgs.pop(); }
+    : columnNameColonType (COMMA columnNameColonType)* 
+    ;
+
+columnNameList
+@init { msgs.push("column name list"); }
+@after { msgs.pop(); }
+    : columnName (COMMA columnName)* 
+    ;
+
+columnName
+@init { msgs.push("column name"); }
+@after { msgs.pop(); }
+    :
+      identifier
+    ;
+
+columnNameOrderList
+@init { msgs.push("column name order list"); }
+@after { msgs.pop(); }
+    : columnNameOrder (COMMA columnNameOrder)* 
+    ;
+
+skewedValueElement
+@init { msgs.push("skewed value element"); }
+@after { msgs.pop(); }
+    : 
+      skewedColumnValues
+     | skewedColumnValuePairList
+    ;
+
+skewedColumnValuePairList
+@init { msgs.push("column value pair list"); }
+@after { msgs.pop(); }
+    : skewedColumnValuePair (COMMA skewedColumnValuePair)* 
+    ;
+
+skewedColumnValuePair
+@init { msgs.push("column value pair"); }
+@after { msgs.pop(); }
+    : 
+      LPAREN colValues=skewedColumnValues RPAREN 
+    ;
+
+skewedColumnValues
+@init { msgs.push("column values"); }
+@after { msgs.pop(); }
+    : skewedColumnValue (COMMA skewedColumnValue)* 
+    ;
+
+skewedColumnValue
+@init { msgs.push("column value"); }
+@after { msgs.pop(); }
+    :
+      constant
+    ;
+
+skewedValueLocationElement
+@init { msgs.push("skewed value location element"); }
+@after { msgs.pop(); }
+    : 
+      skewedColumnValue
+     | skewedColumnValuePair
+    ;
+    
+columnNameOrder
+@init { msgs.push("column name order"); }
+@after { msgs.pop(); }
+    : identifier (asc=KW_ASC | desc=KW_DESC)?
+    ;
+
+columnNameCommentList
+@init { msgs.push("column name comment list"); }
+@after { msgs.pop(); }
+    : columnNameComment (COMMA columnNameComment)* 
+    ;
+
+columnNameComment
+@init { msgs.push("column name comment"); }
+@after { msgs.pop(); }
+    : colName=identifier (KW_COMMENT comment=StringLiteral)?
+    ;
+
+columnRefOrder
+@init { msgs.push("column order"); }
+@after { msgs.pop(); }
+    : expression (asc=KW_ASC | desc=KW_DESC)?
+    ;
+
+columnNameType
+@init { msgs.push("column specification"); }
+@after { msgs.pop(); }
+    : colName=identifier colType (KW_COMMENT comment=StringLiteral)?
+    ;
+
+columnNameColonType
+@init { msgs.push("column specification"); }
+@after { msgs.pop(); }
+    : colName=identifier COLON colType (KW_COMMENT comment=StringLiteral)?
+    ;
+
+colType
+@init { msgs.push("column type"); }
+@after { msgs.pop(); }
+    : type
+    ;
+
+colTypeList
+@init { msgs.push("column type list"); }
+@after { msgs.pop(); }
+    : colType (COMMA colType)* 
+    ;
+
+type
+    : primitiveType
+    | listType
+    | structType
+    | mapType
+    | unionType;
+
+primitiveType
+@init { msgs.push("primitive type specification"); }
+@after { msgs.pop(); }
+    : KW_TINYINT       
+    | KW_SMALLINT      
+    | KW_INT           
+    | KW_BIGINT        
+    | KW_BOOLEAN       
+    | KW_FLOAT         
+    | KW_DOUBLE        
+    | KW_DATE          
+    | KW_DATETIME      
+    | KW_TIMESTAMP     
+    | KW_STRING        
+    | KW_BINARY        
+    | KW_DECIMAL       
+    ;
+
+listType
+@init { msgs.push("list type"); }
+@after { msgs.pop(); }
+    : KW_ARRAY LESSTHAN type GREATERTHAN   
+    ;
+
+structType
+@init { msgs.push("struct type"); }
+@after { msgs.pop(); }
+    : KW_STRUCT LESSTHAN columnNameColonTypeList GREATERTHAN 
+    ;
+
+mapType
+@init { msgs.push("map type"); }
+@after { msgs.pop(); }
+    : KW_MAP LESSTHAN left=primitiveType COMMA right=type GREATERTHAN
+    ;
+
+unionType
+@init { msgs.push("uniontype type"); }
+@after { msgs.pop(); }
+    : KW_UNIONTYPE LESSTHAN colTypeList GREATERTHAN 
+    ;
+
+queryOperator
+@init { msgs.push("query operator"); }
+@after { msgs.pop(); }
+    : KW_UNION KW_ALL 
+    ;
+
+// select statement select ... from ... where ... group by ... order by ...
+queryStatementExpression
+    : 
+    queryStatement (queryOperator queryStatement)*
+    ;
+
+queryStatement
+    :
+    fromClause
+    ( b+=body )+ 
+    | regular_body
+    ;
+
+regular_body
+   :
+   insertClause
+   selectClause
+   fromClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   |
+   selectStatement
+   ;
+
+selectStatement
+   :
+   selectClause
+   fromClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   ;
+
+
+body
+   :
+   insertClause
+   selectClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   |
+   selectClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   ;
+
+insertClause
+@init { msgs.push("insert clause"); }
+@after { msgs.pop(); }
+   :
+     KW_INSERT KW_OVERWRITE destination ifNotExists? 
+   | KW_INSERT KW_INTO KW_TABLE tableOrPartition
+   ;
+
+destination
+@init { msgs.push("destination specification"); }
+@after { msgs.pop(); }
+   :
+     KW_LOCAL KW_DIRECTORY StringLiteral tableRowFormat? tableFileFormat? 
+   | KW_DIRECTORY StringLiteral 
+   | KW_TABLE tableOrPartition 
+   ;
+
+limitClause
+@init { msgs.push("limit clause"); }
+@after { msgs.pop(); }
+   :
+   KW_LIMIT num=Number 
+   ;
+
+    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
new file mode 100644
index 0000000..7fa7973
--- /dev/null
+++ b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
@@ -0,0 +1,479 @@
+/*
+ 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.
+ */
+
+lexer grammar SQLLexer;
+
+@header {
+}
+
+@members {
+}
+
+
+/*
+===============================================================================
+  Tokens for Case Insensitive Keywords
+===============================================================================
+*/
+fragment A
+	:	'A' | 'a';
+
+fragment B
+	:	'B' | 'b';
+
+fragment C
+	:	'C' | 'c';
+
+fragment D
+	:	'D' | 'd';
+
+fragment E
+	:	'E' | 'e';
+
+fragment F
+	:	'F' | 'f';
+
+fragment G
+	:	'G' | 'g';
+
+fragment H
+	:	'H' | 'h';
+
+fragment I
+	:	'I' | 'i';
+
+fragment J
+	:	'J' | 'j';
+
+fragment K
+	:	'K' | 'k';
+
+fragment L
+	:	'L' | 'l';
+
+fragment M
+	:	'M' | 'm';
+
+fragment N
+	:	'N' | 'n';
+
+fragment O
+	:	'O' | 'o';
+
+fragment P
+	:	'P' | 'p';
+
+fragment Q
+	:	'Q' | 'q';
+
+fragment R
+	:	'R' | 'r';
+
+fragment S
+	:	'S' | 's';
+
+fragment T
+	:	'T' | 't';
+
+fragment U
+	:	'U' | 'u';
+
+fragment V
+	:	'V' | 'v';
+
+fragment W
+	:	'W' | 'w';
+
+fragment X
+	:	'X' | 'x';
+
+fragment Y
+	:	'Y' | 'y';
+
+fragment Z
+	:	'Z' | 'z';
+
+/*
+===============================================================================
+  Reserved Keywords
+===============================================================================
+*/
+
+AS : A S;
+ALL : A L L;
+AND : A N D;
+ANY : A N Y;
+ASYMMETRIC : A S Y M M E T R I C;
+ASC : A S C;
+
+
+BOTH : B O T H;
+
+CASE : C A S E;
+CAST : C A S T;
+CREATE : C R E A T E;
+CROSS : C R O S S;
+
+DESC : D E S C;
+DISTINCT : D I S T I N C T;
+
+END : E N D;
+ELSE : E L S E;
+EXCEPT : E X C E P T;
+
+FALSE : F A L S E;
+FULL : F U L L;
+FROM : F R O M;
+
+GROUP : G R O U P;
+
+HAVING : H A V I N G;
+
+ILIKE : I L I K E;
+IN : I N;
+INNER : I N N E R;
+INTERSECT : I N T E R S E C T;
+INTO : I N T O;
+IS : I S;
+
+JOIN : J O I N;
+
+LEADING : L E A D I N G;
+LEFT : L E F T;
+LIKE : L I K E;
+LIMIT : L I M I T;
+
+NATURAL : N A T U R A L;
+NOT : N O T;
+NULL : N U L L;
+
+ON : O N;
+OUTER : O U T E R;
+OR : O R;
+ORDER : O R D E R;
+RIGHT : R I G H T;
+SELECT : S E L E C T;
+SOME : S O M E;
+SYMMETRIC : S Y M M E T R I C;
+
+TABLE : T A B L E;
+THEN : T H E N;
+TRAILING : T R A I L I N G;
+TRUE : T R U E;
+
+UNION : U N I O N;
+UNIQUE : U N I Q U E;
+USING : U S I N G;
+
+WHEN : W H E N;
+WHERE : W H E R E;
+WITH : W I T H;
+
+/*
+===============================================================================
+  Non Reserved Keywords
+===============================================================================
+*/
+AVG : A V G;
+ADD: A D D;
+ALTER : A L T E R;
+
+BETWEEN : B E T W E E N;
+BY : B Y;
+
+CENTURY : C E N T U R Y;
+CHARACTER : C H A R A C T E R;
+COLLECT : C O L L E C T;
+COALESCE : C O A L E S C E;
+COLUMN : C O L U M N;
+COUNT : C O U N T;
+CUBE : C U B E;
+
+DAY : D A Y;
+DATABASE : D A T A B A S E;
+DEC : D E C;
+DECADE : D E C A D E;
+DOW : D O W;
+DOY : D O Y;
+DROP : D R O P;
+
+EPOCH : E P O C H;
+EVERY : E V E R Y;
+EXISTS : E X I S T S;
+EXPLAIN : E X P L A I N;
+EXTERNAL : E X T E R N A L;
+EXTRACT : E X T R A C T;
+
+FILTER : F I L T E R;
+FIRST : F I R S T;
+FORMAT : F O R M A T;
+FUSION : F U S I O N;
+
+GROUPING : G R O U P I N G;
+
+HASH : H A S H;
+HOUR : H O U R;
+
+IF : I F;
+INDEX : I N D E X;
+INSERT : I N S E R T;
+INTERSECTION : I N T E R S E C T I O N;
+ISODOW : I S O D O W;
+ISOYEAR : I S O Y E A R;
+
+LAST : L A S T;
+LESS : L E S S;
+LIST : L I S T;
+LOCATION : L O C A T I O N;
+
+MAX : M A X;
+MAXVALUE : M A X V A L U E;
+MICROSECONDS : M I C R O S E C O N D S;
+MILLENNIUM : M I L L E N N I U M;
+MILLISECONDS : M I L L I S E C O N D S;
+MIN : M I N;
+MINUTE : M I N U T E;
+MONTH : M O N T H;
+
+NATIONAL : N A T I O N A L;
+NULLIF : N U L L I F;
+
+OVERWRITE : O V E R W R I T E;
+
+PARTITION : P A R T I T I O N;
+PARTITIONS : P A R T I T I O N S;
+PRECISION : P R E C I S I O N;
+PURGE : P U R G E;
+
+QUARTER : Q U A R T E R;
+
+RANGE : R A N G E;
+REGEXP : R E G E X P;
+RLIKE : R L I K E;
+ROLLUP : R O L L U P;
+RENAME : R E N A M E;
+
+SECOND : S E C O N D;
+SET : S E T;
+SIMILAR : S I M I L A R;
+STDDEV_POP : S T D D E V UNDERLINE P O P;
+STDDEV_SAMP : S T D D E V UNDERLINE S A M P;
+SUBPARTITION : S U B P A R T I T I O N;
+SUM : S U M;
+
+TABLESPACE : T A B L E S P A C E;
+THAN : T H A N;
+TIMEZONE: T I M E Z O N E;
+TIMEZONE_HOUR: T I M E Z O N E UNDERLINE H O U R;
+TIMEZONE_MINUTE: T I M E Z O N E UNDERLINE M I N U T E;
+TRIM : T R I M;
+TO : T O;
+
+UNKNOWN : U N K N O W N;
+
+VALUES : V A L U E S;
+VAR_SAMP : V A R UNDERLINE S A M P;
+VAR_POP : V A R UNDERLINE P O P;
+VARYING : V A R Y I N G;
+
+WEEK : W E E K;
+
+YEAR : Y E A R;
+
+ZONE : Z O N E;
+
+
+/*
+===============================================================================
+  Data Type Tokens
+===============================================================================
+*/
+BOOLEAN : B O O L E A N;
+BOOL : B O O L;
+BIT : B I T;
+VARBIT : V A R B I T;
+
+INT1 : I N T '1';
+INT2 : I N T '2';
+INT4 : I N T '4';
+INT8 : I N T '8';
+
+TINYINT : T I N Y I N T; // alias for INT1
+SMALLINT : S M A L L I N T; // alias for INT2
+INT : I N T; // alias for INT4
+INTEGER : I N T E G E R; // alias - INT4
+BIGINT : B I G I N T; // alias for INT8
+
+FLOAT4 : F L O A T '4';
+FLOAT8 : F L O A T '8';
+
+REAL : R E A L; // alias for FLOAT4
+FLOAT : F L O A T; // alias for FLOAT8
+DOUBLE : D O U B L E; // alias for FLOAT8
+
+NUMERIC : N U M E R I C;
+DECIMAL : D E C I M A L; // alias for number
+
+CHAR : C H A R;
+VARCHAR : V A R C H A R;
+NCHAR : N C H A R;
+NVARCHAR : N V A R C H A R;
+
+DATE : D A T E;
+TIME : T I M E;
+TIMETZ : T I M E T Z;
+TIMESTAMP : T I M E S T A M P;
+TIMESTAMPTZ : T I M E S T A M P T Z;
+
+TEXT : T E X T;
+
+BINARY : B I N A R Y;
+VARBINARY : V A R B I N A R Y;
+BLOB : B L O B;
+BYTEA : B Y T E A; // alias for BLOB
+
+INET4 : I N E T '4';
+
+// Operators
+Similar_To : '~';
+Not_Similar_To : '!~';
+Similar_To_Case_Insensitive : '~*';
+Not_Similar_To_Case_Insensitive : '!~*';
+
+// Cast Operator
+CAST_EXPRESSION
+  : COLON COLON
+  ;
+
+ASSIGN  : ':=';
+EQUAL  : '=';
+COLON :  ':';
+SEMI_COLON :  ';';
+COMMA : ',';
+CONCATENATION_OPERATOR : VERTICAL_BAR VERTICAL_BAR;
+NOT_EQUAL  : '<>' | '!=' | '~='| '^=' ;
+LTH : '<' ;
+LEQ : '<=';
+GTH   : '>';
+GEQ   : '>=';
+LEFT_PAREN :  '(';
+RIGHT_PAREN : ')';
+PLUS  : '+';
+MINUS : '-';
+MULTIPLY: '*';
+DIVIDE  : '/';
+MODULAR : '%';
+DOT : '.';
+UNDERLINE : '_';
+VERTICAL_BAR : '|';
+QUOTE : '\'';
+DOUBLE_QUOTE : '"';
+
+NUMBER : Digit+;
+
+fragment
+Digit : '0'..'9';
+
+REAL_NUMBER
+    :   ('0'..'9')+ '.' ('0'..'9')* EXPONENT?
+    |   '.' ('0'..'9')+ EXPONENT?
+    |   ('0'..'9')+ EXPONENT
+    ;
+
+BlockComment
+    :   '/*' .*? '*/' -> skip
+    ;
+
+LineComment
+    :   '--' ~[\r\n]* -> skip
+    ;
+
+/*
+===============================================================================
+ Identifiers
+===============================================================================
+*/
+
+Regular_Identifier
+  : ('a'..'z'|'A'..'Z'|'_') ('a'..'z'|'A'..'Z'|Digit|'_')* { setText(getText().toLowerCase()); }
+  ;
+
+Quoted_Identifier
+  : DOUBLE_QUOTE ( ESC_SEQ | ~('\\'|'"') )* DOUBLE_QUOTE { setText(getText().substring(1, getText().length()-1)); }
+  ;
+
+/*
+===============================================================================
+ Literal
+===============================================================================
+*/
+
+// Some Unicode Character Ranges
+fragment
+Control_Characters                  :   '\u0001' .. '\u001F';
+fragment
+Extended_Control_Characters         :   '\u0080' .. '\u009F';
+
+Character_String_Literal
+  : QUOTE ( ESC_SEQ | ~('\\'|'\'') )* QUOTE
+  ;
+
+fragment
+EXPONENT : ('e'|'E') ('+'|'-')? ('0'..'9')+ ;
+
+fragment
+HEX_DIGIT : ('0'..'9'|'a'..'f'|'A'..'F') ;
+
+fragment
+ESC_SEQ
+    :   '\\' ('b'|'t'|'n'|'f'|'r'|'\"'|'\''|'\\')
+    |   UNICODE_ESC
+    |   OCTAL_ESC
+    ;
+
+fragment
+OCTAL_ESC
+    :   '\\' ('0'..'3') ('0'..'7') ('0'..'7')
+    |   '\\' ('0'..'7') ('0'..'7')
+    |   '\\' ('0'..'7')
+    ;
+
+fragment
+UNICODE_ESC
+    :   '\\' 'u' HEX_DIGIT HEX_DIGIT HEX_DIGIT HEX_DIGIT
+    ;
+
+
+/*
+===============================================================================
+ Whitespace Tokens
+===============================================================================
+*/
+
+Space
+  : ' ' -> skip
+  ;
+
+White_Space
+  :	( Control_Characters  | Extended_Control_Characters )+ -> skip
+  ;
+
+
+BAD
+  : . -> skip
+  ;


[48/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
new file mode 100644
index 0000000..f6385eb
--- /dev/null
+++ b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
@@ -0,0 +1,1398 @@
+/**
+ * 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.
+ */
+
+parser grammar SQLParser;
+
+options {
+	language=Java;
+	tokenVocab=SQLLexer;
+}
+
+@header {
+}
+
+@members {
+}
+
+/*
+===============================================================================
+  SQL statement (Start Symbol)
+===============================================================================
+*/
+sql
+  : (explain_clause)? statement (SEMI_COLON)? EOF
+  ;
+
+explain_clause
+  : EXPLAIN
+  ;
+
+statement
+  : data_statement
+  | data_change_statement
+  | schema_statement
+  | index_statement
+  ;
+
+data_statement
+  : query_expression
+  ;
+
+data_change_statement
+  : insert_statement
+  ;
+
+schema_statement
+  : database_definition
+  | drop_database_statement
+  | create_table_statement
+  | drop_table_statement
+  | alter_tablespace_statement
+  | alter_table_statement
+  ;
+
+index_statement
+  : CREATE (u=UNIQUE)? INDEX n=identifier ON t=table_name (m=method_specifier)?
+    LEFT_PAREN s=sort_specifier_list RIGHT_PAREN p=param_clause?
+  ;
+
+database_definition
+  : CREATE DATABASE (if_not_exists)? dbname = identifier
+  ;
+
+if_not_exists
+  : IF NOT EXISTS
+  ;
+
+drop_database_statement
+  : DROP DATABASE (if_exists)? dbname = identifier
+  ;
+
+if_exists
+  : IF EXISTS
+  ;
+
+create_table_statement
+  : CREATE EXTERNAL TABLE (if_not_exists)? table_name table_elements USING file_type=identifier
+    (param_clause)? (table_partitioning_clauses)? (LOCATION path=Character_String_Literal)
+  | CREATE TABLE (if_not_exists)? table_name table_elements (USING file_type=identifier)?
+    (param_clause)? (table_partitioning_clauses)? (AS query_expression)?
+  | CREATE TABLE (if_not_exists)? table_name (USING file_type=identifier)?
+    (param_clause)? (table_partitioning_clauses)? AS query_expression
+  ;
+
+table_elements
+  : LEFT_PAREN field_element (COMMA field_element)* RIGHT_PAREN
+  ;
+
+field_element
+  : name=identifier field_type
+  ;
+
+field_type
+  : data_type
+  ;
+
+param_clause
+  : WITH LEFT_PAREN param (COMMA param)* RIGHT_PAREN
+  ;
+
+param
+  : key=Character_String_Literal EQUAL value=numeric_value_expression
+  ;
+
+method_specifier
+  : USING m=identifier
+  ;
+
+table_space_specifier
+  : TABLESPACE table_space_name
+  ;
+
+table_space_name
+  : identifier
+  ;
+
+table_partitioning_clauses
+  : range_partitions
+  | hash_partitions
+  | list_partitions
+  | column_partitions
+  ;
+
+range_partitions
+  : PARTITION BY RANGE LEFT_PAREN column_reference_list RIGHT_PAREN
+    LEFT_PAREN range_value_clause_list RIGHT_PAREN
+  ;
+
+range_value_clause_list
+  : range_value_clause (COMMA range_value_clause)*
+  ;
+
+range_value_clause
+  : PARTITION partition_name VALUES LESS THAN (LEFT_PAREN value_expression RIGHT_PAREN | LEFT_PAREN? MAXVALUE RIGHT_PAREN?)
+  ;
+
+hash_partitions
+  : PARTITION BY HASH LEFT_PAREN column_reference_list RIGHT_PAREN
+    (LEFT_PAREN individual_hash_partitions RIGHT_PAREN | hash_partitions_by_quantity)
+  ;
+
+individual_hash_partitions
+  : individual_hash_partition (COMMA individual_hash_partition)*
+  ;
+
+individual_hash_partition
+  : PARTITION partition_name
+  ;
+
+hash_partitions_by_quantity
+  : PARTITIONS quantity = numeric_value_expression
+  ;
+
+list_partitions
+  : PARTITION BY LIST LEFT_PAREN column_reference_list RIGHT_PAREN LEFT_PAREN  list_value_clause_list RIGHT_PAREN
+  ;
+
+list_value_clause_list
+  : list_value_partition (COMMA list_value_partition)*
+  ;
+
+list_value_partition
+  : PARTITION partition_name VALUES (IN)? LEFT_PAREN in_value_list RIGHT_PAREN
+  ;
+
+column_partitions
+  : PARTITION BY COLUMN table_elements
+  ;
+
+partition_name
+  : identifier
+  ;
+
+/*
+===============================================================================
+  11.21 <data types>
+===============================================================================
+*/
+
+drop_table_statement
+  : DROP TABLE (if_exists)? table_name (PURGE)?
+  ;
+
+/*
+===============================================================================
+  5.2 <token and separator>
+
+  Specifying lexical units (tokens and separators) that participate in SQL language
+===============================================================================
+*/
+
+identifier
+  : Regular_Identifier
+  | nonreserved_keywords
+  | Quoted_Identifier
+  ;
+
+nonreserved_keywords
+  : ADD
+  | AVG
+  | ALTER
+  | BETWEEN
+  | BY
+  | CENTURY
+  | CHARACTER
+  | COALESCE
+  | COLLECT
+  | COLUMN
+  | COUNT
+  | CUBE
+  | DAY
+  | DEC
+  | DECADE
+  | DOW
+  | DOY
+  | DROP
+  | EPOCH
+  | EVERY
+  | EXISTS
+  | EXPLAIN
+  | EXTERNAL
+  | EXTRACT
+  | FILTER
+  | FIRST
+  | FORMAT
+  | FUSION
+  | GROUPING
+  | HASH
+  | INDEX
+  | INSERT
+  | INTERSECTION
+  | ISODOW
+  | ISOYEAR
+  | LAST
+  | LESS
+  | LIST
+  | LOCATION
+  | MAX
+  | MAXVALUE
+  | MICROSECONDS
+  | MILLENNIUM
+  | MILLISECONDS
+  | MIN
+  | MINUTE
+  | MONTH
+  | NATIONAL
+  | NULLIF
+  | OVERWRITE
+  | PARTITION
+  | PARTITIONS
+  | PRECISION
+  | PURGE
+  | QUARTER
+  | RANGE
+  | REGEXP
+  | RENAME
+  | RLIKE
+  | ROLLUP
+  | SECOND
+  | SET
+  | SIMILAR
+  | STDDEV_POP
+  | STDDEV_SAMP
+  | SUBPARTITION
+  | SUM
+  | TABLESPACE
+  | THAN
+  | TIMEZONE
+  | TIMEZONE_HOUR
+  | TIMEZONE_MINUTE
+  | TRIM
+  | TO
+  | UNKNOWN
+  | VALUES
+  | VAR_POP
+  | VAR_SAMP
+  | VARYING
+  | WEEK
+  | YEAR
+  | ZONE
+
+  | BIGINT
+  | BIT
+  | BLOB
+  | BOOL
+  | BOOLEAN
+  | BYTEA
+  | CHAR
+  | DATE
+  | DECIMAL
+  | DOUBLE
+  | FLOAT
+  | FLOAT4
+  | FLOAT8
+  | INET4
+  | INT
+  | INT1
+  | INT2
+  | INT4
+  | INT8
+  | INTEGER
+  | NCHAR
+  | NUMERIC
+  | NVARCHAR
+  | REAL
+  | SMALLINT
+  | TEXT
+  | TIME
+  | TIMESTAMP
+  | TIMESTAMPTZ
+  | TIMETZ
+  | TINYINT
+  | VARBINARY
+  | VARBIT
+  | VARCHAR
+  ;
+
+/*
+===============================================================================
+  5.3 <literal>
+===============================================================================
+*/
+
+unsigned_literal
+  : unsigned_numeric_literal
+  | general_literal
+  ;
+
+general_literal
+  : Character_String_Literal
+  | datetime_literal
+  | boolean_literal
+  ;
+
+datetime_literal
+  : timestamp_literal
+  | time_literal
+  | date_literal
+  ;
+
+time_literal
+  : TIME time_string=Character_String_Literal
+  ;
+
+timestamp_literal
+  : TIMESTAMP timestamp_string=Character_String_Literal
+  ;
+
+date_literal
+  : DATE date_string=Character_String_Literal
+  ;
+
+boolean_literal
+  : TRUE | FALSE | UNKNOWN
+  ;
+
+/*
+===============================================================================
+  6.1 <data types>
+===============================================================================
+*/
+
+data_type
+  : predefined_type
+  ;
+
+predefined_type
+  : character_string_type
+  | national_character_string_type
+  | binary_large_object_string_type
+  | numeric_type
+  | boolean_type
+  | datetime_type
+  | bit_type
+  | binary_type
+  | network_type
+  ;
+
+network_type
+  : INET4
+  ;
+
+character_string_type
+  : CHARACTER type_length?
+  | CHAR type_length?
+  | CHARACTER VARYING type_length?
+  | CHAR VARYING type_length?
+  | VARCHAR type_length?
+  | TEXT
+  ;
+
+type_length
+  : LEFT_PAREN NUMBER RIGHT_PAREN
+  ;
+
+national_character_string_type
+  : NATIONAL CHARACTER type_length?
+  | NATIONAL CHAR type_length?
+  | NCHAR type_length?
+  | NATIONAL CHARACTER VARYING type_length?
+  | NATIONAL CHAR VARYING type_length?
+  | NCHAR VARYING type_length?
+  | NVARCHAR type_length?
+  ;
+
+binary_large_object_string_type
+  : BLOB type_length?
+  | BYTEA type_length?
+  ;
+
+numeric_type
+  : exact_numeric_type | approximate_numeric_type
+  ;
+
+exact_numeric_type
+  : NUMERIC (precision_param)?
+  | DECIMAL (precision_param)?
+  | DEC (precision_param)?
+  | INT1
+  | TINYINT
+  | INT2
+  | SMALLINT
+  | INT4
+  | INT
+  | INTEGER
+  | INT8
+  | BIGINT
+  ;
+
+approximate_numeric_type
+  : FLOAT (precision_param)?
+  | FLOAT4
+  | REAL
+  | FLOAT8
+  | DOUBLE
+  | DOUBLE PRECISION
+  ;
+
+precision_param
+  : LEFT_PAREN precision=NUMBER RIGHT_PAREN
+  | LEFT_PAREN precision=NUMBER COMMA scale=NUMBER RIGHT_PAREN
+  ;
+
+boolean_type
+  : BOOLEAN
+  | BOOL
+  ;
+
+datetime_type
+  : DATE
+  | TIME
+  | TIME WITH TIME ZONE
+  | TIMETZ
+  | TIMESTAMP
+  | TIMESTAMP WITH TIME ZONE
+  | TIMESTAMPTZ
+  ;
+
+bit_type
+  : BIT type_length?
+  | VARBIT type_length?
+  | BIT VARYING type_length?
+  ;
+
+binary_type
+  : BINARY type_length?
+  | BINARY VARYING type_length?
+  | VARBINARY type_length?
+  ;
+
+/*
+===============================================================================
+  6.3 <value_expression_primary>
+===============================================================================
+*/
+value_expression_primary
+  : parenthesized_value_expression
+  | nonparenthesized_value_expression_primary
+  ;
+
+parenthesized_value_expression
+  : LEFT_PAREN value_expression RIGHT_PAREN
+  ;
+
+nonparenthesized_value_expression_primary
+  : unsigned_value_specification
+  | column_reference
+  | set_function_specification
+  | scalar_subquery
+  | case_expression
+  | cast_specification
+  | routine_invocation
+  ;
+
+/*
+===============================================================================
+  6.4 <unsigned value specification>
+===============================================================================
+*/
+
+unsigned_value_specification
+  : unsigned_literal
+  ;
+
+unsigned_numeric_literal
+  : NUMBER
+  | REAL_NUMBER
+  ;
+
+signed_numerical_literal
+  : sign? unsigned_numeric_literal
+  ;
+
+/*
+===============================================================================
+  6.9 <set function specification>
+
+  Invoke an SQL-invoked routine.
+===============================================================================
+*/
+set_function_specification
+  : aggregate_function
+  ;
+
+aggregate_function
+  : COUNT LEFT_PAREN MULTIPLY RIGHT_PAREN
+  | general_set_function filter_clause?
+  ;
+
+general_set_function
+  : set_function_type LEFT_PAREN set_qualifier? value_expression RIGHT_PAREN
+  ;
+
+set_function_type
+  : AVG
+  | MAX
+  | MIN
+  | SUM
+  | EVERY
+  | ANY
+  | SOME
+  | COUNT
+  | STDDEV_POP
+  | STDDEV_SAMP
+  | VAR_SAMP
+  | VAR_POP
+  | COLLECT
+  | FUSION
+  | INTERSECTION
+  ;
+
+filter_clause
+  : FILTER LEFT_PAREN WHERE search_condition RIGHT_PAREN
+  ;
+
+grouping_operation
+  : GROUPING LEFT_PAREN column_reference_list RIGHT_PAREN
+  ;
+
+/*
+===============================================================================
+  6.11 <case expression>
+===============================================================================
+*/
+
+case_expression
+  : case_specification
+  ;
+
+case_abbreviation
+  : NULLIF LEFT_PAREN numeric_value_expression COMMA boolean_value_expression  RIGHT_PAREN
+  | COALESCE LEFT_PAREN numeric_value_expression ( COMMA boolean_value_expression  )+ RIGHT_PAREN
+  ;
+
+case_specification
+  : simple_case
+  | searched_case
+  ;
+
+simple_case
+  : CASE boolean_value_expression ( simple_when_clause )+ ( else_clause  )? END
+  ;
+
+searched_case
+  : CASE (searched_when_clause)+ (else_clause)? END
+  ;
+
+simple_when_clause : WHEN search_condition THEN result ;
+
+searched_when_clause
+  : WHEN c=search_condition THEN r=result
+  ;
+
+else_clause
+  : ELSE r=result
+  ;
+
+result
+  : value_expression | NULL
+  ;
+
+/*
+===============================================================================
+  6.12 <cast specification>
+===============================================================================
+*/
+
+cast_specification
+  : CAST LEFT_PAREN cast_operand AS cast_target RIGHT_PAREN
+  ;
+
+cast_operand
+  : value_expression
+  ;
+
+cast_target
+  : data_type
+  ;
+
+/*
+===============================================================================
+  6.25 <value expression>
+===============================================================================
+*/
+value_expression
+  : common_value_expression
+  | row_value_expression
+  | boolean_value_expression
+  ;
+
+common_value_expression
+  : numeric_value_expression
+  | string_value_expression
+  | NULL
+  ;
+
+/*
+===============================================================================
+  6.26 <numeric value expression>
+
+  Specify a comparison of two row values.
+===============================================================================
+*/
+
+numeric_value_expression
+  : left=term ((PLUS|MINUS) right=term)*
+  ;
+
+term
+  : left=factor ((MULTIPLY|DIVIDE|MODULAR) right=factor)*
+  ;
+
+factor
+  : (sign)? numeric_primary
+  ;
+
+array
+  : LEFT_PAREN numeric_value_expression (COMMA numeric_value_expression )* RIGHT_PAREN
+  ;
+
+numeric_primary
+  : value_expression_primary (CAST_EXPRESSION cast_target)*
+  | numeric_value_function
+  ;
+
+sign
+  : PLUS | MINUS
+  ;
+
+/*
+===============================================================================
+  6.27 <numeric value function>
+===============================================================================
+*/
+
+numeric_value_function
+  : extract_expression
+  ;
+
+extract_expression
+  : EXTRACT LEFT_PAREN extract_field_string=extract_field FROM extract_source RIGHT_PAREN
+  ;
+
+extract_field
+  : primary_datetime_field
+  | time_zone_field
+  | extended_datetime_field
+  ;
+
+time_zone_field
+  : TIMEZONE | TIMEZONE_HOUR | TIMEZONE_MINUTE
+  ;
+
+extract_source
+  : column_reference
+  | datetime_literal
+  ;
+
+/*
+===============================================================================
+  6.28 <string value expression>
+===============================================================================
+*/
+
+string_value_expression
+  : character_value_expression
+  ;
+
+character_value_expression
+  : character_factor (CONCATENATION_OPERATOR character_factor)*
+  ;
+
+character_factor
+  : character_primary
+  ;
+
+character_primary
+  : value_expression_primary
+  | string_value_function
+  ;
+
+/*
+===============================================================================
+  6.29 <string value function>
+===============================================================================
+*/
+
+string_value_function
+  : trim_function
+  ;
+
+trim_function
+  : TRIM LEFT_PAREN trim_operands RIGHT_PAREN
+  ;
+
+trim_operands
+  : ((trim_specification)? (trim_character=character_value_expression)? FROM)? trim_source=character_value_expression
+  | trim_source=character_value_expression COMMA trim_character=character_value_expression
+  ;
+
+trim_specification
+  : LEADING | TRAILING | BOTH
+  ;
+
+/*
+===============================================================================
+  6.34 <boolean value expression>
+===============================================================================
+*/
+
+boolean_value_expression
+  : or_predicate
+  ;
+
+or_predicate
+  : and_predicate (OR or_predicate)*
+  ;
+
+and_predicate
+  : boolean_factor (AND and_predicate)*
+  ;
+
+boolean_factor
+  : boolean_test
+  | NOT boolean_test
+  ;
+
+boolean_test
+  : boolean_primary is_clause?
+  ;
+
+is_clause
+  : IS NOT? t=truth_value
+  ;
+
+truth_value
+  : TRUE | FALSE | UNKNOWN
+  ;
+
+boolean_primary
+  : predicate
+  | boolean_predicand
+  ;
+
+boolean_predicand
+  : parenthesized_boolean_value_expression 
+  | nonparenthesized_value_expression_primary
+  ;
+
+parenthesized_boolean_value_expression
+  : LEFT_PAREN boolean_value_expression RIGHT_PAREN
+  ;
+
+/*
+===============================================================================
+  7.2 <row value expression>
+===============================================================================
+*/
+row_value_expression
+  : row_value_special_case
+  | explicit_row_value_constructor
+  ;
+
+row_value_special_case
+  : nonparenthesized_value_expression_primary
+  ;
+
+explicit_row_value_constructor
+  : NULL
+  ;
+
+row_value_predicand
+  : row_value_special_case
+  | row_value_constructor_predicand
+  ;
+
+row_value_constructor_predicand
+  : common_value_expression
+  | boolean_predicand
+//  | explicit_row_value_constructor
+  ;
+
+/*
+===============================================================================
+  7.4 <table expression>
+===============================================================================
+*/
+
+table_expression
+  : from_clause
+    where_clause?
+    groupby_clause?
+    having_clause?
+    orderby_clause?
+    limit_clause?
+  ;
+
+/*
+===============================================================================
+  7.5 <from clause>
+===============================================================================
+*/
+
+from_clause
+  : FROM table_reference_list
+  ;
+
+table_reference_list
+  :table_reference (COMMA table_reference)*
+  ;
+
+/*
+===============================================================================
+  7.6 <table reference>
+===============================================================================
+*/
+
+table_reference
+  : joined_table
+  | table_primary
+  ;
+
+/*
+===============================================================================
+  7.7 <joined table>
+===============================================================================
+*/
+
+joined_table
+  : table_primary joined_table_primary+
+  ;
+
+joined_table_primary
+  : CROSS JOIN right=table_primary
+  | (t=join_type)? JOIN right=table_primary s=join_specification
+  | NATURAL (t=join_type)? JOIN right=table_primary
+  | UNION JOIN right=table_primary
+  ;
+
+cross_join
+  : CROSS JOIN r=table_primary
+  ;
+
+qualified_join
+  : (t=join_type)? JOIN r=table_primary s=join_specification
+  ;
+
+natural_join
+  : NATURAL (t=join_type)? JOIN r=table_primary
+  ;
+
+union_join
+  : UNION JOIN r=table_primary
+  ;
+
+join_type
+  : INNER
+  | t=outer_join_type
+  ;
+
+outer_join_type
+  : outer_join_type_part2 OUTER?
+  ;
+
+outer_join_type_part2
+  : LEFT
+  | RIGHT
+  | FULL
+  ;
+
+join_specification
+  : join_condition
+  | named_columns_join
+  ;
+
+join_condition
+  : ON search_condition
+  ;
+
+named_columns_join
+  : USING LEFT_PAREN f=column_reference_list RIGHT_PAREN
+  ;
+
+table_primary
+  : table_or_query_name ((AS)? alias=identifier)? (LEFT_PAREN column_name_list RIGHT_PAREN)?
+  | derived_table (AS)? name=identifier (LEFT_PAREN column_name_list RIGHT_PAREN)?
+  ;
+
+column_name_list
+  :  identifier  ( COMMA identifier  )*
+  ;
+
+derived_table
+  : table_subquery
+  ;
+
+/*
+===============================================================================
+  7.8 <where clause>
+===============================================================================
+*/
+where_clause
+  : WHERE search_condition
+  ;
+
+search_condition
+  : value_expression // instead of boolean_value_expression, we use value_expression for more flexibility.
+  ;
+
+/*
+===============================================================================
+  7.9 <group by clause>
+===============================================================================
+*/
+groupby_clause
+  : GROUP BY g=grouping_element_list
+  ;
+
+grouping_element_list
+  : grouping_element (COMMA grouping_element)*
+  ;
+
+grouping_element
+  : rollup_list
+  | cube_list
+  | empty_grouping_set
+  | ordinary_grouping_set
+  ;
+
+ordinary_grouping_set
+  : row_value_predicand
+  | LEFT_PAREN row_value_predicand_list RIGHT_PAREN
+  ;
+
+ordinary_grouping_set_list
+  : ordinary_grouping_set (COMMA ordinary_grouping_set)*
+  ;
+
+rollup_list
+  : ROLLUP LEFT_PAREN c=ordinary_grouping_set_list RIGHT_PAREN
+  ;
+
+cube_list
+  : CUBE LEFT_PAREN c=ordinary_grouping_set_list RIGHT_PAREN
+  ;
+
+empty_grouping_set
+  : LEFT_PAREN RIGHT_PAREN
+  ;
+
+having_clause
+  : HAVING boolean_value_expression
+  ;
+
+row_value_predicand_list
+  : row_value_predicand (COMMA row_value_predicand)*
+  ;
+
+/*
+===============================================================================
+  7.13 <query expression>
+===============================================================================
+*/
+query_expression
+  : query_expression_body
+  ;
+
+query_expression_body
+  : non_join_query_expression
+  | joined_table
+  ;
+
+non_join_query_expression
+  : (non_join_query_term
+  | joined_table (UNION | EXCEPT) (ALL|DISTINCT)? query_term)
+    ((UNION | EXCEPT) (ALL|DISTINCT)? query_term)*
+  ;
+
+query_term
+  : non_join_query_term
+  | joined_table
+  ;
+
+non_join_query_term
+  : ( non_join_query_primary
+  | joined_table INTERSECT (ALL|DISTINCT)? query_primary)
+    (INTERSECT (ALL|DISTINCT)? query_primary)*
+  ;
+
+query_primary
+  : non_join_query_primary
+  | joined_table
+  ;
+
+non_join_query_primary
+  : simple_table
+  | LEFT_PAREN non_join_query_expression RIGHT_PAREN
+  ;
+
+simple_table
+  : query_specification
+  | explicit_table
+  ;
+
+explicit_table
+  : TABLE table_or_query_name
+  ;
+
+table_or_query_name
+  : table_name
+  | identifier
+  ;
+
+table_name
+  : identifier (DOT identifier ( DOT identifier)? )?
+  ;
+
+column_name
+  : identifier
+  ;
+
+query_specification
+  : SELECT set_qualifier? select_list table_expression?
+  ;
+
+select_list
+  : select_sublist (COMMA select_sublist)*
+  ;
+
+select_sublist
+  : derived_column
+  | qualified_asterisk
+  ;
+
+derived_column
+  : value_expression as_clause?
+  ;
+
+qualified_asterisk
+  : (tb_name=identifier DOT)? MULTIPLY
+  ;
+
+set_qualifier
+  : DISTINCT
+  | ALL
+  ;
+
+column_reference
+  : ((db_name = identifier DOT)? (tb_name=identifier DOT))? name=identifier
+  ;
+
+as_clause
+  : (AS)? identifier
+  ;
+
+column_reference_list
+  : column_reference (COMMA column_reference)*
+  ;
+
+/*
+==============================================================================================
+  7.15 <subquery>
+
+  Specify a scalar value, a row, or a table derived from a query_expression .
+==============================================================================================
+*/
+
+scalar_subquery
+  :  subquery
+  ;
+
+row_subquery
+  :  subquery
+  ;
+
+table_subquery
+  : subquery
+  ;
+
+subquery
+  :  LEFT_PAREN query_expression RIGHT_PAREN
+  ;
+
+/*
+===============================================================================
+  8.1 <predicate>
+===============================================================================
+*/
+
+predicate
+  : comparison_predicate
+  | between_predicate
+  | in_predicate
+  | pattern_matching_predicate // like predicate and other similar predicates
+  | null_predicate
+  | exists_predicate
+  ;
+
+/*
+===============================================================================
+  8.2 <comparison predicate>
+
+  Specify a comparison of two row values.
+===============================================================================
+*/
+comparison_predicate
+  : left=row_value_predicand c=comp_op right=row_value_predicand
+  ;
+
+comp_op
+  : EQUAL
+  | NOT_EQUAL
+  | LTH
+  | LEQ
+  | GTH
+  | GEQ
+  ;
+
+/*
+===============================================================================
+  8.3 <between predicate>
+===============================================================================
+*/
+
+between_predicate
+  : predicand=row_value_predicand between_predicate_part_2
+  ;
+
+between_predicate_part_2
+  : (NOT)? BETWEEN (ASYMMETRIC | SYMMETRIC)? begin=row_value_predicand AND end=row_value_predicand
+  ;
+
+
+/*
+===============================================================================
+  8.4 <in predicate>
+===============================================================================
+*/
+
+in_predicate
+  : predicand=numeric_value_expression  NOT? IN in_predicate_value
+  ;
+
+in_predicate_value
+  : table_subquery
+  | LEFT_PAREN in_value_list RIGHT_PAREN
+  ;
+
+in_value_list
+  : row_value_predicand  ( COMMA row_value_predicand )*
+  ;
+
+/*
+===============================================================================
+  8.5, 8.6 <pattern matching predicate>
+
+  Specify a pattern-matching comparison.
+===============================================================================
+*/
+
+pattern_matching_predicate
+  : f=row_value_predicand pattern_matcher s=Character_String_Literal
+  ;
+
+pattern_matcher
+  : NOT? negativable_matcher
+  | regex_matcher
+  ;
+
+negativable_matcher
+  : LIKE
+  | ILIKE
+  | SIMILAR TO
+  | REGEXP
+  | RLIKE
+  ;
+
+regex_matcher
+  : Similar_To
+  | Not_Similar_To
+  | Similar_To_Case_Insensitive
+  | Not_Similar_To_Case_Insensitive
+  ;
+
+/*
+===============================================================================
+  8.7 <null predicate>
+
+  Specify a test for a null value.
+===============================================================================
+*/
+
+null_predicate
+  : predicand=row_value_predicand IS (n=NOT)? NULL
+  ;
+
+/*
+==============================================================================================
+  8.8 <quantified comparison predicate>
+
+  Specify a quantified comparison.
+==============================================================================================
+*/
+
+quantified_comparison_predicate
+  : l=numeric_value_expression  c=comp_op q=quantifier s=table_subquery
+  ;
+
+quantifier : all  | some ;
+
+all : ALL;
+
+some : SOME | ANY;
+
+/*
+==============================================================================================
+  8.9 <exists predicate>
+
+  Specify a test for a non_empty set.
+==============================================================================================
+*/
+
+exists_predicate
+  : NOT? EXISTS s=table_subquery
+  ;
+
+
+/*
+==============================================================================================
+  8.10 <unique predicate>
+
+  Specify a test for the absence of duplicate rows
+==============================================================================================
+*/
+
+unique_predicate
+  : UNIQUE s=table_subquery
+  ;
+
+/*
+===============================================================================
+  10.1 <interval qualifier>
+
+  Specify the precision of an interval data type.
+===============================================================================
+*/
+
+primary_datetime_field
+	:	non_second_primary_datetime_field
+	|	SECOND
+	;
+
+non_second_primary_datetime_field
+  : YEAR | MONTH | DAY | HOUR | MINUTE
+  ;
+
+extended_datetime_field
+  : CENTURY | DECADE | DOW | DOY | EPOCH | ISODOW | ISOYEAR | MICROSECONDS | MILLENNIUM | MILLISECONDS | QUARTER | WEEK
+  ;
+
+/*
+===============================================================================
+  10.4 <routine invocation>
+
+  Invoke an SQL-invoked routine.
+===============================================================================
+*/
+
+routine_invocation
+  : function_name LEFT_PAREN sql_argument_list? RIGHT_PAREN
+  ;
+
+function_names_for_reserved_words
+  : LEFT
+  | RIGHT
+  ;
+
+function_name
+  : identifier
+  | function_names_for_reserved_words
+  ;
+
+sql_argument_list
+  : value_expression (COMMA value_expression)*
+  ;
+
+/*
+===============================================================================
+  14.1 <declare cursor>
+===============================================================================
+*/
+
+orderby_clause
+  : ORDER BY sort_specifier_list
+  ;
+
+sort_specifier_list
+  : sort_specifier (COMMA sort_specifier)*
+  ;
+
+sort_specifier
+  : key=row_value_predicand order=order_specification? null_order=null_ordering?
+  ;
+
+order_specification
+  : ASC
+  | DESC
+  ;
+
+limit_clause
+  : LIMIT e=numeric_value_expression
+  ;
+
+null_ordering
+  : NULL FIRST
+  | NULL LAST
+  ;
+
+/*
+===============================================================================
+  14.8 <insert statement>
+===============================================================================
+*/
+
+insert_statement
+  : INSERT (OVERWRITE)? INTO table_name (LEFT_PAREN column_name_list RIGHT_PAREN)? query_expression
+  | INSERT (OVERWRITE)? INTO LOCATION path=Character_String_Literal (USING file_type=identifier (param_clause)?)? query_expression
+  ;
+
+/*
+===============================================================================
+  <alter table>
+===============================================================================
+*/
+
+alter_tablespace_statement
+  : ALTER TABLESPACE space_name=identifier LOCATION uri=Character_String_Literal
+  ;
+
+alter_table_statement
+  : ALTER TABLE table_name RENAME TO table_name
+  | ALTER TABLE table_name RENAME COLUMN column_name TO column_name
+  | ALTER TABLE table_name ADD COLUMN field_element
+  ;

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/log4j.properties b/tajo-core/src/main/java/log4j.properties
new file mode 100644
index 0000000..15e5778
--- /dev/null
+++ b/tajo-core/src/main/java/log4j.properties
@@ -0,0 +1,30 @@
+##
+# 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.
+#
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout,EventCounter
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
+
+log4j.appender.EventCounter=org.apache.tajo.util.metrics.TajoLogEventCounter
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java b/tajo-core/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java
new file mode 100644
index 0000000..91dcea1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java
@@ -0,0 +1,120 @@
+/**
+ * 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.benchmark;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tajo.catalog.CatalogConstants;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.store.MemStore;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.util.FileUtil;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class BenchmarkSet {
+  protected TajoClient tajo;
+  protected Map<String, Schema> schemas = new HashMap<String, Schema>();
+  protected Map<String, Schema> outSchemas = new HashMap<String, Schema>();
+  protected Map<String, String> queries = new HashMap<String, String>();
+  protected String dataDir;
+
+  public void init(TajoConf conf, String dataDir) throws IOException {
+    this.dataDir = dataDir;
+    if (System.getProperty(ConfVars.WORKER_PEER_RPC_ADDRESS.varname) != null) {
+      tajo = new TajoClient(NetUtils.createSocketAddr(
+          System.getProperty(ConfVars.WORKER_PEER_RPC_ADDRESS.varname)));
+    } else {
+      conf.set(CatalogConstants.STORE_CLASS, MemStore.class.getCanonicalName());
+      tajo = new TajoClient(conf);
+    }
+  }
+
+  protected void loadQueries(String dir) throws IOException {
+    // TODO - this code dead??
+    File queryDir = new File(dir);
+
+    if(!queryDir.exists()) {
+      queryDir = new File(System.getProperty("user.dir") + "/tajo-core/" + dir);
+    }
+
+    if(!queryDir.exists())
+    {
+      return;
+    }
+    int last;
+    String name, query;
+    for (String file : queryDir.list()) {
+      if (file.endsWith(".sql")) {
+        last = file.indexOf(".sql");
+        name = file.substring(0, last);
+        query = FileUtil.readTextFile(new File(queryDir + "/" + file));
+      }
+    }
+  }
+
+  public abstract void loadSchemas();
+
+  public abstract void loadOutSchema();
+
+  public abstract void loadQueries() throws IOException;
+
+  public abstract void loadTables() throws ServiceException;
+
+  public String [] getTableNames() {
+    return schemas.keySet().toArray(new String[schemas.size()]);
+  }
+
+  public String getQuery(String queryName) {
+    return queries.get(queryName);
+  }
+
+  public Schema getSchema(String name) {
+    return schemas.get(name);
+  }
+
+  public Iterable<Schema> getSchemas() {
+    return schemas.values();
+  }
+
+  public Schema getOutSchema(String name) {
+    return outSchemas.get(name);
+  }
+
+  public void perform(String queryName) throws IOException, ServiceException {
+    String query = getQuery(queryName);
+    if (query == null) {
+      throw new IllegalArgumentException("#{queryName} does not exists");
+    }
+    long start = System.currentTimeMillis();
+    tajo.executeQuery(query);
+    long end = System.currentTimeMillis();
+
+    System.out.println("====================================");
+    System.out.println("QueryName: " + queryName);
+    System.out.println("Query: " + query);
+    System.out.println("Processing Time: " + (end - start));
+    System.out.println("====================================");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/benchmark/Driver.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/benchmark/Driver.java b/tajo-core/src/main/java/org/apache/tajo/benchmark/Driver.java
new file mode 100644
index 0000000..99b9a99
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/benchmark/Driver.java
@@ -0,0 +1,46 @@
+/**
+ * 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.benchmark;
+
+import org.apache.tajo.conf.TajoConf;
+
+public class Driver {
+  private static void printUsage() {
+    System.out.println("benchmark BenchmarkClass datadir query");
+  }
+
+  public static void main(String [] args) throws Exception {
+
+    if (args.length < 3) {
+      printUsage();
+      System.exit(-1);
+    }
+
+    TajoConf conf = new TajoConf();
+    Class clazz = Class.forName(args[0]);
+    BenchmarkSet benchmark = (BenchmarkSet) clazz.newInstance();
+
+    benchmark.init(conf, args[1]);
+    benchmark.loadSchemas();
+    benchmark.loadQueries();
+    benchmark.loadTables();
+    benchmark.perform(args[2]);
+    System.exit(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/benchmark/SimpleQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/benchmark/SimpleQuery.java b/tajo-core/src/main/java/org/apache/tajo/benchmark/SimpleQuery.java
new file mode 100644
index 0000000..af6af8d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/benchmark/SimpleQuery.java
@@ -0,0 +1,29 @@
+/**
+ * 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.benchmark;
+
+import java.io.IOException;
+
+public class SimpleQuery extends TPCH {
+  private static final String BENCHMARK_DIR = "benchmark/simple";
+
+  public void loadQueries() throws IOException {
+    loadQueries(BENCHMARK_DIR);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/benchmark/TPCH.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/benchmark/TPCH.java b/tajo-core/src/main/java/org/apache/tajo/benchmark/TPCH.java
new file mode 100644
index 0000000..abd9e7f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/benchmark/TPCH.java
@@ -0,0 +1,200 @@
+/**
+ * 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.benchmark;
+
+import com.google.common.collect.Maps;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.storage.StorageConstants;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Map;
+
+public class TPCH extends BenchmarkSet {
+  private final Log LOG = LogFactory.getLog(TPCH.class);
+  private final String BENCHMARK_DIR = "benchmark/tpch";
+
+  public static final String LINEITEM = "lineitem";
+  public static final String CUSTOMER = "customer";
+  public static final String NATION = "nation";
+  public static final String PART = "part";
+  public static final String REGION = "region";
+  public static final String ORDERS = "orders";
+  public static final String PARTSUPP = "partsupp";
+  public static final String SUPPLIER = "supplier";
+  public static final String EMPTY_ORDERS = "empty_orders";
+
+
+  public static final Map<String, Long> tableVolumes = Maps.newHashMap();
+
+  static {
+    tableVolumes.put(LINEITEM, 759863287L);
+    tableVolumes.put(CUSTOMER, 24346144L);
+    tableVolumes.put(NATION, 2224L);
+    tableVolumes.put(PART, 24135125L);
+    tableVolumes.put(REGION, 389L);
+    tableVolumes.put(ORDERS, 171952161L);
+    tableVolumes.put(PARTSUPP, 118984616L);
+    tableVolumes.put(SUPPLIER, 1409184L);
+    tableVolumes.put(EMPTY_ORDERS, 0L);
+
+  }
+
+  @Override
+  public void loadSchemas() {
+    Schema lineitem = new Schema()
+        .addColumn("l_orderkey", Type.INT4) // 0
+        .addColumn("l_partkey", Type.INT4) // 1
+        .addColumn("l_suppkey", Type.INT4) // 2
+        .addColumn("l_linenumber", Type.INT4) // 3
+        .addColumn("l_quantity", Type.FLOAT8) // 4
+        .addColumn("l_extendedprice", Type.FLOAT8) // 5
+        .addColumn("l_discount", Type.FLOAT8) // 6
+        .addColumn("l_tax", Type.FLOAT8) // 7
+            // TODO - This is temporal solution. 8 and 9 are actually Char type.
+        .addColumn("l_returnflag", Type.TEXT) // 8
+        .addColumn("l_linestatus", Type.TEXT) // 9
+            // TODO - This is temporal solution. 10,11, and 12 are actually Date type.
+        .addColumn("l_shipdate", Type.TEXT) // 10
+        .addColumn("l_commitdate", Type.TEXT) // 11
+        .addColumn("l_receiptdate", Type.TEXT) // 12
+        .addColumn("l_shipinstruct", Type.TEXT) // 13
+        .addColumn("l_shipmode", Type.TEXT) // 14
+        .addColumn("l_comment", Type.TEXT); // 15
+    schemas.put(LINEITEM, lineitem);
+
+    Schema customer = new Schema()
+        .addColumn("c_custkey", Type.INT4) // 0
+        .addColumn("c_name", Type.TEXT) // 1
+        .addColumn("c_address", Type.TEXT) // 2
+        .addColumn("c_nationkey", Type.INT4) // 3
+        .addColumn("c_phone", Type.TEXT) // 4
+        .addColumn("c_acctbal", Type.FLOAT8) // 5
+        .addColumn("c_mktsegment", Type.TEXT) // 6
+        .addColumn("c_comment", Type.TEXT); // 7
+    schemas.put(CUSTOMER, customer);
+
+    Schema nation = new Schema()
+        .addColumn("n_nationkey", Type.INT4) // 0
+        .addColumn("n_name", Type.TEXT) // 1
+        .addColumn("n_regionkey", Type.INT4) // 2
+        .addColumn("n_comment", Type.TEXT); // 3
+    schemas.put(NATION, nation);
+
+    Schema part = new Schema()
+        .addColumn("p_partkey", Type.INT4) // 0
+        .addColumn("p_name", Type.TEXT) // 1
+        .addColumn("p_mfgr", Type.TEXT) // 2
+        .addColumn("p_brand", Type.TEXT) // 3
+        .addColumn("p_type", Type.TEXT) // 4
+        .addColumn("p_size", Type.INT4) // 5
+        .addColumn("p_container", Type.TEXT) // 6
+        .addColumn("p_retailprice", Type.FLOAT8) // 7
+        .addColumn("p_comment", Type.TEXT); // 8
+    schemas.put(PART, part);
+
+    Schema region = new Schema()
+        .addColumn("r_regionkey", Type.INT4) // 0
+        .addColumn("r_name", Type.TEXT) // 1
+        .addColumn("r_comment", Type.TEXT); // 2
+    schemas.put(REGION, region);
+
+    Schema orders = new Schema()
+        .addColumn("o_orderkey", Type.INT4) // 0
+        .addColumn("o_custkey", Type.INT4) // 1
+        .addColumn("o_orderstatus", Type.TEXT) // 2
+        .addColumn("o_totalprice", Type.FLOAT8) // 3
+            // TODO - This is temporal solution. o_orderdate is actually Date type.
+        .addColumn("o_orderdate", Type.TEXT) // 4
+        .addColumn("o_orderpriority", Type.TEXT) // 5
+        .addColumn("o_clerk", Type.TEXT) // 6
+        .addColumn("o_shippriority", Type.INT4) // 7
+        .addColumn("o_comment", Type.TEXT); // 8
+    schemas.put(ORDERS, orders);
+    schemas.put(EMPTY_ORDERS, orders);
+
+
+    Schema partsupp = new Schema()
+        .addColumn("ps_partkey", Type.INT4) // 0
+        .addColumn("ps_suppkey", Type.INT4) // 1
+        .addColumn("ps_availqty", Type.INT4) // 2
+        .addColumn("ps_supplycost", Type.FLOAT8) // 3
+        .addColumn("ps_comment", Type.TEXT); // 4
+    schemas.put(PARTSUPP, partsupp);
+
+    Schema supplier = new Schema()
+        .addColumn("s_suppkey", Type.INT4) // 0
+        .addColumn("s_name", Type.TEXT) // 1
+        .addColumn("s_address", Type.TEXT) // 2
+        .addColumn("s_nationkey", Type.INT4) // 3
+        .addColumn("s_phone", Type.TEXT) // 4
+        .addColumn("s_acctbal", Type.FLOAT8) // 5
+        .addColumn("s_comment", Type.TEXT); // 6
+    schemas.put(SUPPLIER, supplier);
+  }
+
+  public void loadOutSchema() {
+    Schema q2 = new Schema()
+        .addColumn("s_acctbal", Type.FLOAT8)
+        .addColumn("s_name", Type.TEXT)
+        .addColumn("n_name", Type.TEXT)
+        .addColumn("p_partkey", Type.INT4)
+        .addColumn("p_mfgr", Type.TEXT)
+        .addColumn("s_address", Type.TEXT)
+        .addColumn("s_phone", Type.TEXT)
+        .addColumn("s_comment", Type.TEXT);
+    outSchemas.put("q2", q2);
+  }
+
+  public void loadQueries() throws IOException {
+    loadQueries(BENCHMARK_DIR);
+  }
+
+  public void loadTables() throws ServiceException {
+    loadTable(LINEITEM);
+    loadTable(CUSTOMER);
+    loadTable(NATION);
+    loadTable(PART);
+    loadTable(REGION);
+    loadTable(ORDERS);
+    loadTable(PARTSUPP) ;
+    loadTable(SUPPLIER);
+    loadTable(EMPTY_ORDERS);
+
+  }
+
+  private void loadTable(String tableName) throws ServiceException {
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    meta.putOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+
+    try {
+      tajo.createExternalTable(tableName, getSchema(tableName), new Path(dataDir, tableName), meta);
+    } catch (SQLException s) {
+      throw new ServiceException(s);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/AggregationFunctionCallEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/AggregationFunctionCallEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AggregationFunctionCallEval.java
new file mode 100644
index 0000000..10eadce
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AggregationFunctionCallEval.java
@@ -0,0 +1,92 @@
+/**
+ * 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.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.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+public class AggregationFunctionCallEval extends FunctionEval implements Cloneable {
+  @Expose protected AggFunction instance;
+  @Expose boolean firstPhase = false;
+  private Tuple params;
+
+  public AggregationFunctionCallEval(FunctionDesc desc, AggFunction instance, EvalNode[] givenArgs) {
+    super(EvalType.AGG_FUNCTION, desc, givenArgs);
+    this.instance = instance;
+  }
+
+  public FunctionContext newContext() {
+    return instance.newContext();
+  }
+
+  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));
+      }
+    }
+
+    if (firstPhase) {
+      instance.eval(context, params);
+    } else {
+      instance.merge(context, params);
+    }
+  }
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    throw new UnsupportedOperationException("Cannot execute eval() of aggregation function");
+  }
+
+  public Datum terminate(FunctionContext context) {
+    if (firstPhase) {
+      return instance.getPartialResult(context);
+    } else {
+      return instance.terminate(context);
+    }
+  }
+
+  @Override
+  public DataType getValueType() {
+    if (firstPhase) {
+      return instance.getPartialResultType();
+    } else {
+      return funcDesc.getReturnType();
+    }
+  }
+
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+  public void setFirstPhase() {
+    this.firstPhase = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicException.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicException.java
new file mode 100644
index 0000000..9dba6df
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicException.java
@@ -0,0 +1,41 @@
+/**
+ * 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 AlgebraicException extends RuntimeException {
+  private static final long serialVersionUID = -1813125460274622006L;
+  
+  public AlgebraicException() {
+  }
+
+  public AlgebraicException(String message) {
+    super(message);
+  }
+
+  public AlgebraicException(Throwable cause) {
+    super(cause);
+  }
+
+  public AlgebraicException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java
new file mode 100644
index 0000000..6bb0160
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java
@@ -0,0 +1,398 @@
+/**
+ * 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.Column;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class AlgebraicUtil {
+  
+  /**
+   * Transpose a given comparison expression into the expression 
+   * where the variable corresponding to the target is placed 
+   * on the left-hand side.
+   * 
+   * @param expr
+   * @param target
+   * @return Transposed expression
+   */
+  public static EvalNode transpose(EvalNode expr, Column target) {
+    EvalNode commutated = null;
+    // If the variable is in the right term, inverse the expr.
+    if (!EvalTreeUtil.containColumnRef(expr.getLeftExpr(), target)) {
+      // the commutate method works with a copy of the expr
+      commutated = commutate(expr);
+    } else {
+      try {
+        commutated = (EvalNode) expr.clone();
+      } catch (CloneNotSupportedException e) {
+        throw new AlgebraicException(e);
+      }
+    }
+
+    return _transpose(commutated, target);
+  }
+  
+  private static EvalNode _transpose(EvalNode _expr, Column target) {
+     EvalNode expr = eliminateConstantExprs(_expr);
+     
+     if (isSingleVar(expr.getLeftExpr())) {
+       return expr;
+     }
+     
+     EvalNode left = expr.getLeftExpr();     
+     EvalNode lTerm = null;
+     EvalNode rTerm = null;
+     
+    if (left.getType() == EvalType.PLUS
+        || left.getType() == EvalType.MINUS
+        || left.getType() == EvalType.MULTIPLY
+        || left.getType() == EvalType.DIVIDE) {
+      
+      // If the left-left term is a variable, the left-right term is transposed.
+      if(EvalTreeUtil.containColumnRef(left.getLeftExpr(), target)) {
+        PartialBinaryExpr tmpTerm = splitRightTerm(left);
+        tmpTerm.type = inverseOperator(tmpTerm.type);
+        tmpTerm.setLeftExpr(expr.getRightExpr());
+        lTerm = left.getLeftExpr();
+        rTerm = new BinaryEval(tmpTerm);
+      } else { 
+        // Otherwise, the left-right term is transposed into the left-left term.
+        PartialBinaryExpr tmpTerm = splitLeftTerm(left);
+        tmpTerm.type = inverseOperator(tmpTerm.type);
+        tmpTerm.setLeftExpr(expr.getRightExpr());        
+        lTerm = left.getRightExpr();
+        rTerm = new BinaryEval(tmpTerm);    
+      }
+    }
+    
+    return _transpose(new BinaryEval(expr.getType(), lTerm, rTerm), target);
+  }
+  
+  /**
+   * Inverse a given operator (+, -, *, /)
+   * 
+   * @param type
+   * @return inversed operator type
+   */
+  public static EvalType inverseOperator(EvalType type) {
+    switch (type) {
+    case PLUS:
+      return EvalType.MINUS;
+    case MINUS:
+      return EvalType.PLUS;
+    case MULTIPLY:
+      return EvalType.DIVIDE;
+    case DIVIDE:
+      return EvalType.MULTIPLY;
+    default : throw new AlgebraicException("ERROR: cannot inverse the operator: " 
+      + type);
+    }
+  }
+  
+  /**
+   * Examine if a given expr is a variable.
+   * 
+   * @param node
+   * @return true if a given expr is a variable.
+   */
+  private static boolean isSingleVar(EvalNode node) {
+    if (node.getType() == EvalType.FIELD) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+  
+  /**
+   * Simplify the given expr. That is, all subexprs consisting of only constants
+   * are calculated immediately.
+   * 
+   * @param expr to be simplified
+   * @return the simplified expr
+   */
+  public static EvalNode eliminateConstantExprs(EvalNode expr) {
+
+    if (expr.getType() == EvalType.FIELD) {
+      return expr;
+    }
+
+    EvalNode left = expr.getLeftExpr();
+    EvalNode right = expr.getRightExpr();
+    
+    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 MODULAR:
+      left = eliminateConstantExprs(left);
+      right = eliminateConstantExprs(right);
+
+      if (left.getType() == EvalType.CONST && right.getType() == EvalType.CONST) {
+        return new ConstEval(expr.eval(null, null));
+      } else {
+        return new BinaryEval(expr.getType(), left, right);
+      }
+
+    case CONST:
+      return expr;
+      
+    default: new AlgebraicException("Wrong expression: " + expr);
+    }
+    return expr;
+  }
+  
+  /** 
+   * @param expr to be evaluated if the expr includes one variable
+   * @return true if expr has only one field
+   */
+  public static boolean containSingleVar(EvalNode expr) {
+    Map<EvalType, Integer> counter = EvalTreeUtil.getExprCounters(expr);
+    
+    int sum = 0;
+    for (Integer cnt : counter.values()) {      
+      sum += cnt;
+    }
+    
+    if (sum == 1 && counter.get(EvalType.FIELD) == 1) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+  
+  /**
+   * Split the left term and transform it into the right deep expression.
+   * 
+   * @param expr - notice the left term of this expr will be eliminated 
+   * after done.
+   * @return the separated expression changed into the right deep expression.  
+   * For example, the expr 'x * y' is transformed into '* x'.  
+   *
+   */
+  public static PartialBinaryExpr splitLeftTerm(EvalNode expr) {
+    
+    if (!(expr.getType() == EvalType.PLUS
+        || expr.getType() == EvalType.MINUS
+        || expr.getType() == EvalType.MULTIPLY
+        || expr.getType() == EvalType.DIVIDE)) {
+      throw new AlgebraicException("Invalid algebraic operation: " + expr);
+    }
+    
+    if (expr.getLeftExpr().getType() != EvalType.CONST) {
+      return splitLeftTerm(expr.getLeftExpr());
+    }
+    
+    PartialBinaryExpr splitted = 
+        new PartialBinaryExpr(expr.getType(), null, expr.getLeftExpr());
+    expr.setLeftExpr(null);
+    return splitted;
+  }
+  
+  /**
+   * Split the left term and transform it into the right deep expression.
+   * 
+   * @param expr - to be splited
+   * @return the separated expression changed into the right deep expression.
+   * For example, the expr 'x * y' is transformed into '* y'. 
+   *
+   * @throws CloneNotSupportedException
+   */
+  public static PartialBinaryExpr splitRightTerm(EvalNode expr) {
+    
+    if (!(expr.getType() == EvalType.PLUS
+        || expr.getType() == EvalType.MINUS
+        || expr.getType() == EvalType.MULTIPLY
+        || expr.getType() == EvalType.DIVIDE)) {
+      throw new AlgebraicException("Invalid algebraic operation: " + expr);
+    }
+    
+    if (expr.getRightExpr().getType() != EvalType.CONST) {
+      return splitRightTerm(expr.getRightExpr());
+    }
+    
+    PartialBinaryExpr splitted = 
+        new PartialBinaryExpr(expr.getType(), null, expr.getRightExpr());
+    expr.setRightExpr(null);
+    return splitted;
+  }
+  
+  /**
+   * Commutate two terms which are added, subtracted and multiplied.
+   * 
+   * @param inputExpr
+   * @return
+   */
+  public static EvalNode commutate(EvalNode inputExpr) {
+    EvalNode expr;
+    switch (inputExpr.getType()) {
+    case AND:
+    case OR:
+    case EQUAL:
+    case PLUS:
+    case MINUS:
+    case MULTIPLY: // these types can be commutated w/o any change
+      expr = EvalTreeFactory.create(inputExpr.getType(),
+          inputExpr.getRightExpr(), inputExpr.getLeftExpr());
+      break;
+      
+    case GTH:
+      expr = EvalTreeFactory.create(EvalType.LTH,
+          inputExpr.getRightExpr(), inputExpr.getLeftExpr());
+      break;
+    case GEQ:
+      expr = EvalTreeFactory.create(EvalType.LEQ,
+          inputExpr.getRightExpr(), inputExpr.getLeftExpr());
+      break;
+    case LTH:
+      expr = EvalTreeFactory.create(EvalType.GTH,
+          inputExpr.getRightExpr(), inputExpr.getLeftExpr());
+      break;
+    case LEQ:
+      expr = EvalTreeFactory.create(EvalType.GEQ,
+          inputExpr.getRightExpr(), inputExpr.getLeftExpr());
+      break;
+      
+    default :
+      throw new AlgebraicException("Cannot commutate the expr: " + inputExpr);
+    }
+    
+    return expr;
+  }
+
+  public static boolean isComparisonOperator(EvalNode expr) {
+    return expr.getType() == EvalType.EQUAL ||
+        expr.getType() == EvalType.LEQ ||
+        expr.getType() == EvalType.LTH ||
+        expr.getType() == EvalType.GEQ ||
+        expr.getType() == EvalType.GTH ||
+        expr.getType() == EvalType.BETWEEN;
+  }
+
+  public static boolean isIndexableOperator(EvalNode expr) {
+    return expr.getType() == EvalType.EQUAL ||
+        expr.getType() == EvalType.LEQ ||
+        expr.getType() == EvalType.LTH ||
+        expr.getType() == EvalType.GEQ ||
+        expr.getType() == EvalType.GTH ||
+        expr.getType() == EvalType.BETWEEN ||
+        expr.getType() == EvalType.IN ||
+        (expr.getType() == EvalType.LIKE && !((LikePredicateEval)expr).isLeadingWildCard());
+  }
+
+  /**
+   * Convert a list of conjunctive normal forms into a singleton expression.
+   *
+   * @param cnfExprs
+   * @return The EvalNode object that merges all CNF-formed expressions.
+   */
+  public static EvalNode createSingletonExprFromCNF(EvalNode... cnfExprs) {
+    if (cnfExprs.length == 1) {
+      return cnfExprs[0];
+    }
+
+    return createSingletonExprFromCNFRecursive(cnfExprs, 0);
+  }
+
+  private static EvalNode createSingletonExprFromCNFRecursive(EvalNode[] evalNode, int idx) {
+    if (idx == evalNode.length - 2) {
+      return new BinaryEval(EvalType.AND, evalNode[idx], evalNode[idx + 1]);
+    } else {
+      return new BinaryEval(EvalType.AND, evalNode[idx], createSingletonExprFromCNFRecursive(evalNode, idx + 1));
+    }
+  }
+
+  /**
+   * Transforms a expression to an array of conjunctive normal formed expressions.
+   *
+   * @param expr The expression to be transformed to an array of CNF-formed expressions.
+   * @return An array of CNF-formed expressions
+   */
+  public static EvalNode [] toConjunctiveNormalFormArray(EvalNode expr) {
+    List<EvalNode> list = new ArrayList<EvalNode>();
+    toConjunctiveNormalFormArrayRecursive(expr, list);
+    return list.toArray(new EvalNode[list.size()]);
+  }
+
+  private static void toConjunctiveNormalFormArrayRecursive(EvalNode node, List<EvalNode> found) {
+    if (node.getType() == EvalType.AND) {
+      toConjunctiveNormalFormArrayRecursive(node.getLeftExpr(), found);
+      toConjunctiveNormalFormArrayRecursive(node.getRightExpr(), found);
+    } else {
+      found.add(node);
+    }
+  }
+
+  /**
+   * Convert a list of conjunctive normal forms into a singleton expression.
+   *
+   * @param cnfExprs
+   * @return The EvalNode object that merges all CNF-formed expressions.
+   */
+  public static EvalNode createSingletonExprFromDNF(EvalNode... cnfExprs) {
+    if (cnfExprs.length == 1) {
+      return cnfExprs[0];
+    }
+
+    return createSingletonExprFromDNFRecursive(cnfExprs, 0);
+  }
+
+  private static EvalNode createSingletonExprFromDNFRecursive(EvalNode[] evalNode, int idx) {
+    if (idx == evalNode.length - 2) {
+      return new BinaryEval(EvalType.OR, evalNode[idx], evalNode[idx + 1]);
+    } else {
+      return new BinaryEval(EvalType.OR, evalNode[idx], createSingletonExprFromDNFRecursive(evalNode, idx + 1));
+    }
+  }
+
+  /**
+   * Transforms a expression to an array of disjunctive normal formed expressions.
+   *
+   * @param exprs The expressions to be transformed to an array of CNF-formed expressions.
+   * @return An array of CNF-formed expressions
+   */
+  public static EvalNode [] toDisjunctiveNormalFormArray(EvalNode...exprs) {
+    List<EvalNode> list = new ArrayList<EvalNode>();
+    for (EvalNode expr : exprs) {
+      toDisjunctiveNormalFormArrayRecursive(expr, list);
+    }
+    return list.toArray(new EvalNode[list.size()]);
+  }
+
+  private static void toDisjunctiveNormalFormArrayRecursive(EvalNode node, List<EvalNode> found) {
+    if (node.getType() == EvalType.OR) {
+      toDisjunctiveNormalFormArrayRecursive(node.getLeftExpr(), found);
+      toDisjunctiveNormalFormArrayRecursive(node.getRightExpr(), found);
+    } else {
+      found.add(node);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java
new file mode 100644
index 0000000..2208119
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java
@@ -0,0 +1,341 @@
+/**
+ * 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.Stack;
+
+public class BasicEvalNodeVisitor<CONTEXT, RESULT> implements EvalNodeVisitor2<CONTEXT, RESULT> {
+
+  @Override
+  public RESULT visitChild(CONTEXT context, EvalNode evalNode, Stack<EvalNode> stack) {
+    RESULT result;
+    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;
+
+      // Arithmetic expression
+      case PLUS:
+        result = visitPlus(context, (BinaryEval) evalNode, stack);
+        break;
+      case MINUS:
+        result = visitMinus(context, (BinaryEval) evalNode, stack);
+        break;
+      case MULTIPLY:
+        result = visitMultiply(context, (BinaryEval) evalNode, stack);
+        break;
+      case DIVIDE:
+        result = visitDivide(context, (BinaryEval) evalNode, stack);
+        break;
+      case MODULAR:
+        result = visitModular(context, (BinaryEval) evalNode, stack);
+        break;
+
+      // Logical Predicates
+      case AND:
+        result = visitAnd(context, (BinaryEval) evalNode, stack);
+        break;
+      case OR:
+        result = visitOr(context, (BinaryEval) evalNode, stack);
+        break;
+      case NOT:
+        result = visitNot(context, (NotEval) evalNode, stack);
+        break;
+
+      // Comparison Predicates
+      case EQUAL:
+        result = visitEqual(context, (BinaryEval) evalNode, stack);
+        break;
+      case NOT_EQUAL:
+        result = visitNotEqual(context, (BinaryEval) evalNode, stack);
+        break;
+      case LTH:
+        result = visitLessThan(context, (BinaryEval) evalNode, stack);
+        break;
+      case LEQ:
+        result = visitLessThanOrEqual(context, (BinaryEval) evalNode, stack);
+        break;
+      case GTH:
+        result = visitGreaterThan(context, (BinaryEval) evalNode, stack);
+        break;
+      case GEQ:
+        result = visitGreaterThanOrEqual(context, (BinaryEval) evalNode, stack);
+        break;
+
+      // SQL standard predicates
+      case IS_NULL:
+        result = visitIsNull(context, (IsNullEval) evalNode, stack);
+        break;
+      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;
+      case IN:
+        result = visitInPredicate(context, (InEval) evalNode, stack);
+        break;
+
+      // String operators and Pattern match predicates
+      case LIKE:
+        result = visitLike(context, (LikePredicateEval) evalNode, stack);
+        break;
+      case SIMILAR_TO:
+        result = visitSimilarTo(context, (SimilarToPredicateEval) evalNode, stack);
+        break;
+      case REGEX:
+        result = visitRegex(context, (RegexPredicateEval) evalNode, stack);
+        break;
+      case CONCATENATE:
+        result = visitConcatenate(context, (BinaryEval) evalNode, stack);
+        break;
+
+      // Functions
+      case FUNCTION:
+        result = visitFuncCall(context, (GeneralFunctionEval) evalNode, stack);
+        break;
+      case AGG_FUNCTION:
+        result = visitAggrFuncCall(context, (AggregationFunctionCallEval) evalNode, stack);
+        break;
+
+      case SIGNED:
+        result = visitSigned(context, (SignedEval) evalNode, stack);
+        break;
+      case CAST:
+        result = visitCast(context, (CastEval) evalNode, stack);
+        break;
+
+      default:
+        throw new InvalidEvalException("Unknown EvalNode: " + evalNode);
+    }
+
+    return result;
+  }
+
+  private RESULT visitDefaultBinaryEval(CONTEXT context, Stack<EvalNode> stack, BinaryEval binaryEval) {
+    stack.push(binaryEval);
+    RESULT result = visitChild(context, binaryEval.getLeftExpr(), stack);
+    visitChild(context, binaryEval.getRightExpr(), stack);
+    stack.pop();
+    return result;
+  }
+
+  private RESULT visitDefaultFunctionEval(CONTEXT context, Stack<EvalNode> stack, FunctionEval functionEval) {
+    RESULT result = null;
+    stack.push(functionEval);
+    if (functionEval.getArgs() != null) {
+      for (EvalNode arg : functionEval.getArgs()) {
+        result = visitChild(context, arg, stack);
+      }
+    }
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitConst(CONTEXT context, ConstEval evalNode, Stack<EvalNode> stack) {
+    return null;
+  }
+
+  @Override
+  public RESULT visitRowConstant(CONTEXT context, RowConstantEval evalNode, Stack<EvalNode> stack) {
+    return null;
+  }
+
+  @Override
+  public RESULT visitField(CONTEXT context, Stack<EvalNode> stack, FieldEval evalNode) {
+    return null;
+  }
+
+  @Override
+  public RESULT visitPlus(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitMinus(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitMultiply(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitDivide(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitModular(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitAnd(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitOr(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitNot(CONTEXT context, NotEval evalNode, Stack<EvalNode> stack) {
+    RESULT result;
+    stack.push(evalNode);
+    result = visitChild(context, evalNode.getChild(), stack);
+    stack.pop();
+
+    return result;
+  }
+
+  @Override
+  public RESULT visitEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitNotEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitLessThan(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitLessThanOrEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitGreaterThan(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitGreaterThanOrEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitIsNull(CONTEXT context, IsNullEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitBetween(CONTEXT context, BetweenPredicateEval evalNode, Stack<EvalNode> stack) {
+    stack.push(evalNode);
+    RESULT result = visitChild(context, evalNode.getPredicand(), stack);
+    visitChild(context, evalNode.getBegin(), stack);
+    visitChild(context, evalNode.getEnd(), stack);
+    return result;
+  }
+
+  @Override
+  public RESULT visitCaseWhen(CONTEXT context, CaseWhenEval evalNode, Stack<EvalNode> stack) {
+    RESULT result = null;
+    stack.push(evalNode);
+    for (CaseWhenEval.IfThenEval ifThenEval : evalNode.getIfThenEvals()) {
+      result = visitIfThen(context, ifThenEval, stack);
+    }
+    if (evalNode.hasElse()) {
+      result = visitChild(context, evalNode.getElse(), stack);
+    }
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitIfThen(CONTEXT context, CaseWhenEval.IfThenEval evalNode, Stack<EvalNode> stack) {
+    RESULT result;
+    stack.push(evalNode);
+    result = visitChild(context, evalNode.getConditionExpr(), stack);
+    visitChild(context, evalNode.getResultExpr(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitInPredicate(CONTEXT context, InEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitLike(CONTEXT context, LikePredicateEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitSimilarTo(CONTEXT context, SimilarToPredicateEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitRegex(CONTEXT context, RegexPredicateEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitConcatenate(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultBinaryEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitFuncCall(CONTEXT context, GeneralFunctionEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultFunctionEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitAggrFuncCall(CONTEXT context, AggregationFunctionCallEval evalNode, Stack<EvalNode> stack) {
+    return visitDefaultFunctionEval(context, stack, evalNode);
+  }
+
+  @Override
+  public RESULT visitSigned(CONTEXT context, SignedEval signedEval, Stack<EvalNode> stack) {
+    RESULT result;
+    stack.push(signedEval);
+    result = visitChild(context, signedEval.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitCast(CONTEXT context, CastEval signedEval, Stack<EvalNode> stack) {
+    RESULT result;
+    stack.push(signedEval);
+    result = visitChild(context, signedEval.getOperand(), stack);
+    stack.pop();
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java
new file mode 100644
index 0000000..0f30ac9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java
@@ -0,0 +1,219 @@
+/**
+ * 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.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.Tuple;
+
+public class BetweenPredicateEval extends EvalNode {
+  private static final TajoDataTypes.DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN);
+  @Expose private boolean not;
+  @Expose private boolean symmetric;
+  @Expose private EvalNode predicand;
+  @Expose private EvalNode begin;
+  @Expose private EvalNode end;
+
+  private Checker checker;
+
+  public BetweenPredicateEval(boolean not, boolean symmetric, EvalNode predicand, EvalNode begin, EvalNode end) {
+    super(EvalType.BETWEEN);
+    this.not = not;
+    this.symmetric = symmetric;
+    this.predicand = predicand;
+    this.begin = begin;
+    this.end = end;
+  }
+
+  public EvalNode getPredicand() {
+    return predicand;
+  }
+
+  public EvalNode getBegin() {
+    return begin;
+  }
+
+  public EvalNode getEnd() {
+    return end;
+  }
+
+  private static interface Checker {
+    Datum eval(Schema schema, Tuple param);
+  }
+
+  private static class ConstantChecker implements Checker {
+    EvalNode predicand;
+    Datum begin;
+    Datum end;
+    private boolean not;
+
+    private ConstantChecker(boolean not, EvalNode predicand, Datum begin, Datum end) {
+      this.predicand = predicand;
+      this.not = not;
+      if (begin.compareTo(end) > 0) {
+        this.begin = end;
+        this.end = begin;
+      } else {
+        this.begin = begin;
+        this.end = end;
+      }
+    }
+
+    @Override
+    public Datum eval(Schema schema, Tuple param) {
+      Datum predicandValue = predicand.eval(schema, param);
+
+      if (!predicandValue.isNull()) {
+        return DatumFactory.createBool(not ^ (predicandValue.greaterThanEqual(begin).asBool()
+                && predicandValue.lessThanEqual(end).asBool()));
+      } else {
+        return NullDatum.get();
+      }
+    }
+  }
+
+  private static class AsymmetricChecker implements Checker {
+    EvalNode predicand;
+    EvalNode begin;
+    EvalNode end;
+    private boolean not;
+
+    private AsymmetricChecker(boolean not, EvalNode predicand, EvalNode begin, EvalNode end) {
+      this.not = not;
+      this.predicand = predicand;
+      this.begin = begin;
+      this.end = end;
+    }
+
+    @Override
+    public Datum eval(Schema schema, Tuple param) {
+      Datum predicandValue = predicand.eval(schema, param);
+      Datum beginValue = begin.eval(schema, param);
+      Datum endValue = end.eval(schema, param);
+
+      if (!(predicandValue.isNull() || beginValue.isNull() || endValue.isNull())) {
+        return
+            DatumFactory.createBool(not ^ (predicandValue.greaterThanEqual(beginValue).asBool()
+                && predicandValue.lessThanEqual(endValue).asBool()));
+      } else {
+        return NullDatum.get();
+      }
+    }
+  }
+
+  private static class SymmetricChecker implements Checker {
+    boolean not;
+    EvalNode predicand;
+    EvalNode begin;
+    EvalNode end;
+
+    SymmetricChecker(boolean not, EvalNode predicand, EvalNode begin, EvalNode end) {
+      this.not = not;
+      this.predicand = predicand;
+      this.begin = begin;
+      this.end = end;
+    }
+
+    @Override
+    public Datum eval(Schema schema, Tuple param) {
+      Datum predicandValue = predicand.eval(schema, param);
+      Datum beginValue = begin.eval(schema, param);
+      Datum endValue = end.eval(schema, param);
+
+      if (!(predicandValue.isNull()|| beginValue.isNull() || endValue.isNull())) {
+        return DatumFactory.createBool( not ^
+            (predicandValue.greaterThanEqual(beginValue).asBool() && predicandValue.lessThanEqual(endValue).asBool()) ||
+            (predicandValue.lessThanEqual(beginValue).asBool() && predicandValue.greaterThanEqual(endValue).asBool())
+        );
+      } else {
+        return NullDatum.get();
+      }
+    }
+  }
+
+  @Override
+  public TajoDataTypes.DataType getValueType() {
+    return RES_TYPE;
+  }
+
+  @Override
+  public String getName() {
+    return "between";
+  }
+
+  @Override
+  public String toString() {
+    return predicand + " BETWEEN " + (symmetric ? "SYMMETRIC" : "ASYMMETRIC") + " " + begin + " AND " + end;
+  }
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    if (checker == null) {
+      if (begin.getType() == EvalType.CONST && end.getType() == EvalType.CONST) {
+        Datum beginValue = ((ConstEval)begin).getValue();
+        Datum endValue = ((ConstEval)end).getValue();
+
+        if (symmetric || beginValue.compareTo(endValue) <= 0) {
+          checker = new ConstantChecker(not, predicand, beginValue, endValue);
+        } else {
+          checker = new AsymmetricChecker(not, predicand, begin, end);
+        }
+      } else {
+        if (symmetric) {
+          checker = new SymmetricChecker(not, predicand, begin, end);
+        } else {
+          checker = new AsymmetricChecker(not, predicand, begin, end);
+        }
+      }
+    }
+
+    return checker.eval(schema, tuple);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof BetweenPredicateEval) {
+      BetweenPredicateEval another = (BetweenPredicateEval) obj;
+      return not == another.not && symmetric == another.symmetric && predicand.equals(another.predicand) &&
+          begin.equals(another.begin) && end.equals(another.end);
+    }
+    return false;
+  }
+
+  @Deprecated
+  public void preOrder(EvalNodeVisitor visitor) {
+    visitor.visit(this);
+    predicand.preOrder(visitor);
+    begin.preOrder(visitor);
+    end.preOrder(visitor);
+  }
+
+  @Deprecated
+  public void postOrder(EvalNodeVisitor visitor) {
+    predicand.postOrder(visitor);
+    begin.postOrder(visitor);
+    end.postOrder(visitor);
+    visitor.visit(this);
+  }
+}
\ No newline at end of file


[33/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java
new file mode 100644
index 0000000..b8a9680
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java
@@ -0,0 +1,69 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
+
+public class SelectionNode extends UnaryNode implements Cloneable {
+	@Expose private EvalNode qual;
+
+  public SelectionNode(int pid) {
+    super(pid, NodeType.SELECTION);
+  }
+
+	public EvalNode getQual() {
+		return this.qual;
+	}
+
+	public void setQual(EvalNode qual) {
+		this.qual = qual;
+	}
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    planStr.addExplan("Search Cond: " + getQual());
+    return planStr;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof SelectionNode) {
+      SelectionNode other = (SelectionNode) obj;
+      return super.equals(other) 
+          && this.qual.equals(other.qual);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    SelectionNode selNode = (SelectionNode) super.clone();
+    selNode.qual = (EvalNode) this.qual.clone();
+    
+    return selNode;
+  }
+
+  public String toString() {
+    return "Selection (filter=" + qual + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java
new file mode 100644
index 0000000..c7ea454
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java
@@ -0,0 +1,104 @@
+/**
+ * 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.planner.logical;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.util.TUtil;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.NONE_SHUFFLE;
+
+/**
+ * ShuffeFileWriteNode is an expression for an intermediate data materialization step.
+ */
+public class ShuffleFileWriteNode extends PersistentStoreNode implements Cloneable {
+  @Expose private TajoWorkerProtocol.ShuffleType shuffleType = NONE_SHUFFLE;
+  @Expose private int numOutputs;
+  @Expose private Column [] shuffleKeys;
+
+  public ShuffleFileWriteNode(int pid) {
+    super(pid, NodeType.STORE);
+  }
+    
+  public final int getNumOutputs() {
+    return this.numOutputs;
+  }
+  
+  public final boolean hasShuffleKeys() {
+    return this.shuffleKeys != null;
+  }
+  
+  public final Column [] getShuffleKeys() {
+    return shuffleKeys;
+  }
+  
+  public final void setShuffle(TajoWorkerProtocol.ShuffleType type, Column[] keys, int numPartitions) {
+    Preconditions.checkArgument(keys.length >= 0, 
+        "At least one partition key must be specified.");
+    // In outer join, zero can be passed into this value because of empty tables.
+    // So, we should allow zero.
+    Preconditions.checkArgument(numPartitions >= 0,
+        "The number of partitions must be positive: %s", numPartitions);
+
+    this.shuffleType = type;
+    this.shuffleKeys = keys;
+    this.numOutputs = numPartitions;
+  }
+
+  public TajoWorkerProtocol.ShuffleType getShuffleType() {
+    return this.shuffleType;
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof ShuffleFileWriteNode) {
+      ShuffleFileWriteNode other = (ShuffleFileWriteNode) obj;
+      boolean eq = super.equals(other);
+      eq = eq && this.numOutputs == other.numOutputs;
+      eq = eq && TUtil.checkEquals(shuffleKeys, other.shuffleKeys);
+      return eq;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    ShuffleFileWriteNode store = (ShuffleFileWriteNode) super.clone();
+    store.numOutputs = numOutputs;
+    store.shuffleKeys = shuffleKeys != null ? shuffleKeys.clone() : null;
+    return store;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Shuffle Write (type=" + shuffleType.name().toLowerCase());
+    if (storageType != null) {
+      sb.append(", storage="+ storageType.name());
+    }
+    sb.append(", part number=").append(numOutputs);
+    if (shuffleKeys != null) {
+      sb.append(", keys: ").append(TUtil.arrayToString(shuffleKeys));
+    }
+    sb.append(")");
+    
+    return sb.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java
new file mode 100644
index 0000000..a732710
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java
@@ -0,0 +1,94 @@
+/**
+ * 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.planner.logical;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.util.TUtil;
+
+public final class SortNode extends UnaryNode implements Cloneable {
+	@Expose private SortSpec [] sortKeys;
+
+  public SortNode(int pid) {
+    super(pid, NodeType.SORT);
+  }
+
+  public void setSortSpecs(SortSpec[] sortSpecs) {
+    Preconditions.checkArgument(sortSpecs.length > 0, "At least one sort key must be specified");
+    this.sortKeys = sortSpecs;
+  }
+  
+  public SortSpec[] getSortKeys() {
+    return this.sortKeys;
+  }
+  
+  @Override 
+  public boolean equals(Object obj) {
+    if (obj instanceof SortNode) {
+      SortNode other = (SortNode) obj;
+      boolean eq = super.equals(other);
+      eq = eq && TUtil.checkEquals(sortKeys, other.sortKeys);
+      return eq;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    SortNode sort = (SortNode) super.clone();
+    sort.sortKeys = sortKeys.clone();
+    
+    return sort;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    StringBuilder sb = new StringBuilder("Sort Keys: ");
+    for (int i = 0; i < sortKeys.length; i++) {
+      sb.append(sortKeys[i].getSortKey().getSimpleName()).append(" ")
+          .append(sortKeys[i].isAscending() ? "asc" : "desc");
+      if( i < sortKeys.length - 1) {
+        sb.append(",");
+      }
+    }
+    planStr.addExplan(sb.toString());
+    return planStr;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Sort [key= ");
+    for (int i = 0; i < sortKeys.length; i++) {    
+      sb.append(sortKeys[i].getSortKey().getQualifiedName()).append(" ")
+          .append(sortKeys[i].isAscending() ? "asc" : "desc");
+      if(i < sortKeys.length - 1) {
+        sb.append(",");
+      }
+    }
+    sb.append("]");
+
+    sb.append("\n\"out schema: " + getOutSchema()
+        + "\n\"in schema: " + getInSchema());
+    return sb.toString()+"\n"
+        + getChild().toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
new file mode 100644
index 0000000..b0f7b7a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
@@ -0,0 +1,100 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.util.TUtil;
+
+public class StoreTableNode extends PersistentStoreNode implements Cloneable {
+  @Expose protected String tableName;
+  @Expose private PartitionMethodDesc partitionDesc;
+
+  public StoreTableNode(int pid) {
+    super(pid, NodeType.STORE);
+  }
+
+  protected StoreTableNode(int pid, NodeType nodeType) {
+    super(pid, nodeType);
+  }
+
+  public boolean hasTargetTable() {
+    return tableName != null;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public final String getTableName() {
+    return this.tableName;
+  }
+
+  public boolean hasPartition() {
+    return this.partitionDesc != null;
+  }
+
+  public PartitionMethodDesc getPartitionMethod() {
+    return partitionDesc;
+  }
+
+  public void setPartitionMethod(PartitionMethodDesc partitionDesc) {
+    this.partitionDesc = partitionDesc;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    planStr.appendTitle(" into ").appendTitle(tableName);
+    planStr.addExplan("Store type: " + storageType);
+
+    return planStr;
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof StoreTableNode) {
+      StoreTableNode other = (StoreTableNode) obj;
+      boolean eq = super.equals(other);
+      eq = eq && this.tableName.equals(other.tableName);
+      eq = eq && TUtil.checkEquals(partitionDesc, other.partitionDesc);
+      return eq;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    StoreTableNode store = (StoreTableNode) super.clone();
+    store.tableName = tableName;
+    store.partitionDesc = partitionDesc != null ? (PartitionMethodDesc) partitionDesc.clone() : null;
+    return store;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Store Table (table=").append(tableName);
+    if (storageType != null) {
+      sb.append(", storage="+ storageType.name());
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
new file mode 100644
index 0000000..4d0090b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
@@ -0,0 +1,181 @@
+/**
+ * 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.planner.logical;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.engine.utils.SchemaUtil;
+
+public class TableSubQueryNode extends RelationNode implements Projectable {
+  @Expose private String tableName;
+  @Expose private LogicalNode subQuery;
+  @Expose private Target [] targets; // unused
+
+  public TableSubQueryNode(int pid) {
+    super(pid, NodeType.TABLE_SUBQUERY);
+  }
+
+  public void init(String tableName, LogicalNode subQuery) {
+    this.tableName = tableName;
+    if (subQuery != null) {
+      this.subQuery = subQuery;
+      setOutSchema(SchemaUtil.clone(this.subQuery.getOutSchema()));
+      setInSchema(SchemaUtil.clone(this.subQuery.getOutSchema()));
+      getInSchema().setQualifier(this.tableName);
+      getOutSchema().setQualifier(this.tableName);
+    }
+  }
+
+  @Override
+  public boolean hasAlias() {
+    return false;
+  }
+
+  @Override
+  public String getAlias() {
+    return null;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public String getCanonicalName() {
+    return tableName;
+  }
+
+  @Override
+  public Schema getTableSchema() {
+    // an output schema can be determined by targets. So, an input schema of
+    // TableSubQueryNode is only eligible for table schema.
+    //
+    // TODO - but, a derived table can have column alias. For that, we should improve here.
+    //
+    // example) select * from (select col1, col2, col3 from t1) view (c1, c2);
+
+    return getInSchema();
+  }
+
+  public void setSubQuery(LogicalNode node) {
+    this.subQuery = node;
+    setInSchema(SchemaUtil.clone(this.subQuery.getOutSchema()));
+    getInSchema().setQualifier(this.tableName);
+    if (hasTargets()) {
+      setOutSchema(PlannerUtil.targetToSchema(targets));
+    } else {
+      setOutSchema(SchemaUtil.clone(this.subQuery.getOutSchema()));
+    }
+    getOutSchema().setQualifier(this.tableName);
+  }
+
+  public LogicalNode getSubQuery() {
+    return subQuery;
+  }
+
+  @Override
+  public boolean hasTargets() {
+    return targets != null;
+  }
+
+  @Override
+  public void setTargets(Target[] targets) {
+    this.targets = targets;
+    setOutSchema(PlannerUtil.targetToSchema(targets));
+  }
+
+  @Override
+  public Target[] getTargets() {
+    return targets;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    planStr.appendTitle(" as ").appendTitle(tableName);
+
+    if (hasTargets()) {
+      StringBuilder sb = new StringBuilder("Targets: ");
+      for (int i = 0; i < targets.length; i++) {
+        sb.append(targets[i]);
+        if( i < targets.length - 1) {
+          sb.append(", ");
+        }
+      }
+      planStr.addExplan(sb.toString());
+      if (getOutSchema() != null) {
+        planStr.addExplan("out schema: " + getOutSchema().toString());
+      }
+      if (getInSchema() != null) {
+        planStr.addExplan("in  schema: " + getInSchema().toString());
+      }
+    }
+
+    return planStr;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(tableName, subQuery);
+  }
+
+  @Override
+  public boolean equals(Object object) {
+    if (object instanceof TableSubQueryNode) {
+      TableSubQueryNode another = (TableSubQueryNode) object;
+      return tableName.equals(another.tableName) && subQuery.equals(another.subQuery);
+    }
+
+    return false;
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    TableSubQueryNode newTableSubQueryNode = (TableSubQueryNode) super.clone();
+    newTableSubQueryNode.tableName = tableName;
+    newTableSubQueryNode.subQuery = (LogicalNode) subQuery.clone();
+    if (hasTargets()) {
+      newTableSubQueryNode.targets = new Target[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        newTableSubQueryNode.targets[i] = (Target) targets[i].clone();
+      }
+    }
+    return newTableSubQueryNode;
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+    subQuery.preOrder(visitor);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    subQuery.preOrder(visitor);
+    visitor.visit(this);
+  }
+
+  public String toString() {
+    return "Inline view (name=" + tableName + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java
new file mode 100644
index 0000000..0b06e9e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java
@@ -0,0 +1,69 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.util.TUtil;
+
+
+public abstract class UnaryNode extends LogicalNode implements Cloneable {
+	@Expose LogicalNode child;
+	
+	/**
+	 * @param type
+	 */
+	public UnaryNode(int pid, NodeType type) {
+		super(pid, type);
+	}
+	
+	public void setChild(LogicalNode subNode) {
+		this.child = subNode;
+	}
+	
+	public <T extends LogicalNode> T getChild() {
+		return (T) this.child;
+	}
+
+  @Override
+  public boolean deepEquals(Object o) {
+    if (o instanceof UnaryNode) {
+      UnaryNode u = (UnaryNode) o;
+      return equals(o) && TUtil.checkEquals(child, u.child);
+    }
+    return false;
+  }
+	
+	@Override
+  public Object clone() throws CloneNotSupportedException {
+	  UnaryNode unary = (UnaryNode) super.clone();
+	  unary.child = (LogicalNode) (child == null ? null : child.clone());
+	  
+	  return unary;
+	}
+	
+	public void preOrder(LogicalNodeVisitor visitor) {
+	  visitor.visit(this);
+	  child.preOrder(visitor);
+  }
+	
+	public void postOrder(LogicalNodeVisitor visitor) {
+	  child.postOrder(visitor);
+	  visitor.visit(this);
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java
new file mode 100644
index 0000000..49183d0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java
@@ -0,0 +1,37 @@
+/**
+ * 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.planner.logical;
+
+import org.apache.tajo.engine.planner.PlanString;
+
+public class UnionNode extends BinaryNode {
+
+  public UnionNode(int pid) {
+    super(pid, NodeType.UNION);
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    return planStr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/Edge.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/Edge.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/Edge.java
new file mode 100644
index 0000000..c8bca03
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/Edge.java
@@ -0,0 +1,50 @@
+/**
+ * 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.planner.logical.join;
+
+import org.apache.tajo.engine.eval.EvalNode;
+
+public class Edge {
+  private String src;
+  private String target;
+  private EvalNode joinQual;
+
+  public Edge(String src, String target, EvalNode joinQual) {
+    this.src = src;
+    this.target = target;
+    this.joinQual = joinQual;
+  }
+
+  public String getSrc() {
+    return this.src;
+  }
+
+  public String getTarget() {
+    return this.target;
+  }
+
+  public EvalNode getJoinQual() {
+    return this.joinQual;
+  }
+
+  @Override
+  public String toString() {
+    return "(" + src + "=> " + target + ", " + joinQual + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/FoundJoinOrder.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/FoundJoinOrder.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/FoundJoinOrder.java
new file mode 100644
index 0000000..5ae34f7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/FoundJoinOrder.java
@@ -0,0 +1,47 @@
+/**
+ * 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.planner.logical.join;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+
+/**
+ * It contains the result of join enumeration.
+ */
+@InterfaceStability.Evolving
+public class FoundJoinOrder {
+  private JoinNode joinNode;
+  private double cost;
+
+  public FoundJoinOrder(JoinNode joinNode, double cost) {
+    this.joinNode = joinNode;
+    this.cost = cost;
+  }
+
+  /**
+   * @return a ordered join operators
+   */
+  public JoinNode getOrderedJoin() {
+    return this.joinNode;
+  }
+
+  public double getCost() {
+    return cost;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/GreedyHeuristicJoinOrderAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/GreedyHeuristicJoinOrderAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/GreedyHeuristicJoinOrderAlgorithm.java
new file mode 100644
index 0000000..f2bcd77
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/GreedyHeuristicJoinOrderAlgorithm.java
@@ -0,0 +1,262 @@
+/**
+ * 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.planner.logical.join;
+
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.AlgebraicUtil;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.utils.SchemaUtil;
+
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+/**
+ * This is a greedy heuristic algorithm to find a bushy join tree. This algorithm finds
+ * the best join order with join conditions and pushed-down join conditions to
+ * all join operators.
+ */
+public class GreedyHeuristicJoinOrderAlgorithm implements JoinOrderAlgorithm {
+  public static double DEFAULT_SELECTION_FACTOR = 0.1;
+
+  @Override
+  public FoundJoinOrder findBestOrder(LogicalPlan plan, LogicalPlan.QueryBlock block, JoinGraph joinGraph,
+                                      Set<String> relationsWithoutQual) throws PlanningException {
+
+    // Setup a remain relation set to be joined
+    // Why we should use LinkedHashSet? - it should keep the deterministic for the order of joins.
+    // Otherwise, join orders can be different even if join costs are the same to each other.
+    Set<LogicalNode> remainRelations = new LinkedHashSet<LogicalNode>();
+    for (RelationNode relation : block.getRelations()) {
+      remainRelations.add(relation);
+    }
+
+    LogicalNode latestJoin;
+    JoinEdge bestPair;
+
+    while (remainRelations.size() > 1) {
+      // Find the best join pair among all joinable operators in candidate set.
+      bestPair = getBestPair(plan, joinGraph, remainRelations);
+
+      remainRelations.remove(bestPair.getLeftRelation()); // remainRels = remainRels \ Ti
+      remainRelations.remove(bestPair.getRightRelation()); // remainRels = remainRels \ Tj
+
+      latestJoin = createJoinNode(plan, bestPair);
+      remainRelations.add(latestJoin);
+
+      // all logical nodes should be registered to corresponding blocks
+      block.registerNode(latestJoin);
+    }
+
+    JoinNode joinTree = (JoinNode) remainRelations.iterator().next();
+    // all generated nodes should be registered to corresponding blocks
+    block.registerNode(joinTree);
+    return new FoundJoinOrder(joinTree, getCost(joinTree));
+  }
+
+  private static JoinNode createJoinNode(LogicalPlan plan, JoinEdge joinEdge) {
+    LogicalNode left = joinEdge.getLeftRelation();
+    LogicalNode right = joinEdge.getRightRelation();
+
+    JoinNode joinNode = plan.createNode(JoinNode.class);
+
+    if (PlannerUtil.isCommutativeJoin(joinEdge.getJoinType())) {
+      // if only one operator is relation
+      if ((left instanceof RelationNode) && !(right instanceof RelationNode)) {
+        // for left deep
+        joinNode.init(joinEdge.getJoinType(), right, left);
+      } else {
+        // if both operators are relation or if both are relations
+        // we don't need to concern the left-right position.
+        joinNode.init(joinEdge.getJoinType(), left, right);
+      }
+    } else {
+      joinNode.init(joinEdge.getJoinType(), left, right);
+    }
+
+    Schema mergedSchema = SchemaUtil.merge(joinNode.getLeftChild().getOutSchema(),
+        joinNode.getRightChild().getOutSchema());
+    joinNode.setInSchema(mergedSchema);
+    joinNode.setOutSchema(mergedSchema);
+    if (joinEdge.hasJoinQual()) {
+      joinNode.setJoinQual(AlgebraicUtil.createSingletonExprFromCNF(joinEdge.getJoinQual()));
+    }
+    return joinNode;
+  }
+
+  /**
+   * Find the best join pair among all joinable operators in candidate set.
+   *
+   * @param plan a logical plan
+   * @param graph a join graph which consists of vertices and edges, where vertex is relation and
+   *              each edge is join condition.
+   * @param candidateSet candidate operators to be joined.
+   * @return The best join pair among them
+   * @throws PlanningException
+   */
+  private JoinEdge getBestPair(LogicalPlan plan, JoinGraph graph, Set<LogicalNode> candidateSet)
+      throws PlanningException {
+    double minCost = Double.MAX_VALUE;
+    JoinEdge bestJoin = null;
+
+    double minNonCrossJoinCost = Double.MAX_VALUE;
+    JoinEdge bestNonCrossJoin = null;
+
+    for (LogicalNode outer : candidateSet) {
+      for (LogicalNode inner : candidateSet) {
+        if (outer.equals(inner)) {
+          continue;
+        }
+
+        JoinEdge foundJoin = findJoin(plan, graph, outer, inner);
+        if (foundJoin == null) {
+          continue;
+        }
+        double cost = getCost(foundJoin);
+
+        if (cost < minCost) {
+          minCost = cost;
+          bestJoin = foundJoin;
+        }
+
+        // Keep the min cost join
+        // But, if there exists a qualified join, the qualified join must be chosen
+        // rather than cross join regardless of cost.
+        if (foundJoin.hasJoinQual()) {
+          if (cost < minNonCrossJoinCost) {
+            minNonCrossJoinCost = cost;
+            bestNonCrossJoin = foundJoin;
+          }
+        }
+      }
+    }
+
+    if (bestNonCrossJoin != null) {
+      return bestNonCrossJoin;
+    } else {
+      return bestJoin;
+    }
+  }
+
+  /**
+   * Find a join between two logical operator trees
+   *
+   * @return If there is no join condition between two relation, it returns NULL value.
+   */
+  private static JoinEdge findJoin(LogicalPlan plan, JoinGraph graph, LogicalNode outer, LogicalNode inner)
+      throws PlanningException {
+    JoinEdge foundJoinEdge = null;
+
+    for (String outerName : PlannerUtil.getRelationLineageWithinQueryBlock(plan, outer)) {
+      for (String innerName : PlannerUtil.getRelationLineageWithinQueryBlock(plan, inner)) {
+
+        // Find all joins between two relations and merge them into one join if possible
+        if (graph.hasEdge(outerName, innerName)) {
+          JoinEdge existJoinEdge = graph.getEdge(outerName, innerName);
+          if (foundJoinEdge == null) {
+            foundJoinEdge = new JoinEdge(existJoinEdge.getJoinType(), outer, inner,
+                existJoinEdge.getJoinQual());
+          } else {
+            foundJoinEdge.addJoinQual(AlgebraicUtil.createSingletonExprFromCNF(
+                existJoinEdge.getJoinQual()));
+          }
+        }
+      }
+    }
+
+    if (foundJoinEdge == null) {
+      foundJoinEdge = new JoinEdge(JoinType.CROSS, outer, inner);
+    }
+
+    return foundJoinEdge;
+  }
+
+  /**
+   * Getting a cost of one join
+   * @param joinEdge
+   * @return
+   */
+  public static double getCost(JoinEdge joinEdge) {
+    double filterFactor = 1;
+    if (joinEdge.hasJoinQual()) {
+      // TODO - should consider join type
+      // TODO - should statistic information obtained from query history
+      filterFactor = filterFactor * Math.pow(DEFAULT_SELECTION_FACTOR, joinEdge.getJoinQual().length);
+      return getCost(joinEdge.getLeftRelation()) * getCost(joinEdge.getRightRelation()) * filterFactor;
+    } else {
+      // make cost bigger if cross join
+      return Math.pow(getCost(joinEdge.getLeftRelation()) * getCost(joinEdge.getRightRelation()), 2);
+    }
+  }
+
+  // TODO - costs of other operator operators (e.g., group-by and sort) should be computed in proper manners.
+  public static double getCost(LogicalNode node) {
+    switch (node.getType()) {
+
+    case PROJECTION:
+      ProjectionNode projectionNode = (ProjectionNode) node;
+      return getCost(projectionNode.getChild());
+
+    case JOIN:
+      JoinNode joinNode = (JoinNode) node;
+      double filterFactor = 1;
+      if (joinNode.hasJoinQual()) {
+        filterFactor = Math.pow(DEFAULT_SELECTION_FACTOR,
+            AlgebraicUtil.toConjunctiveNormalFormArray(joinNode.getJoinQual()).length);
+        return getCost(joinNode.getLeftChild()) * getCost(joinNode.getRightChild()) * filterFactor;
+      } else {
+        return Math.pow(getCost(joinNode.getLeftChild()) * getCost(joinNode.getRightChild()), 2);
+      }
+
+    case SELECTION:
+      SelectionNode selectionNode = (SelectionNode) node;
+      return getCost(selectionNode.getChild()) *
+          Math.pow(DEFAULT_SELECTION_FACTOR, AlgebraicUtil.toConjunctiveNormalFormArray(selectionNode.getQual()).length);
+
+    case TABLE_SUBQUERY:
+      TableSubQueryNode subQueryNode = (TableSubQueryNode) node;
+      return getCost(subQueryNode.getSubQuery());
+
+    case SCAN:
+      ScanNode scanNode = (ScanNode) node;
+      if (scanNode.getTableDesc().getStats() != null) {
+        double cost = ((ScanNode)node).getTableDesc().getStats().getNumBytes();
+        return cost;
+      } else {
+        return Long.MAX_VALUE;
+      }
+
+    case UNION:
+      UnionNode unionNode = (UnionNode) node;
+      return getCost(unionNode.getLeftChild()) + getCost(unionNode.getRightChild());
+
+    case EXCEPT:
+    case INTERSECT:
+      throw new UnsupportedOperationException("getCost() does not support EXCEPT or INTERSECT yet");
+
+    default:
+      // all binary operators (join, union, except, and intersect) are handled in the above cases.
+      // So, we need to handle only unary nodes in default.
+      return getCost(((UnaryNode) node).getChild());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinEdge.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinEdge.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinEdge.java
new file mode 100644
index 0000000..e5c29f0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinEdge.java
@@ -0,0 +1,76 @@
+/**
+ * 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.planner.logical.join;
+
+import com.google.common.collect.Sets;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.RelationNode;
+import org.apache.tajo.util.TUtil;
+
+import java.util.Collections;
+import java.util.Set;
+
+public class JoinEdge {
+  private final JoinType joinType;
+  private final LogicalNode leftRelation;
+  private final LogicalNode rightRelation;
+  private final Set<EvalNode> joinQual = Sets.newHashSet();
+
+  public JoinEdge(JoinType joinType, LogicalNode leftRelation, LogicalNode rightRelation) {
+    this.joinType = joinType;
+    this.leftRelation = leftRelation;
+    this.rightRelation = rightRelation;
+  }
+
+  public JoinEdge(JoinType joinType, LogicalNode leftRelation, LogicalNode rightRelation,
+                  EvalNode ... condition) {
+    this(joinType, leftRelation, rightRelation);
+    Collections.addAll(joinQual, condition);
+  }
+
+  public JoinType getJoinType() {
+    return joinType;
+  }
+
+  public LogicalNode getLeftRelation() {
+    return leftRelation;
+  }
+
+  public LogicalNode getRightRelation() {
+    return rightRelation;
+  }
+
+  public boolean hasJoinQual() {
+    return joinQual.size() > 0;
+  }
+
+  public void addJoinQual(EvalNode joinQual) {
+    this.joinQual.add(joinQual);
+  }
+
+  public EvalNode [] getJoinQual() {
+    return joinQual.toArray(new EvalNode[joinQual.size()]);
+  }
+
+  public String toString() {
+    return leftRelation + " " + joinType + " " + rightRelation + " ON " + TUtil.collectionToString(joinQual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinGraph.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinGraph.java
new file mode 100644
index 0000000..77e03ea
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinGraph.java
@@ -0,0 +1,115 @@
+/**
+ * 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.planner.logical.join;
+
+import com.google.common.collect.Sets;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.eval.AlgebraicUtil;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.eval.EvalTreeUtil;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.NamedExprsManager;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.graph.SimpleUndirectedGraph;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+
+import java.util.Collection;
+import java.util.Set;
+
+public class JoinGraph extends SimpleUndirectedGraph<String, JoinEdge> {
+
+  private String [] guessRelationsFromJoinQual(LogicalPlan.QueryBlock block, EvalNode joinCondition)
+      throws PlanningException {
+
+    // Note that we can guarantee that each join qual used here is a singleton.
+    // This is because we use dissect a join qual into conjunctive normal forms.
+    // In other words, each join qual has a form 'col1 = col2'.
+    Column leftExpr = EvalTreeUtil.findAllColumnRefs(joinCondition.getLeftExpr()).get(0);
+    Column rightExpr = EvalTreeUtil.findAllColumnRefs(joinCondition.getRightExpr()).get(0);
+
+    // 0 - left table, 1 - right table
+    String [] relationNames = new String[2];
+
+    NamedExprsManager namedExprsMgr = block.getNamedExprsManager();
+    if (leftExpr.hasQualifier()) {
+      relationNames[0] = leftExpr.getQualifier();
+    } else {
+      if (namedExprsMgr.isAliasedName(leftExpr.getSimpleName())) {
+        String columnName = namedExprsMgr.getOriginalName(leftExpr.getSimpleName());
+        String qualifier = CatalogUtil.extractQualifier(columnName);
+        relationNames[0] = qualifier;
+      } else {
+        throw new PlanningException("Cannot expect a referenced relation: " + leftExpr);
+      }
+    }
+
+    if (rightExpr.hasQualifier()) {
+      relationNames[1] = rightExpr.getQualifier();
+    } else {
+      if (namedExprsMgr.isAliasedName(rightExpr.getSimpleName())) {
+        String columnName = namedExprsMgr.getOriginalName(rightExpr.getSimpleName());
+        String qualifier = CatalogUtil.extractQualifier(columnName);
+        relationNames[1] = qualifier;
+      } else {
+        throw new PlanningException("Cannot expect a referenced relation: " + rightExpr);
+      }
+    }
+
+    return relationNames;
+  }
+  public Collection<EvalNode> addJoin(LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                      JoinNode joinNode) throws PlanningException {
+    Set<EvalNode> cnf = Sets.newHashSet(AlgebraicUtil.toConjunctiveNormalFormArray(joinNode.getJoinQual()));
+    Set<EvalNode> nonJoinQuals = Sets.newHashSet();
+    for (EvalNode singleQual : cnf) {
+      if (EvalTreeUtil.isJoinQual(singleQual, true)) {
+
+        String [] relations = guessRelationsFromJoinQual(block, singleQual);
+        String leftExprRelName = relations[0];
+        String rightExprRelName = relations[1];
+
+        Collection<String> leftLineage = PlannerUtil.getRelationLineageWithinQueryBlock(plan, joinNode.getLeftChild());
+
+        boolean isLeftExprForLeftTable = leftLineage.contains(leftExprRelName);
+        JoinEdge edge;
+        edge = getEdge(leftExprRelName, rightExprRelName);
+
+        if (edge != null) {
+          edge.addJoinQual(singleQual);
+        } else {
+          if (isLeftExprForLeftTable) {
+            edge = new JoinEdge(joinNode.getJoinType(),
+                block.getRelation(leftExprRelName), block.getRelation(rightExprRelName), singleQual);
+            addEdge(leftExprRelName, rightExprRelName, edge);
+          } else {
+            edge = new JoinEdge(joinNode.getJoinType(),
+                block.getRelation(rightExprRelName), block.getRelation(leftExprRelName), singleQual);
+            addEdge(rightExprRelName, leftExprRelName, edge);
+          }
+        }
+      } else {
+        nonJoinQuals.add(singleQual);
+      }
+    }
+    cnf.retainAll(nonJoinQuals);
+    return cnf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinOrderAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinOrderAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinOrderAlgorithm.java
new file mode 100644
index 0000000..eafa671
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinOrderAlgorithm.java
@@ -0,0 +1,46 @@
+/**
+ * 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.planner.logical.join;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.PlanningException;
+
+import java.util.Set;
+
+/**
+ * An interface for join order algorithms
+ */
+@InterfaceStability.Evolving
+public interface JoinOrderAlgorithm {
+
+  /**
+   *
+   * @param plan
+   * @param block
+   * @param joinGraph A join graph represents join conditions and their connections among relations.
+   *                  Given a graph, each vertex represents a relation, and each edge contains a join condition.
+   *                  A join graph does not contain relations that do not have any corresponding join condition.
+   * @param relationsWithoutQual The names of relations that do not have any corresponding join condition.
+   * @return
+   * @throws PlanningException
+   */
+  FoundJoinOrder findBestOrder(LogicalPlan plan, LogicalPlan.QueryBlock block, JoinGraph joinGraph,
+                               Set<String> relationsWithoutQual) throws PlanningException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java
new file mode 100644
index 0000000..208973e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java
@@ -0,0 +1,69 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.AggregationFunctionCallEval;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public abstract class AggregationExec extends UnaryPhysicalExec {
+  protected GroupbyNode plan;
+
+  protected final int groupingKeyNum;
+  protected int groupingKeyIds[];
+  protected final int aggFunctionsNum;
+  protected final AggregationFunctionCallEval aggFunctions[];
+
+  protected Schema evalSchema;
+
+  public AggregationExec(final TaskAttemptContext context, GroupbyNode plan,
+                         PhysicalExec child) throws IOException {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+    this.plan = plan;
+
+    evalSchema = plan.getOutSchema();
+
+    final Column [] keyColumns = plan.getGroupingColumns();
+    groupingKeyNum = keyColumns.length;
+    groupingKeyIds = new int[groupingKeyNum];
+    Column col;
+    for (int idx = 0; idx < plan.getGroupingColumns().length; idx++) {
+      col = keyColumns[idx];
+      groupingKeyIds[idx] = inSchema.getColumnId(col.getQualifiedName());
+    }
+
+    if (plan.hasAggFunctions()) {
+      aggFunctions = plan.getAggFunctions();
+      aggFunctionsNum = aggFunctions.length;
+    } else {
+      aggFunctions = new AggregationFunctionCallEval[0];
+      aggFunctionsNum = 0;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    plan = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java
new file mode 100644
index 0000000..60a7c19
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java
@@ -0,0 +1,223 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class BNLJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  private JoinNode plan;
+  private final boolean hasJoinQual;
+  private EvalNode joinQual;
+
+  private List<Tuple> leftTupleSlots;
+  private List<Tuple> rightTupleSlots;
+  private Iterator<Tuple> leftIterator;
+  private Iterator<Tuple> rightIterator;
+
+  private boolean leftEnd;
+  private boolean rightEnd;
+
+  // temporal tuples and states for nested loop join
+  private FrameTuple frameTuple;
+  private Tuple leftTuple = null;
+  private Tuple outputTuple = null;
+  private Tuple rightNext = null;
+
+  private final int TUPLE_SLOT_SIZE = 10000;
+
+  // projection
+  private Projector projector;
+
+  public BNLJoinExec(final TaskAttemptContext context, final JoinNode plan,
+                     final PhysicalExec leftExec, PhysicalExec rightExec) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), leftExec, rightExec);
+    this.plan = plan;
+    this.joinQual = plan.getJoinQual();
+    if (joinQual != null) { // if join type is not 'cross join'
+      hasJoinQual = true;
+    } else {
+      hasJoinQual = false;
+    }
+    this.leftTupleSlots = new ArrayList<Tuple>(TUPLE_SLOT_SIZE);
+    this.rightTupleSlots = new ArrayList<Tuple>(TUPLE_SLOT_SIZE);
+    this.leftIterator = leftTupleSlots.iterator();
+    this.rightIterator = rightTupleSlots.iterator();
+    this.rightEnd = false;
+    this.leftEnd = false;
+
+    // for projection
+    if (!plan.hasTargets()) {
+      plan.setTargets(PlannerUtil.schemaToTargets(outSchema));
+    }
+
+    projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    frameTuple = new FrameTuple();
+    outputTuple = new VTuple(outSchema.size());
+  }
+
+  public JoinNode getPlan() {
+    return plan;
+  }
+
+  public Tuple next() throws IOException {
+
+    if (leftTupleSlots.isEmpty()) {
+      for (int k = 0; k < TUPLE_SLOT_SIZE; k++) {
+        Tuple t = leftChild.next();
+        if (t == null) {
+          leftEnd = true;
+          break;
+        }
+        leftTupleSlots.add(t);
+      }
+      leftIterator = leftTupleSlots.iterator();
+      leftTuple = leftIterator.next();
+    }
+
+    if (rightTupleSlots.isEmpty()) {
+      for (int k = 0; k < TUPLE_SLOT_SIZE; k++) {
+        Tuple t = rightChild.next();
+        if (t == null) {
+          rightEnd = true;
+          break;
+        }
+        rightTupleSlots.add(t);
+      }
+      rightIterator = rightTupleSlots.iterator();
+    }
+
+    if((rightNext = rightChild.next()) == null){
+      rightEnd = true;
+    }
+
+    while (true) {
+      if (!rightIterator.hasNext()) { // if leftIterator ended
+        if (leftIterator.hasNext()) { // if rightTupleslot remains
+          leftTuple = leftIterator.next();
+          rightIterator = rightTupleSlots.iterator();
+        } else {
+          if (rightEnd) {
+            rightChild.rescan();
+            rightEnd = false;
+            
+            if (leftEnd) {
+              return null;
+            }
+            leftTupleSlots.clear();
+            for (int k = 0; k < TUPLE_SLOT_SIZE; k++) {
+              Tuple t = leftChild.next();
+              if (t == null) {
+                leftEnd = true;
+                break;
+              }
+              leftTupleSlots.add(t);
+            }
+            if (leftTupleSlots.isEmpty()) {
+              return null;
+            }
+            leftIterator = leftTupleSlots.iterator();
+            leftTuple = leftIterator.next();
+            
+          } else {
+            leftIterator = leftTupleSlots.iterator();
+            leftTuple = leftIterator.next();
+          }
+          
+          rightTupleSlots.clear();
+          if (rightNext != null) {
+            rightTupleSlots.add(rightNext);
+            for (int k = 1; k < TUPLE_SLOT_SIZE; k++) { // fill right
+              Tuple t = rightChild.next();
+              if (t == null) {
+                rightEnd = true;
+                break;
+              }
+              rightTupleSlots.add(t);
+            }
+          } else {
+            for (int k = 0; k < TUPLE_SLOT_SIZE; k++) { // fill right
+              Tuple t = rightChild.next();
+              if (t == null) {
+                rightEnd = true;
+                break;
+              }
+              rightTupleSlots.add(t);
+            }
+          }
+          
+          if ((rightNext = rightChild.next()) == null) {
+            rightEnd = true;
+          }
+          rightIterator = rightTupleSlots.iterator();
+        }
+      }
+
+      frameTuple.set(leftTuple, rightIterator.next());
+      if (hasJoinQual) {
+        if (joinQual.eval(inSchema, frameTuple).isTrue()) {
+          projector.eval(frameTuple, outputTuple);
+          return outputTuple;
+        }
+      } else {
+        projector.eval(frameTuple, outputTuple);
+        return outputTuple;
+      }
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+    rightEnd = false;
+    rightTupleSlots.clear();
+    leftTupleSlots.clear();
+    rightIterator = rightTupleSlots.iterator();
+    leftIterator = leftTupleSlots.iterator();
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+
+    rightTupleSlots.clear();
+    leftTupleSlots.clear();
+    rightTupleSlots = null;
+    leftTupleSlots = null;
+    rightIterator = null;
+    leftIterator = null;
+    plan = null;
+    joinQual = null;
+    projector = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
new file mode 100644
index 0000000..35de707
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
@@ -0,0 +1,143 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class BSTIndexScanExec extends PhysicalExec {
+  private ScanNode scanNode;
+  private SeekableScanner fileScanner;
+  
+  private EvalNode qual;
+  private BSTIndex.BSTIndexReader reader;
+  
+  private Projector projector;
+  
+  private Datum[] datum = null;
+  
+  private boolean initialize = true;
+
+  private float progress;
+
+  public BSTIndexScanExec(TaskAttemptContext context,
+                          AbstractStorageManager sm , ScanNode scanNode ,
+       FileFragment fragment, Path fileName , Schema keySchema,
+       TupleComparator comparator , Datum[] datum) throws IOException {
+    super(context, scanNode.getInSchema(), scanNode.getOutSchema());
+    this.scanNode = scanNode;
+    this.qual = scanNode.getQual();
+    this.datum = datum;
+
+    this.fileScanner = StorageManagerFactory.getSeekableScanner(context.getConf(),
+        scanNode.getTableDesc().getMeta(), scanNode.getInSchema(), fragment, outSchema);
+    this.fileScanner.init();
+    this.projector = new Projector(inSchema, outSchema, scanNode.getTargets());
+
+    this.reader = new BSTIndex(sm.getFileSystem().getConf()).
+        getIndexReader(fileName, keySchema, comparator);
+    this.reader.open();
+  }
+
+  @Override
+  public void init() throws IOException {
+    progress = 0.0f;
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if(initialize) {
+      //TODO : more complicated condition
+      Tuple key = new VTuple(datum.length);
+      key.put(datum);
+      long offset = reader.find(key);
+      if (offset == -1) {
+        reader.close();
+        fileScanner.close();
+        return null;
+      }else {
+        fileScanner.seek(offset);
+      }
+      initialize = false;
+    } else {
+      if(!reader.isCurInMemory()) {
+        return null;
+      }
+      long offset = reader.next();
+      if(offset == -1 ) {
+        reader.close();
+        fileScanner.close();
+        return null;
+      } else { 
+      fileScanner.seek(offset);
+      }
+    }
+    Tuple tuple;
+    Tuple outTuple = new VTuple(this.outSchema.size());
+    if (!scanNode.hasQual()) {
+      if ((tuple = fileScanner.next()) != null) {
+        projector.eval(tuple, outTuple);
+        return outTuple;
+      } else {
+        return null;
+      }
+    } else {
+       while(reader.isCurInMemory() && (tuple = fileScanner.next()) != null) {
+         if (qual.eval(inSchema, tuple).isTrue()) {
+           projector.eval(tuple, outTuple);
+           return outTuple;
+         } else {
+           fileScanner.seek(reader.next());
+         }
+       }
+     }
+
+    return null;
+  }
+  @Override
+  public void rescan() throws IOException {
+    fileScanner.reset();
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.cleanup(null, reader, fileScanner);
+    reader = null;
+    fileScanner = null;
+    scanNode = null;
+    qual = null;
+    projector = null;
+  }
+
+  @Override
+  public float getProgress() {
+    return progress;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java
new file mode 100644
index 0000000..f6f3e52
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java
@@ -0,0 +1,265 @@
+/**
+ * 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.planner.physical;
+
+import java.util.Stack;
+
+public class BasicPhysicalExecutorVisitor<CONTEXT, RESULT> implements PhysicalExecutorVisitor<CONTEXT, RESULT> {
+
+  public RESULT visit(PhysicalExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+      throws PhysicalPlanningException {
+
+    // Please keep all physical executors except for abstract class.
+    // They should be ordered in an lexicography order of their names for easy code maintenance.
+    if (exec instanceof BNLJoinExec) {
+      return visitBNLJoin(context, (BNLJoinExec) exec, stack);
+    } else if (exec instanceof BSTIndexScanExec) {
+      return visitBSTIndexScan(context, (BSTIndexScanExec) exec, stack);
+    } else if (exec instanceof EvalExprExec) {
+      return visitEvalExpr(context, (EvalExprExec) exec, stack);
+    } else if (exec instanceof ExternalSortExec) {
+      return visitExternalSort(context, (ExternalSortExec) exec, stack);
+    } else if (exec instanceof HashAggregateExec) {
+      return visitHashAggregate(context, (HashAggregateExec) exec, stack);
+    } else if (exec instanceof HashBasedColPartitionStoreExec) {
+      return visitHashBasedColPartitionStore(context, (HashBasedColPartitionStoreExec) exec, stack);
+    } else if (exec instanceof HashFullOuterJoinExec) {
+      return visitHashFullOuterJoin(context, (HashFullOuterJoinExec) exec, stack);
+    } else if (exec instanceof HashJoinExec) {
+      return visitHashJoin(context, (HashJoinExec) exec, stack);
+    } else if (exec instanceof HashLeftAntiJoinExec) {
+      return visitHashLeftAntiJoin(context, (HashLeftAntiJoinExec) exec, stack);
+    } else if (exec instanceof HashLeftOuterJoinExec) {
+      return visitHashLeftOuterJoin(context, (HashLeftOuterJoinExec) exec, stack);
+    } else if (exec instanceof HashLeftSemiJoinExec) {
+      return visitLeftHashSemiJoin(context, (HashLeftSemiJoinExec) exec, stack);
+    } else if (exec instanceof HashShuffleFileWriteExec) {
+      return visitHashShuffleFileWrite(context, (HashShuffleFileWriteExec) exec, stack);
+    } else if (exec instanceof HavingExec) {
+      return visitHaving(context, (HavingExec) exec, stack);
+    } else if (exec instanceof LimitExec) {
+      return visitLimit(context, (LimitExec) exec, stack);
+    } else if (exec instanceof MemSortExec) {
+      return visitMemSort(context, (MemSortExec) exec, stack);
+    } else if (exec instanceof MergeFullOuterJoinExec) {
+      return visitMergeFullOuterJoin(context, (MergeFullOuterJoinExec) exec, stack);
+    } else if (exec instanceof MergeJoinExec) {
+      return visitMergeJoin(context, (MergeJoinExec) exec, stack);
+    } else if (exec instanceof NLJoinExec) {
+      return visitNLJoin(context, (NLJoinExec) exec, stack);
+    } else if (exec instanceof NLLeftOuterJoinExec) {
+      return visitNLLeftOuterJoin(context, (NLLeftOuterJoinExec) exec, stack);
+    } else if (exec instanceof ProjectionExec) {
+      return visitProjection(context, (ProjectionExec) exec, stack);
+    } else if (exec instanceof RangeShuffleFileWriteExec) {
+      return visitRangeShuffleFileWrite(context, (RangeShuffleFileWriteExec) exec, stack);
+    } else if (exec instanceof RightOuterMergeJoinExec) {
+      return visitRightOuterMergeJoin(context, (RightOuterMergeJoinExec) exec, stack);
+    } else if (exec instanceof SelectionExec) {
+      return visitSelection(context, (SelectionExec) exec, stack);
+    } else if (exec instanceof SeqScanExec) {
+      return visitSeqScan(context, (SeqScanExec) exec, stack);
+    } else if (exec instanceof SortAggregateExec) {
+      return visitSortAggregate(context, (SortAggregateExec) exec, stack);
+    } else if (exec instanceof SortBasedColPartitionStoreExec) {
+      return visitSortBasedColPartitionStore(context, (SortBasedColPartitionStoreExec) exec, stack);
+    } else if (exec instanceof StoreTableExec) {
+      return visitStoreTable(context, (StoreTableExec) exec, stack);
+    }
+
+    throw new PhysicalPlanningException("Unsupported Type: " + exec.getClass().getSimpleName());
+  }
+
+  private RESULT visitUnaryExecutor(CONTEXT context, UnaryPhysicalExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    stack.push(exec);
+    RESULT r = visit(exec.getChild(), stack, context);
+    stack.pop();
+    return r;
+  }
+
+  private RESULT visitBinaryExecutor(CONTEXT context, BinaryPhysicalExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    stack.push(exec);
+    RESULT r = visit(exec.getLeftChild(), stack, context);
+    visit(exec.getRightChild(), stack, context);
+    stack.pop();
+    return r;
+  }
+
+  @Override
+  public RESULT visitBNLJoin(CONTEXT context, BNLJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitBSTIndexScan(CONTEXT context, BSTIndexScanExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitEvalExpr(CONTEXT context, EvalExprExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitExternalSort(CONTEXT context, ExternalSortExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHashAggregate(CONTEXT context, HashAggregateExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHashBasedColPartitionStore(CONTEXT context, HashBasedColPartitionStoreExec exec,
+                                                Stack<PhysicalExec> stack) throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHashFullOuterJoin(CONTEXT context, HashFullOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHashJoin(CONTEXT context, HashJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHashLeftAntiJoin(CONTEXT context, HashLeftAntiJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHashLeftOuterJoin(CONTEXT context, HashLeftOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitLeftHashSemiJoin(CONTEXT context, HashLeftSemiJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHashShuffleFileWrite(CONTEXT context, HashShuffleFileWriteExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitHaving(CONTEXT context, HavingExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitLimit(CONTEXT context, LimitExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitMemSort(CONTEXT context, MemSortExec exec, Stack<PhysicalExec> stack) throws
+      PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitMergeFullOuterJoin(CONTEXT context, MergeFullOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitMergeJoin(CONTEXT context, MergeJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitNLJoin(CONTEXT context, NLJoinExec exec, Stack<PhysicalExec> stack) throws
+      PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitNLLeftOuterJoin(CONTEXT context, NLLeftOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitProjection(CONTEXT context, ProjectionExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitRangeShuffleFileWrite(CONTEXT context, RangeShuffleFileWriteExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitRightOuterMergeJoin(CONTEXT context, RightOuterMergeJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitBinaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitSelection(CONTEXT context, SelectionExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitSeqScan(CONTEXT context, SeqScanExec exec, Stack<PhysicalExec> stack) {
+    return null;
+  }
+
+  @Override
+  public RESULT visitSortAggregate(CONTEXT context, SortAggregateExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitSortBasedColPartitionStore(CONTEXT context, SortBasedColPartitionStoreExec exec,
+                                                Stack<PhysicalExec> stack) throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+
+  @Override
+  public RESULT visitStoreTable(CONTEXT context, StoreTableExec exec, Stack<PhysicalExec> stack) throws PhysicalPlanningException {
+    return visitUnaryExecutor(context, exec, stack);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BinaryPhysicalExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BinaryPhysicalExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BinaryPhysicalExec.java
new file mode 100644
index 0000000..628c18c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BinaryPhysicalExec.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.catalog.Schema;
+
+import java.io.IOException;
+
+public abstract class BinaryPhysicalExec extends PhysicalExec {
+  protected PhysicalExec leftChild;
+  protected PhysicalExec rightChild;
+  protected float progress;
+  protected TableStats inputStats;
+
+  public BinaryPhysicalExec(final TaskAttemptContext context,
+                            final Schema inSchema, final Schema outSchema,
+                            final PhysicalExec outer, final PhysicalExec inner) {
+    super(context, inSchema, outSchema);
+    this.leftChild = outer;
+    this.rightChild = inner;
+    this.inputStats = new TableStats();
+  }
+
+  public PhysicalExec getLeftChild() {
+    return leftChild;
+  }
+
+  public PhysicalExec getRightChild() {
+    return rightChild;
+  }
+
+  @Override
+  public void init() throws IOException {
+    leftChild.init();
+    rightChild.init();
+    progress = 0.0f;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    leftChild.rescan();
+    rightChild.rescan();
+  }
+
+  @Override
+  public void close() throws IOException {
+    leftChild.close();
+    rightChild.close();
+
+    getInputStats();
+    
+    leftChild = null;
+    rightChild = null;
+    
+    progress = 1.0f;
+  }
+
+  @Override
+  public float getProgress() {
+    if (leftChild == null) {
+      return progress;
+    }
+    return leftChild.getProgress() * 0.5f + rightChild.getProgress() * 0.5f;
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    if (leftChild == null) {
+      return inputStats;
+    }
+    TableStats leftInputStats = leftChild.getInputStats();
+    inputStats.setNumBytes(0);
+    inputStats.setReadBytes(0);
+    inputStats.setNumRows(0);
+
+    if (leftInputStats != null) {
+      inputStats.setNumBytes(leftInputStats.getNumBytes());
+      inputStats.setReadBytes(leftInputStats.getReadBytes());
+      inputStats.setNumRows(leftInputStats.getNumRows());
+    }
+
+    TableStats rightInputStats = rightChild.getInputStats();
+    if (rightInputStats != null) {
+      inputStats.setNumBytes(inputStats.getNumBytes() + rightInputStats.getNumBytes());
+      inputStats.setReadBytes(inputStats.getReadBytes() + rightInputStats.getReadBytes());
+      inputStats.setNumRows(inputStats.getNumRows() + rightInputStats.getNumRows());
+    }
+
+    return inputStats;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
new file mode 100644
index 0000000..fe36905
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
@@ -0,0 +1,100 @@
+/**
+ * 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.planner.physical;
+
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.engine.planner.logical.InsertNode;
+import org.apache.tajo.engine.planner.logical.CreateTableNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.StoreTableNode;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public abstract class ColPartitionStoreExec extends UnaryPhysicalExec {
+  protected final TableMeta meta;
+  protected final StoreTableNode plan;
+  protected Path storeTablePath;
+
+  protected final int keyNum;
+  protected final int [] keyIds;
+  protected final String [] keyNames;
+
+  public ColPartitionStoreExec(TaskAttemptContext context, StoreTableNode plan, PhysicalExec child) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+    this.plan = plan;
+
+    if (plan.getType() == NodeType.CREATE_TABLE) {
+      this.outSchema = ((CreateTableNode)plan).getTableSchema();
+    }
+
+    // set table meta
+    if (this.plan.hasOptions()) {
+      meta = CatalogUtil.newTableMeta(plan.getStorageType(), plan.getOptions());
+    } else {
+      meta = CatalogUtil.newTableMeta(plan.getStorageType());
+    }
+
+    // Find column index to name subpartition directory path
+    keyNum = this.plan.getPartitionMethod().getExpressionSchema().size();
+
+    keyIds = new int[keyNum];
+    keyNames = new String[keyNum];
+    for (int i = 0; i < keyNum; i++) {
+      Column column = this.plan.getPartitionMethod().getExpressionSchema().getColumn(i);
+      keyNames[i] = column.getSimpleName();
+
+      if (this.plan.getType() == NodeType.INSERT) {
+        InsertNode insertNode = ((InsertNode)plan);
+        int idx = insertNode.getTableSchema().getColumnId(column.getQualifiedName());
+        keyIds[i] = idx;
+      } else if (this.plan.getType() == NodeType.CREATE_TABLE) {
+        CreateTableNode createTable = (CreateTableNode) plan;
+        int idx = createTable.getLogicalSchema().getColumnId(column.getQualifiedName());
+        keyIds[i] = idx;
+      } else {
+        // We can get partition column from a logical schema.
+        // Don't use output schema because it is rewritten.
+        keyIds[i] = plan.getOutSchema().getColumnId(column.getQualifiedName());
+      }
+    }
+  }
+
+  @Override
+  public void init() throws IOException {
+    super.init();
+
+    storeTablePath = context.getOutputPath();
+    FileSystem fs = storeTablePath.getFileSystem(context.getConf());
+    if (!fs.exists(storeTablePath.getParent())) {
+      fs.mkdirs(storeTablePath.getParent());
+    }
+  }
+
+
+  protected Path getDataFile(String partition) {
+    return StorageUtil.concatPath(storeTablePath.getParent(), partition, storeTablePath.getName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java
new file mode 100644
index 0000000..a843bce
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java
@@ -0,0 +1,66 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.engine.planner.logical.EvalExprNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class EvalExprExec extends PhysicalExec {
+  private final EvalExprNode plan;
+  private float progress;
+
+  public EvalExprExec(final TaskAttemptContext context, final EvalExprNode plan) {
+    super(context, plan.getInSchema(), plan.getOutSchema());
+    this.plan = plan;
+  }
+
+  @Override
+  public void init() throws IOException {
+    progress = 0.0f;
+  }
+
+  @Override
+  public Tuple next() throws IOException {    
+    Target [] targets = plan.getTargets();
+    Tuple t = new VTuple(targets.length);
+    for (int i = 0; i < targets.length; i++) {
+      t.put(i, targets[i].getEvalTree().eval(inSchema, null));
+    }
+    return t;
+  }
+
+  @Override
+  public void rescan() throws IOException {    
+  }
+
+  @Override
+  public void close() throws IOException {
+    progress = 1.0f;
+  }
+
+  @Override
+  public float getProgress() {
+    return progress;
+  }
+}


[25/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java
new file mode 100644
index 0000000..8cc17cb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.event;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.storage.fragment.FileFragment;
+
+import java.util.Collection;
+
+public class FragmentScheduleEvent extends TaskSchedulerEvent {
+  private final FileFragment leftFragment;
+  private final Collection<FileFragment> rightFragments;
+
+  public FragmentScheduleEvent(final EventType eventType, final ExecutionBlockId blockId,
+                               final FileFragment fragment) {
+    this(eventType, blockId, fragment, null);
+  }
+
+  public FragmentScheduleEvent(final EventType eventType,
+                               final ExecutionBlockId blockId,
+                               final FileFragment leftFragment,
+                               final Collection<FileFragment> rightFragments) {
+    super(eventType, blockId);
+    this.leftFragment = leftFragment;
+    this.rightFragments = rightFragments;
+  }
+
+  public boolean hasRightFragments() {
+    return this.rightFragments != null && !this.rightFragments.isEmpty();
+  }
+
+  public FileFragment getLeftFragment() {
+    return leftFragment;
+  }
+
+  public Collection<FileFragment> getRightFragments() { return rightFragments; }
+
+  @Override
+  public String toString() {
+    return "FragmentScheduleEvent{" +
+        "leftFragment=" + leftFragment +
+        ", rightFragments=" + rightFragments +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
new file mode 100644
index 0000000..c34b174
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.event;
+
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tajo.ExecutionBlockId;
+
+import java.util.Map;
+
+public class GrouppedContainerAllocatorEvent
+    extends ContainerAllocationEvent {
+  private final Map<String, Integer> requestMap;
+
+  public GrouppedContainerAllocatorEvent(ContainerAllocatorEventType eventType,
+                                         ExecutionBlockId executionBlockId,
+                                         Priority priority,
+                                         Resource resource,
+                                         Map<String, Integer> requestMap,
+                                         boolean isLeafQuery, float progress) {
+    super(eventType, executionBlockId, priority,
+        resource, requestMap.size(), isLeafQuery, progress);
+    this.requestMap = requestMap;
+  }
+
+  public Map<String, Integer> getRequestMap() {
+    return this.requestMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java
new file mode 100644
index 0000000..92e6695
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.event;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryUnitAttemptId;
+
+/**
+ * This event is sent to a running TaskAttempt on a worker.
+ */
+public class LocalTaskEvent extends AbstractEvent<LocalTaskEventType> {
+  private final QueryUnitAttemptId taskAttemptId;
+  private final ContainerId containerId;
+
+  public LocalTaskEvent(QueryUnitAttemptId taskAttemptId, ContainerId containerId, LocalTaskEventType eventType) {
+    super(eventType);
+    this.taskAttemptId = taskAttemptId;
+    this.containerId = containerId;
+  }
+
+  public QueryUnitAttemptId getTaskAttemptId() {
+    return taskAttemptId;
+  }
+
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEventType.java
new file mode 100644
index 0000000..00b548e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEventType.java
@@ -0,0 +1,23 @@
+/**
+ * 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.master.event;
+
+public enum LocalTaskEventType {
+  KILL
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java
new file mode 100644
index 0000000..dc75a1d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java
@@ -0,0 +1,42 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.querymaster.SubQueryState;
+
+public class QueryCompletedEvent extends QueryEvent {
+  private final ExecutionBlockId executionBlockId;
+  private final SubQueryState finalState;
+
+  public QueryCompletedEvent(final ExecutionBlockId executionBlockId,
+                             SubQueryState finalState) {
+    super(executionBlockId.getQueryId(), QueryEventType.QUERY_COMPLETED);
+    this.executionBlockId = executionBlockId;
+    this.finalState = finalState;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
+
+  public SubQueryState getState() {
+    return finalState;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
new file mode 100644
index 0000000..eebf5c0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
@@ -0,0 +1,34 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.QueryId;
+
+public class QueryDiagnosticsUpdateEvent extends QueryEvent {
+  private final String msg;
+
+  public QueryDiagnosticsUpdateEvent(final QueryId id, String diagnostic) {
+    super(id, QueryEventType.DIAGNOSTIC_UPDATE);
+    this.msg = diagnostic;
+  }
+
+  public String getDiagnosticUpdate() {
+    return msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEvent.java
new file mode 100644
index 0000000..a3b7667
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEvent.java
@@ -0,0 +1,36 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryId;
+
+public class QueryEvent extends AbstractEvent<QueryEventType> {
+  private final QueryId id;
+
+
+  public QueryEvent(final QueryId id, final QueryEventType queryEvent) {
+    super(queryEvent);
+    this.id = id;
+  }
+
+  public QueryId getQueryId() {
+    return id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEventType.java
new file mode 100644
index 0000000..edc0cd8
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryEventType.java
@@ -0,0 +1,36 @@
+/**
+ * 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.master.event;
+
+public enum QueryEventType {
+
+  // Producer: TajoMaster
+  START,
+  KILL,
+
+  // Producer: SubQuery
+  SUBQUERY_COMPLETED,
+
+  // Producer: Query
+  QUERY_COMPLETED,
+
+  // Producer: Any component
+  DIAGNOSTIC_UPDATE,
+  INTERNAL_ERROR,
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QueryMasterQueryCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryMasterQueryCompletedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryMasterQueryCompletedEvent.java
new file mode 100644
index 0000000..bc7e0f4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryMasterQueryCompletedEvent.java
@@ -0,0 +1,39 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryId;
+
+public class QueryMasterQueryCompletedEvent extends AbstractEvent<QueryMasterQueryCompletedEvent.EventType> {
+  public enum EventType {
+    QUERY_FINISH
+  }
+
+  private final QueryId queryId;
+
+  public QueryMasterQueryCompletedEvent(QueryId queryId) {
+    super(EventType.QUERY_FINISH);
+    this.queryId = queryId;
+  }
+
+  public QueryId getQueryId() {
+    return this.queryId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
new file mode 100644
index 0000000..dd996e6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
@@ -0,0 +1,74 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.session.Session;
+
+/**
+ * This event is conveyed to QueryMaster.
+ */
+public class QueryStartEvent extends AbstractEvent {
+  public enum EventType {
+    QUERY_START
+  }
+
+  private final QueryId queryId;
+  private final Session session;
+  private final QueryContext queryContext;
+  private final String sql;
+  private final String logicalPlanJson;
+
+  public QueryStartEvent(QueryId queryId, Session session, QueryContext queryContext, String sql,
+                         String logicalPlanJson) {
+    super(EventType.QUERY_START);
+    this.queryId = queryId;
+    this.session = session;
+    this.queryContext = queryContext;
+    this.sql = sql;
+    this.logicalPlanJson = logicalPlanJson;
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public Session getSession() {
+    return this.session;
+  }
+
+  public QueryContext getQueryContext() {
+    return this.queryContext;
+  }
+
+  public String getSql() {
+    return this.sql;
+  }
+
+  public String getLogicalPlanJson() {
+    return logicalPlanJson;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getName() + "," + getType() + "," + queryId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
new file mode 100644
index 0000000..ae36a69
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
@@ -0,0 +1,35 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.ExecutionBlockId;
+
+public class QuerySubQueryEvent extends QueryEvent {
+  private ExecutionBlockId executionBlockId;
+
+  public QuerySubQueryEvent(final ExecutionBlockId id,
+                            final QueryEventType queryEvent) {
+    super(id.getQueryId(), queryEvent);
+    this.executionBlockId = id;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return this.executionBlockId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java
new file mode 100644
index 0000000..a2acc7e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java
@@ -0,0 +1,87 @@
+/*
+ * 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.master.event;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
+import org.apache.tajo.master.querymaster.QueryUnitAttempt;
+
+public class QueryUnitAttemptScheduleEvent extends TaskSchedulerEvent {
+  private final QueryUnitAttemptScheduleContext context;
+  private final QueryUnitAttempt queryUnitAttempt;
+
+  public QueryUnitAttemptScheduleEvent(EventType eventType, ExecutionBlockId executionBlockId,
+                                       QueryUnitAttemptScheduleContext context, QueryUnitAttempt queryUnitAttempt) {
+    super(eventType, executionBlockId);
+    this.context = context;
+    this.queryUnitAttempt = queryUnitAttempt;
+  }
+
+  public QueryUnitAttempt getQueryUnitAttempt() {
+    return queryUnitAttempt;
+  }
+
+  public QueryUnitAttemptScheduleContext getContext() {
+    return context;
+  }
+
+  public static class QueryUnitAttemptScheduleContext {
+    private ContainerId containerId;
+    private String host;
+    private RpcCallback<QueryUnitRequestProto> callback;
+
+    public QueryUnitAttemptScheduleContext() {
+
+    }
+
+    public QueryUnitAttemptScheduleContext(ContainerId containerId,
+                                           String host,
+                                           RpcCallback<QueryUnitRequestProto> callback) {
+      this.containerId = containerId;
+      this.host = host;
+      this.callback = callback;
+    }
+
+    public ContainerId getContainerId() {
+      return containerId;
+    }
+
+    public void setContainerId(ContainerId containerId) {
+      this.containerId = containerId;
+    }
+
+    public String getHost() {
+      return host;
+    }
+
+    public void setHost(String host) {
+      this.host = host;
+    }
+
+    public RpcCallback<QueryUnitRequestProto> getCallback() {
+      return callback;
+    }
+
+    public void setCallback(RpcCallback<QueryUnitRequestProto> callback) {
+      this.callback = callback;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
new file mode 100644
index 0000000..6389798
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
@@ -0,0 +1,42 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.querymaster.SubQueryState;
+
+public class SubQueryCompletedEvent extends QueryEvent {
+  private final ExecutionBlockId executionBlockId;
+  private final SubQueryState finalState;
+
+  public SubQueryCompletedEvent(final ExecutionBlockId executionBlockId,
+                                SubQueryState finalState) {
+    super(executionBlockId.getQueryId(), QueryEventType.SUBQUERY_COMPLETED);
+    this.executionBlockId = executionBlockId;
+    this.finalState = finalState;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
+
+  public SubQueryState getState() {
+    return finalState;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
new file mode 100644
index 0000000..a8f4800
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.event;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.tajo.ExecutionBlockId;
+
+import java.util.List;
+
+public class SubQueryContainerAllocationEvent extends SubQueryEvent {
+  private List<Container> allocatedContainer;
+
+  public SubQueryContainerAllocationEvent(final ExecutionBlockId id,
+                                          List<Container> allocatedContainer) {
+    super(id, SubQueryEventType.SQ_CONTAINER_ALLOCATED);
+    this.allocatedContainer = allocatedContainer;
+  }
+
+  public List<Container> getAllocatedContainer() {
+    return this.allocatedContainer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryDiagnosticsUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryDiagnosticsUpdateEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryDiagnosticsUpdateEvent.java
new file mode 100644
index 0000000..0810e81
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryDiagnosticsUpdateEvent.java
@@ -0,0 +1,34 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.ExecutionBlockId;
+
+public class SubQueryDiagnosticsUpdateEvent extends SubQueryEvent {
+  private final String msg;
+
+  public SubQueryDiagnosticsUpdateEvent(final ExecutionBlockId id, String diagnostic) {
+    super(id, SubQueryEventType.SQ_DIAGNOSTIC_UPDATE);
+    this.msg = diagnostic;
+  }
+
+  public String getDiagnosticUpdate() {
+    return msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
new file mode 100644
index 0000000..2b3d598
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
@@ -0,0 +1,35 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.ExecutionBlockId;
+
+public class SubQueryEvent extends AbstractEvent<SubQueryEventType> {
+  private final ExecutionBlockId id;
+
+  public SubQueryEvent(ExecutionBlockId id, SubQueryEventType subQueryEventType) {
+    super(subQueryEventType);
+    this.id = id;
+  }
+
+  public ExecutionBlockId getSubQueryId() {
+    return id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
new file mode 100644
index 0000000..8003ef3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
@@ -0,0 +1,43 @@
+/**
+ * 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.master.event;
+
+/**
+ * Event Types handled by SubQuery
+ */
+public enum SubQueryEventType {
+
+  // Producer: Query
+  SQ_INIT,
+  SQ_START,
+  SQ_CONTAINER_ALLOCATED,
+  SQ_KILL,
+  SQ_LAUNCH,
+
+  // Producer: QueryUnit
+  SQ_TASK_COMPLETED,
+  SQ_FAILED,
+
+  // Producer: Completed
+  SQ_SUBQUERY_COMPLETED,
+
+  // Producer: Any component
+  SQ_DIAGNOSTIC_UPDATE,
+  SQ_INTERNAL_ERROR
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
new file mode 100644
index 0000000..0502534
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
@@ -0,0 +1,43 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.QueryUnitId;
+import org.apache.tajo.master.TaskState;
+
+/**
+ * Event Class: From Task to SubQuery
+ */
+public class SubQueryTaskEvent extends SubQueryEvent {
+  private QueryUnitId taskId;
+  private TaskState state;
+  public SubQueryTaskEvent(QueryUnitId taskId, TaskState state) {
+    super(taskId.getExecutionBlockId(), SubQueryEventType.SQ_TASK_COMPLETED);
+    this.taskId = taskId;
+    this.state = state;
+  }
+
+  public QueryUnitId getTaskId() {
+    return this.taskId;
+  }
+
+  public TaskState getState() {
+    return state;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java
new file mode 100644
index 0000000..4934633
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java
@@ -0,0 +1,48 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tajo.QueryUnitAttemptId;
+
+public class TaskAttemptAssignedEvent extends TaskAttemptEvent {
+  private final ContainerId cId;
+  private final String hostName;
+  private final int pullServerPort;
+
+  public TaskAttemptAssignedEvent(QueryUnitAttemptId id, ContainerId cId,
+                                  String hostname, int pullServerPort) {
+    super(id, TaskAttemptEventType.TA_ASSIGNED);
+    this.cId = cId;
+    this.hostName = hostname;
+    this.pullServerPort = pullServerPort;
+  }
+
+  public ContainerId getContainerId() {
+    return cId;
+  }
+
+  public String getHostName() {
+    return hostName;
+  }
+
+  public int getPullServerPort() {
+    return pullServerPort;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEvent.java
new file mode 100644
index 0000000..f2df144
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEvent.java
@@ -0,0 +1,36 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryUnitAttemptId;
+
+public class TaskAttemptEvent extends AbstractEvent<TaskAttemptEventType> {
+  private final QueryUnitAttemptId id;
+
+  public TaskAttemptEvent(QueryUnitAttemptId id,
+                          TaskAttemptEventType taskAttemptEventType) {
+    super(taskAttemptEventType);
+    this.id = id;
+  }
+
+  public QueryUnitAttemptId getTaskAttemptId() {
+    return this.id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java
new file mode 100644
index 0000000..e35b154
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java
@@ -0,0 +1,54 @@
+/**
+ * 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.master.event;
+
+/**
+ * Event types handled by TaskAttempt.
+ */
+public enum TaskAttemptEventType {
+
+  //Producer:Task
+  TA_SCHEDULE,
+  TA_RESCHEDULE,
+
+  //Producer:Client, Task
+  TA_KILL,
+  TA_LOCAL_KILLED,
+
+  //Producer:Scheduler
+  TA_ASSIGNED,
+  TA_SCHEDULE_CANCELED,
+
+  //Producer:Scheduler
+  TA_LAUNCHED,
+
+  //Producer:TaskAttemptListener
+  TA_DIAGNOSTICS_UPDATE,
+  TA_COMMIT_PENDING,
+  TA_DONE,
+  TA_FATAL_ERROR,
+  TA_UPDATE,
+  TA_TIMED_OUT,
+
+  //Producer:TaskCleaner
+  TA_CLEANUP_DONE,
+
+  //Producer:Job
+  TA_TOO_MANY_FETCH_FAILURE,
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptScheduleEvent.java
new file mode 100644
index 0000000..8f153af
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptScheduleEvent.java
@@ -0,0 +1,37 @@
+/*
+ * 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.master.event;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.QueryUnitAttemptId;
+
+public class TaskAttemptScheduleEvent extends TaskAttemptEvent {
+  private Configuration conf;
+
+  public TaskAttemptScheduleEvent(final Configuration conf,
+                                  final QueryUnitAttemptId id,
+                                  final TaskAttemptEventType taskAttemptEventType) {
+    super(id, taskAttemptEventType);
+    this.conf = conf;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
new file mode 100644
index 0000000..d980e05
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
@@ -0,0 +1,36 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskStatusProto;
+
+public class TaskAttemptStatusUpdateEvent extends TaskAttemptEvent {
+  private final TaskStatusProto status;
+
+  public TaskAttemptStatusUpdateEvent(final QueryUnitAttemptId id,
+                                      TaskStatusProto status) {
+    super(id, TaskAttemptEventType.TA_UPDATE);
+    this.status = status;
+  }
+
+  public TaskStatusProto getStatus() {
+    return status;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
new file mode 100644
index 0000000..3ee389a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
@@ -0,0 +1,35 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
+
+public class TaskCompletionEvent extends TaskAttemptEvent {
+  private TaskCompletionReport report;
+
+  public TaskCompletionEvent(TaskCompletionReport report) {
+    super(new QueryUnitAttemptId(report.getId()), TaskAttemptEventType.TA_DONE);
+    this.report = report;
+  }
+
+  public TaskCompletionReport getReport() {
+    return report;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEvent.java
new file mode 100644
index 0000000..234491b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEvent.java
@@ -0,0 +1,35 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryUnitId;
+
+public class TaskEvent extends AbstractEvent<TaskEventType> {
+  private final QueryUnitId id;
+
+  public TaskEvent(QueryUnitId id, TaskEventType taskEventType) {
+    super(taskEventType);
+    this.id = id;
+  }
+
+  public QueryUnitId getTaskId() {
+    return id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEventType.java
new file mode 100644
index 0000000..9448863
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskEventType.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.event;
+
+/**
+ * Event types handled by Task.
+ */
+public enum TaskEventType {
+
+  //Producer:Client, SubQuery
+  T_KILL,
+
+  //Producer:SubQuery
+  T_SCHEDULE,
+
+  //Producer:TaskAttempt
+  T_ATTEMPT_LAUNCHED,
+  T_ATTEMPT_COMMIT_PENDING,
+  T_ATTEMPT_FAILED,
+  T_ATTEMPT_SUCCEEDED,
+  T_ATTEMPT_KILLED
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
new file mode 100644
index 0000000..a4d9900
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
@@ -0,0 +1,41 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskFatalErrorReport;
+
+public class TaskFatalErrorEvent extends TaskAttemptEvent {
+  private final String message;
+
+  public TaskFatalErrorEvent(TaskFatalErrorReport report) {
+    super(new QueryUnitAttemptId(report.getId()),
+        TaskAttemptEventType.TA_FATAL_ERROR);
+    this.message = report.getErrorMessage();
+  }
+
+  public TaskFatalErrorEvent(QueryUnitAttemptId attemptId, String message) {
+    super(attemptId, TaskAttemptEventType.TA_FATAL_ERROR);
+    this.message = message;
+  }
+
+  public String errorMessage() {
+    return message;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
new file mode 100644
index 0000000..9be7cab
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
@@ -0,0 +1,59 @@
+/**
+ * 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.master.event;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
+import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType;
+
+public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
+
+  public enum TaskRequestEventType {
+    TASK_REQ
+  }
+
+  private final ContainerId workerId;
+  private final ExecutionBlockId executionBlockId;
+
+  private final RpcCallback<QueryUnitRequestProto> callback;
+
+  public TaskRequestEvent(ContainerId workerId,
+                          ExecutionBlockId executionBlockId,
+                          RpcCallback<QueryUnitRequestProto> callback) {
+    super(TaskRequestEventType.TASK_REQ);
+    this.workerId = workerId;
+    this.executionBlockId = executionBlockId;
+    this.callback = callback;
+  }
+
+  public ContainerId getContainerId() {
+    return this.workerId;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
+
+  public RpcCallback<QueryUnitRequestProto> getCallback() {
+    return this.callback;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
new file mode 100644
index 0000000..383845f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
@@ -0,0 +1,41 @@
+/**
+ * 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.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+
+public abstract class TaskSchedulerEvent extends AbstractEvent<EventType> {
+  public enum EventType {
+    T_SCHEDULE,
+    T_SCHEDULE_CANCEL
+  }
+
+  protected final ExecutionBlockId executionBlockId;
+
+  public TaskSchedulerEvent(EventType eventType, ExecutionBlockId executionBlockId) {
+    super(eventType);
+    this.executionBlockId = executionBlockId;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return this.executionBlockId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/event/TaskTAttemptEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskTAttemptEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskTAttemptEvent.java
new file mode 100644
index 0000000..28654f0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskTAttemptEvent.java
@@ -0,0 +1,34 @@
+/**
+ * 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.master.event;
+
+import org.apache.tajo.QueryUnitAttemptId;
+
+public class TaskTAttemptEvent extends TaskEvent {
+  private final QueryUnitAttemptId attemptId;
+  public TaskTAttemptEvent(QueryUnitAttemptId attemptId,
+                           TaskEventType eventType) {
+    super(attemptId.getQueryUnitId(), eventType);
+    this.attemptId = attemptId;
+  }
+
+  public QueryUnitAttemptId getTaskAttemptId() {
+    return attemptId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java
new file mode 100644
index 0000000..7c3d283
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+import org.apache.tajo.master.TajoMaster;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+
+public class CatalogMetricsGaugeSet implements MetricSet {
+  TajoMaster.MasterContext tajoMasterContext;
+  public CatalogMetricsGaugeSet(TajoMaster.MasterContext tajoMasterContext) {
+    this.tajoMasterContext = tajoMasterContext;
+  }
+
+  @Override
+  public Map<String, Metric> getMetrics() {
+    Map<String, Metric> metricsMap = new HashMap<String, Metric>();
+    metricsMap.put("numTables", new Gauge<Integer>() {
+      @Override
+      public Integer getValue() {
+        return tajoMasterContext.getCatalog().getAllTableNames(DEFAULT_DATABASE_NAME).size();
+      }
+    });
+
+    metricsMap.put("numFunctions", new Gauge<Integer>() {
+      @Override
+      public Integer getValue() {
+        return tajoMasterContext.getCatalog().getFunctions().size();
+      }
+    });
+
+    return metricsMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java
new file mode 100644
index 0000000..993d3b7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java
@@ -0,0 +1,74 @@
+/**
+ * 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.master.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.Worker;
+import org.apache.tajo.master.rm.WorkerState;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class WorkerResourceMetricsGaugeSet implements MetricSet {
+  TajoMaster.MasterContext tajoMasterContext;
+  public WorkerResourceMetricsGaugeSet(TajoMaster.MasterContext tajoMasterContext) {
+    this.tajoMasterContext = tajoMasterContext;
+  }
+
+  @Override
+  public Map<String, Metric> getMetrics() {
+    Map<String, Metric> metricsMap = new HashMap<String, Metric>();
+    metricsMap.put("totalWorkers", new Gauge<Integer>() {
+      @Override
+      public Integer getValue() {
+        return tajoMasterContext.getResourceManager().getWorkers().size();
+      }
+    });
+
+    metricsMap.put("liveWorkers", new Gauge<Integer>() {
+      @Override
+      public Integer getValue() {
+        return getNumWorkers(WorkerState.RUNNING);
+      }
+    });
+
+    metricsMap.put("deadWorkers", new Gauge<Integer>() {
+      @Override
+      public Integer getValue() {
+        return getNumWorkers(WorkerState.LOST);
+      }
+    });
+
+    return metricsMap;
+  }
+
+  protected int getNumWorkers(WorkerState status) {
+    int numWorkers = 0;
+    for(Worker eachWorker: tajoMasterContext.getResourceManager().getWorkers().values()) {
+      if(eachWorker.getState() == status) {
+        numWorkers++;
+      }
+    }
+
+    return numWorkers;
+  }
+}


[22/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
new file mode 100644
index 0000000..63b50ac
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -0,0 +1,1125 @@
+/**
+ * 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.master.querymaster;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.QueryUnitId;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.engine.planner.logical.StoreTableNode;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.*;
+import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.fragment.FileFragment;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType;
+
+
+/**
+ * SubQuery plays a role in controlling an ExecutionBlock and is a finite state machine.
+ */
+public class SubQuery implements EventHandler<SubQueryEvent> {
+
+  private static final Log LOG = LogFactory.getLog(SubQuery.class);
+
+  private MasterPlan masterPlan;
+  private ExecutionBlock block;
+  private int priority;
+  private Schema schema;
+  private TableMeta meta;
+  private TableStats resultStatistics;
+  private TableStats inputStatistics;
+  private EventHandler<Event> eventHandler;
+  private final AbstractStorageManager sm;
+  private AbstractTaskScheduler taskScheduler;
+  private QueryMasterTask.QueryMasterTaskContext context;
+  private final List<String> diagnostics = new ArrayList<String>();
+
+  private long startTime;
+  private long finishTime;
+
+  volatile Map<QueryUnitId, QueryUnit> tasks = new ConcurrentHashMap<QueryUnitId, QueryUnit>();
+  volatile Map<ContainerId, Container> containers = new ConcurrentHashMap<ContainerId, Container>();
+
+  private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
+  private static final InternalErrorTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+  private static final ContainerLaunchTransition CONTAINER_LAUNCH_TRANSITION = new ContainerLaunchTransition();
+  private static final TaskCompletedTransition TASK_COMPLETED_TRANSITION = new TaskCompletedTransition();
+  private static final AllocatedContainersCancelTransition CONTAINERS_CANCEL_TRANSITION =
+      new AllocatedContainersCancelTransition();
+  private static final SubQueryCompleteTransition SUBQUERY_COMPLETED_TRANSITION =
+      new SubQueryCompleteTransition();
+  private StateMachine<SubQueryState, SubQueryEventType, SubQueryEvent> stateMachine;
+
+  protected static final StateMachineFactory<SubQuery, SubQueryState,
+      SubQueryEventType, SubQueryEvent> stateMachineFactory =
+      new StateMachineFactory <SubQuery, SubQueryState,
+          SubQueryEventType, SubQueryEvent> (SubQueryState.NEW)
+
+          // Transitions from NEW state
+          .addTransition(SubQueryState.NEW,
+              EnumSet.of(SubQueryState.INITED, SubQueryState.ERROR, SubQueryState.SUCCEEDED),
+              SubQueryEventType.SQ_INIT,
+              new InitAndRequestContainer())
+          .addTransition(SubQueryState.NEW, SubQueryState.NEW,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.NEW, SubQueryState.KILLED,
+              SubQueryEventType.SQ_KILL)
+          .addTransition(SubQueryState.NEW, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from INITED state
+          .addTransition(SubQueryState.INITED, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINER_LAUNCH_TRANSITION)
+          .addTransition(SubQueryState.INITED, SubQueryState.INITED,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.INITED, SubQueryState.KILL_WAIT,
+              SubQueryEventType.SQ_KILL)
+          .addTransition(SubQueryState.INITED, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from RUNNING state
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINER_LAUNCH_TRANSITION)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_TASK_COMPLETED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(SubQueryState.RUNNING,
+              EnumSet.of(SubQueryState.SUCCEEDED, SubQueryState.FAILED),
+              SubQueryEventType.SQ_SUBQUERY_COMPLETED,
+              SUBQUERY_COMPLETED_TRANSITION)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_FAILED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.KILL_WAIT,
+              SubQueryEventType.SQ_KILL,
+              new KillTasksTransition())
+          .addTransition(SubQueryState.RUNNING, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+          // Ignore-able Transition
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_START)
+
+          // Transitions from KILL_WAIT state
+          .addTransition(SubQueryState.KILL_WAIT, SubQueryState.KILL_WAIT,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(SubQueryState.KILL_WAIT, SubQueryState.KILL_WAIT,
+              EnumSet.of(SubQueryEventType.SQ_KILL))
+          .addTransition(SubQueryState.KILL_WAIT, SubQueryState.KILL_WAIT,
+              SubQueryEventType.SQ_TASK_COMPLETED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(SubQueryState.KILL_WAIT,
+              EnumSet.of(SubQueryState.SUCCEEDED, SubQueryState.FAILED, SubQueryState.KILLED),
+              SubQueryEventType.SQ_SUBQUERY_COMPLETED,
+              SUBQUERY_COMPLETED_TRANSITION)
+          .addTransition(SubQueryState.KILL_WAIT, SubQueryState.KILL_WAIT,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.KILL_WAIT, SubQueryState.KILL_WAIT,
+              SubQueryEventType.SQ_FAILED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(SubQueryState.KILL_WAIT, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+              // Transitions from SUCCEEDED state
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+              // Ignore-able events
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
+              EnumSet.of(
+                  SubQueryEventType.SQ_START,
+                  SubQueryEventType.SQ_KILL,
+                  SubQueryEventType.SQ_CONTAINER_ALLOCATED))
+
+          // Transitions from FAILED state
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.FAILED, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+          // Ignore-able transitions
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              EnumSet.of(
+                  SubQueryEventType.SQ_START,
+                  SubQueryEventType.SQ_KILL,
+                  SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+                  SubQueryEventType.SQ_FAILED))
+
+          // Transitions from FAILED state
+          .addTransition(SubQueryState.ERROR, SubQueryState.ERROR,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(SubQueryState.ERROR, SubQueryState.ERROR,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          // Ignore-able transitions
+          .addTransition(SubQueryState.ERROR, SubQueryState.ERROR,
+              EnumSet.of(
+                  SubQueryEventType.SQ_START,
+                  SubQueryEventType.SQ_KILL,
+                  SubQueryEventType.SQ_FAILED,
+                  SubQueryEventType.SQ_INTERNAL_ERROR))
+
+          .installTopology();
+
+  private final Lock readLock;
+  private final Lock writeLock;
+
+  private int totalScheduledObjectsCount;
+  private int succeededObjectCount = 0;
+  private int completedTaskCount = 0;
+  private int succeededTaskCount = 0;
+  private int killedObjectCount = 0;
+  private int failedObjectCount = 0;
+  private TaskSchedulerContext schedulerContext;
+
+  public SubQuery(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block, AbstractStorageManager sm) {
+    this.context = context;
+    this.masterPlan = masterPlan;
+    this.block = block;
+    this.sm = sm;
+    this.eventHandler = context.getEventHandler();
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+    stateMachine = stateMachineFactory.make(this);
+  }
+
+  public static boolean isRunningState(SubQueryState state) {
+    return state == SubQueryState.INITED || state == SubQueryState.NEW || state == SubQueryState.RUNNING;
+  }
+
+  public QueryMasterTask.QueryMasterTaskContext getContext() {
+    return context;
+  }
+
+  public MasterPlan getMasterPlan() {
+    return masterPlan;
+  }
+
+  public DataChannel getDataChannel() {
+    return masterPlan.getOutgoingChannels(getId()).iterator().next();
+  }
+
+  public EventHandler<Event> getEventHandler() {
+    return eventHandler;
+  }
+
+  public AbstractTaskScheduler getTaskScheduler() {
+    return taskScheduler;
+  }
+
+  public void setStartTime() {
+    startTime = context.getClock().getTime();
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  public void setFinishTime() {
+    finishTime = context.getClock().getTime();
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public long getFinishTime() {
+    return this.finishTime;
+  }
+
+  public float getTaskProgress() {
+    readLock.lock();
+    try {
+      if (getState() == SubQueryState.NEW) {
+        return 0;
+      } else {
+        return (float)(succeededObjectCount) / (float)totalScheduledObjectsCount;
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public float getProgress() {
+    List<QueryUnit> tempTasks = null;
+    readLock.lock();
+    try {
+      if (getState() == SubQueryState.NEW) {
+        return 0;
+      } else {
+        tempTasks = new ArrayList<QueryUnit>(tasks.values());
+      }
+    } finally {
+      readLock.unlock();
+    }
+
+    float totalProgress = 0.0f;
+    for (QueryUnit eachQueryUnit: tempTasks) {
+      if (eachQueryUnit.getLastAttempt() != null) {
+        totalProgress += eachQueryUnit.getLastAttempt().getProgress();
+      }
+    }
+
+    return totalProgress/(float)tempTasks.size();
+  }
+
+  public int getSucceededObjectCount() {
+    return succeededObjectCount;
+  }
+
+  public int getTotalScheduledObjectsCount() {
+    return totalScheduledObjectsCount;
+  }
+
+  public ExecutionBlock getBlock() {
+    return block;
+  }
+
+  public void addTask(QueryUnit task) {
+    tasks.put(task.getId(), task);
+  }
+
+  /**
+   * It finalizes this subquery. It is only invoked when the subquery is succeeded.
+   */
+  public void complete() {
+    cleanup();
+    finalizeStats();
+    setFinishTime();
+    eventHandler.handle(new SubQueryCompletedEvent(getId(), SubQueryState.SUCCEEDED));
+  }
+
+  /**
+   * It finalizes this subquery. Unlike {@link SubQuery#complete()},
+   * it is invoked when a subquery is abnormally finished.
+   *
+   * @param finalState The final subquery state
+   */
+  public void abort(SubQueryState finalState) {
+    // TODO -
+    // - committer.abortSubQuery(...)
+    // - record SubQuery Finish Time
+    // - CleanUp Tasks
+    // - Record History
+    cleanup();
+    setFinishTime();
+    eventHandler.handle(new SubQueryCompletedEvent(getId(), finalState));
+  }
+
+  public StateMachine<SubQueryState, SubQueryEventType, SubQueryEvent> getStateMachine() {
+    return this.stateMachine;
+  }
+
+  public void setPriority(int priority) {
+    this.priority = priority;
+  }
+
+
+  public int getPriority() {
+    return this.priority;
+  }
+
+  public AbstractStorageManager getStorageManager() {
+    return sm;
+  }
+  
+  public ExecutionBlockId getId() {
+    return block.getId();
+  }
+  
+  public QueryUnit[] getQueryUnits() {
+    return tasks.values().toArray(new QueryUnit[tasks.size()]);
+  }
+  
+  public QueryUnit getQueryUnit(QueryUnitId qid) {
+    return tasks.get(qid);
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  public TableMeta getTableMeta() {
+    return meta;
+  }
+
+  public TableStats getResultStats() {
+    return resultStatistics;
+  }
+
+  public TableStats getInputStats() {
+    return inputStatistics;
+  }
+
+  public List<String> getDiagnostics() {
+    readLock.lock();
+    try {
+      return diagnostics;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  protected void addDiagnostic(String diag) {
+    diagnostics.add(diag);
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(this.getId());
+    return sb.toString();
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof SubQuery) {
+      SubQuery other = (SubQuery)o;
+      return getId().equals(other.getId());
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    return getId().hashCode();
+  }
+  
+  public int compareTo(SubQuery other) {
+    return getId().compareTo(other.getId());
+  }
+
+  public SubQueryState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public static TableStats[] computeStatFromUnionBlock(SubQuery subQuery) {
+    TableStats[] stat = new TableStats[]{new TableStats(), new TableStats()};
+    long[] avgRows = new long[]{0, 0};
+    long[] numBytes = new long[]{0, 0};
+    long[] readBytes = new long[]{0, 0};
+    long[] numRows = new long[]{0, 0};
+    int[] numBlocks = new int[]{0, 0};
+    int[] numOutputs = new int[]{0, 0};
+
+    List<ColumnStats> columnStatses = Lists.newArrayList();
+
+    MasterPlan masterPlan = subQuery.getMasterPlan();
+    Iterator<ExecutionBlock> it = masterPlan.getChilds(subQuery.getBlock()).iterator();
+    while (it.hasNext()) {
+      ExecutionBlock block = it.next();
+      SubQuery childSubQuery = subQuery.context.getSubQuery(block.getId());
+      TableStats[] childStatArray = new TableStats[]{
+          childSubQuery.getInputStats(), childSubQuery.getResultStats()
+      };
+      for (int i = 0; i < 2; i++) {
+        if (childStatArray[i] == null) {
+          continue;
+        }
+        avgRows[i] += childStatArray[i].getAvgRows();
+        numBlocks[i] += childStatArray[i].getNumBlocks();
+        numBytes[i] += childStatArray[i].getNumBytes();
+        readBytes[i] += childStatArray[i].getReadBytes();
+        numOutputs[i] += childStatArray[i].getNumShuffleOutputs();
+        numRows[i] += childStatArray[i].getNumRows();
+      }
+      columnStatses.addAll(childStatArray[1].getColumnStats());
+    }
+
+    for (int i = 0; i < 2; i++) {
+      stat[i].setNumBlocks(numBlocks[i]);
+      stat[i].setNumBytes(numBytes[i]);
+      stat[i].setReadBytes(readBytes[i]);
+      stat[i].setNumShuffleOutputs(numOutputs[i]);
+      stat[i].setNumRows(numRows[i]);
+      stat[i].setAvgRows(avgRows[i]);
+    }
+    stat[1].setColumnStats(columnStatses);
+
+    return stat;
+  }
+
+  private TableStats[] computeStatFromTasks() {
+    List<TableStats> inputStatsList = Lists.newArrayList();
+    List<TableStats> resultStatsList = Lists.newArrayList();
+    for (QueryUnit unit : getQueryUnits()) {
+      resultStatsList.add(unit.getStats());
+      if (unit.getLastAttempt().getInputStats() != null) {
+        inputStatsList.add(unit.getLastAttempt().getInputStats());
+      }
+    }
+    TableStats inputStats = StatisticsUtil.aggregateTableStat(inputStatsList);
+    TableStats resultStats = StatisticsUtil.aggregateTableStat(resultStatsList);
+    return new TableStats[]{inputStats, resultStats};
+  }
+
+  private void stopScheduler() {
+    // If there are launched TaskRunners, send the 'shouldDie' message to all r
+    // via received task requests.
+    if (taskScheduler != null) {
+      taskScheduler.stop();
+    }
+  }
+
+  private void releaseContainers() {
+    // If there are still live TaskRunners, try to kill the containers.
+    eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), containers.values()));
+  }
+
+  public void releaseContainer(ContainerId containerId) {
+    // try to kill the container.
+    ArrayList<Container> list = new ArrayList<Container>();
+    list.add(containers.get(containerId));
+    eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), list));
+  }
+
+  /**
+   * It computes all stats and sets the intermediate result.
+   */
+  private void finalizeStats() {
+    TableStats[] statsArray;
+    if (block.hasUnion()) {
+      statsArray = computeStatFromUnionBlock(this);
+    } else {
+      statsArray = computeStatFromTasks();
+    }
+
+    DataChannel channel = masterPlan.getOutgoingChannels(getId()).get(0);
+    // get default or store type
+    CatalogProtos.StoreType storeType = CatalogProtos.StoreType.CSV; // default setting
+
+    // if store plan (i.e., CREATE or INSERT OVERWRITE)
+    StoreTableNode storeTableNode = PlannerUtil.findTopNode(getBlock().getPlan(), NodeType.STORE);
+    if (storeTableNode != null) {
+      storeType = storeTableNode.getStorageType();
+    }
+    schema = channel.getSchema();
+    meta = CatalogUtil.newTableMeta(storeType, new Options());
+    inputStatistics = statsArray[0];
+    resultStatistics = statsArray[1];
+  }
+
+  @Override
+  public void handle(SubQueryEvent event) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getSubQueryId() + " of type " + event.getType() + ", preState=" + getState());
+    }
+
+    try {
+      writeLock.lock();
+      SubQueryState oldState = getState();
+      try {
+        getStateMachine().doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state", e);
+        eventHandler.handle(new SubQueryEvent(getId(),
+            SubQueryEventType.SQ_INTERNAL_ERROR));
+      }
+
+      // notify the eventhandler of state change
+      if (LOG.isDebugEnabled()) {
+        if (oldState != getState()) {
+          LOG.debug(getId() + " SubQuery Transitioned from " + oldState + " to "
+              + getState());
+        }
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+    taskScheduler.handleTaskRequestEvent(event);
+  }
+
+  private static class InitAndRequestContainer implements MultipleArcTransition<SubQuery,
+      SubQueryEvent, SubQueryState> {
+
+    @Override
+    public SubQueryState transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      subQuery.setStartTime();
+      ExecutionBlock execBlock = subQuery.getBlock();
+      SubQueryState state;
+
+      try {
+        // Union operator does not require actual query processing. It is performed logically.
+        if (execBlock.hasUnion()) {
+          subQuery.finalizeStats();
+          state = SubQueryState.SUCCEEDED;
+        } else {
+          ExecutionBlock parent = subQuery.getMasterPlan().getParent(subQuery.getBlock());
+          DataChannel channel = subQuery.getMasterPlan().getChannel(subQuery.getId(), parent.getId());
+          setShuffleIfNecessary(subQuery, channel);
+          initTaskScheduler(subQuery);
+          schedule(subQuery);
+          subQuery.totalScheduledObjectsCount = subQuery.getTaskScheduler().remainingScheduledObjectNum();
+          LOG.info(subQuery.totalScheduledObjectsCount + " objects are scheduled");
+
+          if (subQuery.getTaskScheduler().remainingScheduledObjectNum() == 0) { // if there is no tasks
+            subQuery.stopScheduler();
+            subQuery.finalizeStats();
+            subQuery.eventHandler.handle(new SubQueryCompletedEvent(subQuery.getId(), SubQueryState.SUCCEEDED));
+            return SubQueryState.SUCCEEDED;
+          } else {
+            subQuery.taskScheduler.start();
+            allocateContainers(subQuery);
+            return SubQueryState.INITED;
+          }
+        }
+      } catch (Exception e) {
+        LOG.error("SubQuery (" + subQuery.getId() + ") ERROR: ", e);
+        subQuery.setFinishTime();
+        subQuery.eventHandler.handle(new SubQueryDiagnosticsUpdateEvent(subQuery.getId(), e.getMessage()));
+        subQuery.eventHandler.handle(new SubQueryCompletedEvent(subQuery.getId(), SubQueryState.ERROR));
+        return SubQueryState.ERROR;
+      }
+
+      return state;
+    }
+
+    private void initTaskScheduler(SubQuery subQuery) throws IOException {
+      TajoConf conf = subQuery.context.getConf();
+      subQuery.schedulerContext = new TaskSchedulerContext(subQuery.context,
+          subQuery.getMasterPlan().isLeaf(subQuery.getId()), subQuery.getId());
+      subQuery.taskScheduler = TaskSchedulerFactory.get(conf, subQuery.schedulerContext, subQuery);
+      subQuery.taskScheduler.init(conf);
+      LOG.info(subQuery.taskScheduler.getName() + " is chosen for the task scheduling for " + subQuery.getId());
+    }
+
+    /**
+     * If a parent block requires a repartition operation, the method sets proper repartition
+     * methods and the number of partitions to a given subquery.
+     */
+    private static void setShuffleIfNecessary(SubQuery subQuery, DataChannel channel) {
+      if (channel.getShuffleType() != ShuffleType.NONE_SHUFFLE) {
+        int numTasks = calculateShuffleOutputNum(subQuery, channel);
+        Repartitioner.setShuffleOutputNumForTwoPhase(subQuery, numTasks, channel);
+      }
+    }
+
+    /**
+     * Getting the total memory of cluster
+     *
+     * @param subQuery
+     * @return mega bytes
+     */
+    private static int getClusterTotalMemory(SubQuery subQuery) {
+      List<TajoMasterProtocol.WorkerResourceProto> workers =
+          subQuery.context.getQueryMasterContext().getQueryMaster().getAllWorker();
+
+      int totalMem = 0;
+      for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
+        totalMem += worker.getMemoryMB();
+      }
+      return totalMem;
+    }
+    /**
+     * Getting the desire number of partitions according to the volume of input data.
+     * This method is only used to determine the partition key number of hash join or aggregation.
+     *
+     * @param subQuery
+     * @return
+     */
+    public static int calculateShuffleOutputNum(SubQuery subQuery, DataChannel channel) {
+      TajoConf conf = subQuery.context.getConf();
+      MasterPlan masterPlan = subQuery.getMasterPlan();
+      ExecutionBlock parent = masterPlan.getParent(subQuery.getBlock());
+
+      GroupbyNode grpNode = null;
+      if (parent != null) {
+        grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.GROUP_BY);
+      }
+
+      // Is this subquery the first step of join?
+      if (parent != null && parent.getScanNodes().length == 2) {
+        List<ExecutionBlock> childs = masterPlan.getChilds(parent);
+
+        // for outer
+        ExecutionBlock outer = childs.get(0);
+        long outerVolume = getInputVolume(subQuery.masterPlan, subQuery.context, outer);
+
+        // for inner
+        ExecutionBlock inner = childs.get(1);
+        long innerVolume = getInputVolume(subQuery.masterPlan, subQuery.context, inner);
+        LOG.info(subQuery.getId() + ", Outer volume: " + Math.ceil((double) outerVolume / 1048576) + "MB, "
+            + "Inner volume: " + Math.ceil((double) innerVolume / 1048576) + "MB");
+
+        long bigger = Math.max(outerVolume, innerVolume);
+
+        int mb = (int) Math.ceil((double) bigger / 1048576);
+        LOG.info(subQuery.getId() + ", Bigger Table's volume is approximately " + mb + " MB");
+
+        int taskNum = (int) Math.ceil((double) mb /
+            conf.getIntVar(ConfVars.DIST_QUERY_JOIN_PARTITION_VOLUME));
+
+        int totalMem = getClusterTotalMemory(subQuery);
+        LOG.info(subQuery.getId() + ", Total memory of cluster is " + totalMem + " MB");
+        int slots = Math.max(totalMem / conf.getIntVar(ConfVars.TASK_DEFAULT_MEMORY), 1);
+
+        // determine the number of task
+        taskNum = Math.min(taskNum, slots);
+        LOG.info(subQuery.getId() + ", The determined number of join partitions is " + taskNum);
+
+        return taskNum;
+
+        // Is this subquery the first step of group-by?
+      } else if (grpNode != null) {
+
+        if (grpNode.getGroupingColumns().length == 0) {
+          return 1;
+        } else {
+          long volume = getInputVolume(subQuery.masterPlan, subQuery.context, subQuery.block);
+
+          int mb = (int) Math.ceil((double) volume / 1048576);
+          LOG.info(subQuery.getId() + ", Table's volume is approximately " + mb + " MB");
+          // determine the number of task
+          int taskNumBySize = (int) Math.ceil((double) mb /
+              conf.getIntVar(ConfVars.DIST_QUERY_GROUPBY_PARTITION_VOLUME));
+
+          int totalMem = getClusterTotalMemory(subQuery);
+
+          LOG.info(subQuery.getId() + ", Total memory of cluster is " + totalMem + " MB");
+          int slots = Math.max(totalMem / conf.getIntVar(ConfVars.TASK_DEFAULT_MEMORY), 1);
+          int taskNum = Math.min(taskNumBySize, slots); //Maximum partitions
+          LOG.info(subQuery.getId() + ", The determined number of aggregation partitions is " + taskNum);
+          return taskNum;
+        }
+      } else {
+        LOG.info("============>>>>> Unexpected Case! <<<<<================");
+        long volume = getInputVolume(subQuery.masterPlan, subQuery.context, subQuery.block);
+
+        int mb = (int) Math.ceil((double)volume / 1048576);
+        LOG.info(subQuery.getId() + ", Table's volume is approximately " + mb + " MB");
+        // determine the number of task per 128MB
+        int taskNum = (int) Math.ceil((double)mb / 128);
+        LOG.info(subQuery.getId() + ", The determined number of partitions is " + taskNum);
+        return taskNum;
+      }
+    }
+
+    private static void schedule(SubQuery subQuery) throws IOException {
+      MasterPlan masterPlan = subQuery.getMasterPlan();
+      ExecutionBlock execBlock = subQuery.getBlock();
+      if (subQuery.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
+        scheduleFragmentsForLeafQuery(subQuery);
+      } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join
+        Repartitioner.scheduleFragmentsForJoinQuery(subQuery.schedulerContext, subQuery);
+      } else { // Case 3: Others (Sort or Aggregation)
+        int numTasks = getNonLeafTaskNum(subQuery);
+        Repartitioner.scheduleFragmentsForNonLeafTasks(subQuery.schedulerContext, masterPlan, subQuery, numTasks);
+      }
+    }
+
+    /**
+     * Getting the desire number of tasks according to the volume of input data
+     *
+     * @param subQuery
+     * @return
+     */
+    public static int getNonLeafTaskNum(SubQuery subQuery) {
+      // Getting intermediate data size
+      long volume = getInputVolume(subQuery.getMasterPlan(), subQuery.context, subQuery.getBlock());
+
+      int mb = (int) Math.ceil((double)volume / 1048576);
+      LOG.info("Table's volume is approximately " + mb + " MB");
+      // determine the number of task per 64MB
+      int maxTaskNum = Math.max(1, (int) Math.ceil((double)mb / 64));
+      LOG.info("The determined number of non-leaf tasks is " + maxTaskNum);
+      return maxTaskNum;
+    }
+
+    public static long getInputVolume(MasterPlan masterPlan, QueryMasterTask.QueryMasterTaskContext context,
+                                      ExecutionBlock execBlock) {
+      Map<String, TableDesc> tableMap = context.getTableDescMap();
+      if (masterPlan.isLeaf(execBlock)) {
+        ScanNode[] outerScans = execBlock.getScanNodes();
+        long maxVolume = 0;
+        for (ScanNode eachScanNode: outerScans) {
+          TableStats stat = tableMap.get(eachScanNode.getCanonicalName()).getStats();
+          if (stat.getNumBytes() > maxVolume) {
+            maxVolume = stat.getNumBytes();
+          }
+        }
+        return maxVolume;
+      } else {
+        long aggregatedVolume = 0;
+        for (ExecutionBlock childBlock : masterPlan.getChilds(execBlock)) {
+          SubQuery subquery = context.getSubQuery(childBlock.getId());
+          if (subquery == null || subquery.getState() != SubQueryState.SUCCEEDED) {
+            aggregatedVolume += getInputVolume(masterPlan, context, childBlock);
+          } else {
+            aggregatedVolume += subquery.getResultStats().getNumBytes();
+          }
+        }
+
+        return aggregatedVolume;
+      }
+    }
+
+    public static void allocateContainers(SubQuery subQuery) {
+      ExecutionBlock execBlock = subQuery.getBlock();
+
+      //TODO consider disk slot
+      int requiredMemoryMBPerTask = 512;
+
+      int numRequest = subQuery.getContext().getResourceAllocator().calculateNumRequestContainers(
+          subQuery.getContext().getQueryMasterContext().getWorkerContext(),
+          subQuery.schedulerContext.getEstimatedTaskNum(),
+          requiredMemoryMBPerTask
+      );
+
+      final Resource resource = Records.newRecord(Resource.class);
+
+      resource.setMemory(requiredMemoryMBPerTask);
+
+      LOG.info("Request Container for " + subQuery.getId() + " containers=" + numRequest);
+
+      Priority priority = Records.newRecord(Priority.class);
+      priority.setPriority(subQuery.getPriority());
+      ContainerAllocationEvent event =
+          new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ,
+              subQuery.getId(), priority, resource, numRequest,
+              subQuery.masterPlan.isLeaf(execBlock), 0.0f);
+      subQuery.eventHandler.handle(event);
+    }
+
+    private static void scheduleFragmentsForLeafQuery(SubQuery subQuery) throws IOException {
+      ExecutionBlock execBlock = subQuery.getBlock();
+      ScanNode[] scans = execBlock.getScanNodes();
+      Preconditions.checkArgument(scans.length == 1, "Must be Scan Query");
+      ScanNode scan = scans[0];
+      TableDesc table = subQuery.context.getTableDescMap().get(scan.getCanonicalName());
+
+      Collection<FileFragment> fragments;
+      TableMeta meta = table.getMeta();
+
+      // Depending on scanner node's type, it creates fragments. If scan is for
+      // a partitioned table, It will creates lots fragments for all partitions.
+      // Otherwise, it creates at least one fragments for a table, which may
+      // span a number of blocks or possibly consists of a number of files.
+      if (scan.getType() == NodeType.PARTITIONS_SCAN) {
+        fragments = Repartitioner.getFragmentsFromPartitionedTable(subQuery.getStorageManager(), scan, table);
+      } else {
+        Path inputPath = table.getPath();
+        fragments = subQuery.getStorageManager().getSplits(scan.getCanonicalName(), meta, table.getSchema(), inputPath);
+      }
+
+      SubQuery.scheduleFragments(subQuery, fragments);
+      if (subQuery.getTaskScheduler() instanceof DefaultTaskScheduler) {
+        //Leaf task of DefaultTaskScheduler should be fragment size
+        // EstimatedTaskNum determined number of initial container
+        subQuery.schedulerContext.setTaskSize(fragments.size());
+        subQuery.schedulerContext.setEstimatedTaskNum(fragments.size());
+      } else {
+        TajoConf conf = subQuery.context.getConf();
+        subQuery.schedulerContext.setTaskSize(conf.getIntVar(ConfVars.TASK_DEFAULT_SIZE) * 1024 * 1024);
+        int estimatedTaskNum = (int) Math.ceil((double) table.getStats().getNumBytes() /
+            (double) subQuery.schedulerContext.getTaskSize());
+        subQuery.schedulerContext.setEstimatedTaskNum(estimatedTaskNum);
+      }
+    }
+  }
+
+  public static void scheduleFragment(SubQuery subQuery, FileFragment fragment) {
+    subQuery.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+        subQuery.getId(), fragment));
+  }
+
+
+  public static void scheduleFragments(SubQuery subQuery, Collection<FileFragment> fragments) {
+    for (FileFragment eachFragment : fragments) {
+      scheduleFragment(subQuery, eachFragment);
+    }
+  }
+
+  public static void scheduleFragments(SubQuery subQuery, Collection<FileFragment> leftFragments,
+                                       Collection<FileFragment> broadcastFragments) {
+    for (FileFragment eachLeafFragment : leftFragments) {
+      scheduleFragment(subQuery, eachLeafFragment, broadcastFragments);
+    }
+  }
+
+  public static void scheduleFragment(SubQuery subQuery,
+                                      FileFragment leftFragment, Collection<FileFragment> rightFragments) {
+    subQuery.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+        subQuery.getId(), leftFragment, rightFragments));
+  }
+
+  public static void scheduleFetches(SubQuery subQuery, Map<String, List<URI>> fetches) {
+    subQuery.taskScheduler.handle(new FetchScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+        subQuery.getId(), fetches));
+  }
+
+  public static QueryUnit newEmptyQueryUnit(TaskSchedulerContext schedulerContext,
+                                            QueryUnitAttemptScheduleContext queryUnitContext,
+                                            SubQuery subQuery, int taskId) {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    QueryUnit unit = new QueryUnit(schedulerContext.getMasterContext().getConf(),
+        queryUnitContext,
+        QueryIdFactory.newQueryUnitId(schedulerContext.getBlockId(), taskId),
+        schedulerContext.isLeafQuery(), subQuery.eventHandler);
+    unit.setLogicalPlan(execBlock.getPlan());
+    subQuery.addTask(unit);
+    return unit;
+  }
+
+  private static class ContainerLaunchTransition
+      implements SingleArcTransition<SubQuery, SubQueryEvent> {
+
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent event) {
+      try {
+        SubQueryContainerAllocationEvent allocationEvent =
+            (SubQueryContainerAllocationEvent) event;
+        for (Container container : allocationEvent.getAllocatedContainer()) {
+          ContainerId cId = container.getId();
+          if (subQuery.containers.containsKey(cId)) {
+            subQuery.eventHandler.handle(new SubQueryDiagnosticsUpdateEvent(subQuery.getId(),
+                "Duplicated containers are allocated: " + cId.toString()));
+            subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_INTERNAL_ERROR));
+          }
+          subQuery.containers.put(cId, container);
+        }
+        LOG.info("SubQuery (" + subQuery.getId() + ") has " + subQuery.containers.size() + " containers!");
+        subQuery.eventHandler.handle(
+            new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_LAUNCH,
+                subQuery.getId(), allocationEvent.getAllocatedContainer()));
+
+        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_START));
+      } catch (Throwable t) {
+        subQuery.eventHandler.handle(new SubQueryDiagnosticsUpdateEvent(subQuery.getId(),
+            ExceptionUtils.getStackTrace(t)));
+        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_INTERNAL_ERROR));
+      }
+    }
+  }
+
+  /**
+   * It is used in KILL_WAIT state against Contained Allocated event.
+   * It just returns allocated containers to resource manager.
+   */
+  private static class AllocatedContainersCancelTransition implements SingleArcTransition<SubQuery, SubQueryEvent> {
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent event) {
+      try {
+        SubQueryContainerAllocationEvent allocationEvent =
+            (SubQueryContainerAllocationEvent) event;
+        subQuery.eventHandler.handle(
+            new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP,
+                subQuery.getId(), allocationEvent.getAllocatedContainer()));
+        LOG.info(String.format("[%s] %d allocated containers are canceled",
+            subQuery.getId().toString(),
+            allocationEvent.getAllocatedContainer().size()));
+      } catch (Throwable t) {
+        subQuery.eventHandler.handle(new SubQueryDiagnosticsUpdateEvent(subQuery.getId(),
+            ExceptionUtils.getStackTrace(t)));
+        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_INTERNAL_ERROR));
+      }
+    }
+  }
+
+  private static class TaskCompletedTransition implements SingleArcTransition<SubQuery, SubQueryEvent> {
+
+    @Override
+    public void transition(SubQuery subQuery,
+                           SubQueryEvent event) {
+      SubQueryTaskEvent taskEvent = (SubQueryTaskEvent) event;
+      QueryUnit task = subQuery.getQueryUnit(taskEvent.getTaskId());
+
+      if (task == null) { // task failed
+        LOG.error(String.format("Task %s is absent", taskEvent.getTaskId()));
+        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_FAILED));
+      } else {
+        subQuery.completedTaskCount++;
+
+        if (taskEvent.getState() == TaskState.SUCCEEDED) {
+//          if (task.isLeafTask()) {
+//            subQuery.succeededObjectCount += task.getTotalFragmentNum();
+//          } else {
+//            subQuery.succeededObjectCount++;
+//          }
+          subQuery.succeededObjectCount++;
+        } else if (task.getState() == TaskState.KILLED) {
+//          if (task.isLeafTask()) {
+//            subQuery.killedObjectCount += task.getTotalFragmentNum();
+//          } else {
+//            subQuery.killedObjectCount++;
+//          }
+          subQuery.killedObjectCount++;
+        } else if (task.getState() == TaskState.FAILED) {
+//          if (task.isLeafTask()) {
+//            subQuery.failedObjectCount+= task.getTotalFragmentNum();
+//          } else {
+//            subQuery.failedObjectCount++;
+//          }
+          subQuery.failedObjectCount++;
+          // if at least one task is failed, try to kill all tasks.
+          subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_KILL));
+        }
+
+        LOG.info(String.format("[%s] Task Completion Event (Total: %d, Success: %d, Killed: %d, Failed: %d",
+            subQuery.getId(),
+            subQuery.getTotalScheduledObjectsCount(),
+            subQuery.succeededObjectCount,
+            subQuery.killedObjectCount,
+            subQuery.failedObjectCount));
+
+        if (subQuery.totalScheduledObjectsCount ==
+            subQuery.succeededObjectCount + subQuery.killedObjectCount + subQuery.failedObjectCount) {
+          subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_SUBQUERY_COMPLETED));
+        }
+      }
+    }
+  }
+
+  private static class KillTasksTransition implements SingleArcTransition<SubQuery, SubQueryEvent> {
+
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      subQuery.getTaskScheduler().stop();
+      for (QueryUnit queryUnit : subQuery.getQueryUnits()) {
+        subQuery.eventHandler.handle(new TaskEvent(queryUnit.getId(), TaskEventType.T_KILL));
+      }
+    }
+  }
+
+  private void cleanup() {
+    stopScheduler();
+    releaseContainers();
+  }
+
+  private static class SubQueryCompleteTransition
+      implements MultipleArcTransition<SubQuery, SubQueryEvent, SubQueryState> {
+
+    @Override
+    public SubQueryState transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      // TODO - Commit subQuery & do cleanup
+      // TODO - records succeeded, failed, killed completed task
+      // TODO - records metrics
+      try {
+        LOG.info(String.format("subQuery completed - %s (total=%d, success=%d, killed=%d)",
+            subQuery.getId().toString(),
+            subQuery.getTotalScheduledObjectsCount(),
+            subQuery.getSucceededObjectCount(),
+            subQuery.killedObjectCount));
+
+        if (subQuery.killedObjectCount > 0 || subQuery.failedObjectCount > 0) {
+          if (subQuery.failedObjectCount > 0) {
+            subQuery.abort(SubQueryState.FAILED);
+            return SubQueryState.FAILED;
+          } else if (subQuery.killedObjectCount > 0) {
+            subQuery.abort(SubQueryState.KILLED);
+            return SubQueryState.KILLED;
+          } else {
+            LOG.error("Invalid State " + subQuery.getState() + " State");
+            subQuery.abort(SubQueryState.ERROR);
+            return SubQueryState.ERROR;
+          }
+        } else {
+          subQuery.complete();
+          return SubQueryState.SUCCEEDED;
+        }
+      } catch (Throwable t) {
+        LOG.error(t);
+        subQuery.abort(SubQueryState.ERROR);
+        return SubQueryState.ERROR;
+      }
+    }
+  }
+
+  private static class DiagnosticsUpdateTransition implements SingleArcTransition<SubQuery, SubQueryEvent> {
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent event) {
+      subQuery.addDiagnostic(((SubQueryDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
+    }
+  }
+
+  private static class InternalErrorTransition implements SingleArcTransition<SubQuery, SubQueryEvent> {
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      subQuery.abort(SubQueryState.ERROR);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
new file mode 100644
index 0000000..effcfde
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
@@ -0,0 +1,30 @@
+/**
+ * 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.master.querymaster;
+
+public enum SubQueryState {
+  NEW,
+  INITED,
+  RUNNING,
+  SUCCEEDED,
+  FAILED,
+  KILL_WAIT,
+  KILLED,
+  ERROR
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java
new file mode 100644
index 0000000..a995058
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java
@@ -0,0 +1,84 @@
+/**
+ * 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.master.rm;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.tajo.QueryId;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+
+/**
+ * It's a worker resource manager context. It contains all context data about TajoWorkerResourceManager.
+ */
+public class TajoRMContext {
+
+  final Dispatcher rmDispatcher;
+
+  /** map between workerIds and running workers */
+  private final ConcurrentMap<String, Worker> workers = new ConcurrentHashMap<String, Worker>();
+
+  /** map between workerIds and inactive workers */
+  private final ConcurrentMap<String, Worker> inactiveWorkers = new ConcurrentHashMap<String, Worker>();
+
+  /** map between queryIds and query master ContainerId */
+  private final ConcurrentMap<QueryId, ContainerIdProto> qmContainerMap = Maps.newConcurrentMap();
+
+  private final Set<String> liveQueryMasterWorkerResources =
+      Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
+
+  public TajoRMContext(Dispatcher dispatcher) {
+    this.rmDispatcher = dispatcher;
+  }
+
+  public Dispatcher getDispatcher() {
+    return rmDispatcher;
+  }
+
+  /**
+   * @return The Map for active workers
+   */
+  public ConcurrentMap<String, Worker> getWorkers() {
+    return workers;
+  }
+
+  /**
+   * @return The Map for inactive workers
+   */
+  public ConcurrentMap<String, Worker> getInactiveWorkers() {
+    return inactiveWorkers;
+  }
+
+  /**
+   *
+   * @return The Map for query master containers
+   */
+  public ConcurrentMap<QueryId, ContainerIdProto> getQueryMasterContainer() {
+    return qmContainerMap;
+  }
+
+  public Set<String> getQueryMasterWorker() {
+    return liveQueryMasterWorkerResources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java
new file mode 100644
index 0000000..4bd7adb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java
@@ -0,0 +1,253 @@
+/**
+ * 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.master.rm;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoResourceTrackerProtocol;
+import org.apache.tajo.rpc.AsyncRpcServer;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.ProtoUtil;
+
+import java.io.IOError;
+import java.net.InetSocketAddress;
+
+import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeatResponse;
+import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeatResponse.Builder;
+import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.NodeHeartbeat;
+import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService;
+
+/**
+ * It receives pings that workers periodically send. The ping messages contains the worker resources and their statuses.
+ * From ping messages, {@link TajoResourceTracker} tracks the recent status of all workers.
+ *
+ * In detail, it has two main roles as follows:
+ *
+ * <ul>
+ *   <li>Membership management for nodes which join to a Tajo cluster</li>
+ *   <ul>
+ *    <li>Register - It receives the ping from a new worker. It registers the worker.</li>
+ *    <li>Unregister - It unregisters a worker who does not send ping for some expiry time.</li>
+ *   <ul>
+ *   <li>Status Update - It updates the status of all participating workers</li>
+ * </ul>
+ */
+public class TajoResourceTracker extends AbstractService implements TajoResourceTrackerProtocolService.Interface {
+  /** Class logger */
+  private Log LOG = LogFactory.getLog(TajoResourceTracker.class);
+  /** the context of TajoWorkerResourceManager */
+  private final TajoRMContext rmContext;
+  /** Liveliness monitor which checks ping expiry times of workers */
+  private final WorkerLivelinessMonitor workerLivelinessMonitor;
+
+  /** RPC server for worker resource tracker */
+  private AsyncRpcServer server;
+  /** The bind address of RPC server of worker resource tracker */
+  private InetSocketAddress bindAddress;
+
+  public TajoResourceTracker(TajoRMContext rmContext, WorkerLivelinessMonitor workerLivelinessMonitor) {
+    super(TajoResourceTracker.class.getSimpleName());
+    this.rmContext = rmContext;
+    this.workerLivelinessMonitor = workerLivelinessMonitor;
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    Preconditions.checkArgument(conf instanceof TajoConf, "Configuration must be a TajoConf instance");
+    TajoConf systemConf = (TajoConf) conf;
+
+    String confMasterServiceAddr = systemConf.getVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS);
+    InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr);
+
+    try {
+      server = new AsyncRpcServer(TajoResourceTrackerProtocol.class, this, initIsa, 3);
+    } catch (Exception e) {
+      LOG.error(e);
+      throw new IOError(e);
+    }
+
+    server.start();
+    bindAddress = NetUtils.getConnectAddress(server.getListenAddress());
+    // Set actual bind address to the systemConf
+    systemConf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress));
+
+    LOG.info("TajoResourceTracker starts up (" + this.bindAddress + ")");
+    super.start();
+  }
+
+  @Override
+  public void serviceStop() {
+    // server can be null if some exception occurs before the rpc server starts up.
+    if(server != null) {
+      server.shutdown();
+      server = null;
+    }
+    super.stop();
+  }
+
+  /** The response builder */
+  private static final Builder builder = TajoHeartbeatResponse.newBuilder().setHeartbeatResult(ProtoUtil.TRUE);
+
+  private static WorkerStatusEvent createStatusEvent(String workerKey, NodeHeartbeat heartbeat) {
+    return new WorkerStatusEvent(
+        workerKey,
+        heartbeat.getServerStatus().getRunningTaskNum(),
+        heartbeat.getServerStatus().getJvmHeap().getMaxHeap(),
+        heartbeat.getServerStatus().getJvmHeap().getFreeHeap(),
+        heartbeat.getServerStatus().getJvmHeap().getTotalHeap());
+  }
+
+  @Override
+  public void heartbeat(
+      RpcController controller,
+      NodeHeartbeat heartbeat,
+      RpcCallback<TajoHeartbeatResponse> done) {
+
+    try {
+      // get a workerId from the heartbeat
+      String workerId = createWorkerId(heartbeat);
+
+      if(rmContext.getWorkers().containsKey(workerId)) { // if worker is running
+
+        // status update
+        rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(workerId, heartbeat));
+        // refresh ping
+        workerLivelinessMonitor.receivedPing(workerId);
+
+      } else if (rmContext.getInactiveWorkers().containsKey(workerId)) { // worker was inactive
+
+        // remove the inactive worker from the list of inactive workers.
+        Worker worker = rmContext.getInactiveWorkers().remove(workerId);
+        workerLivelinessMonitor.unregister(worker.getWorkerId());
+
+        // create new worker instance
+        Worker newWorker = createWorkerResource(heartbeat);
+        String newWorkerId = newWorker.getWorkerId();
+        // add the new worker to the list of active workers
+        rmContext.getWorkers().putIfAbsent(newWorkerId, newWorker);
+
+        // Transit the worker to RUNNING
+        rmContext.getDispatcher().getEventHandler().handle(new WorkerEvent(newWorkerId, WorkerEventType.STARTED));
+        // register the worker to the liveliness monitor
+        workerLivelinessMonitor.register(newWorkerId);
+
+      } else { // if new worker pings firstly
+
+        // create new worker instance
+        Worker newWorker = createWorkerResource(heartbeat);
+        Worker oldWorker = rmContext.getWorkers().putIfAbsent(workerId, newWorker);
+
+        if (oldWorker == null) {
+          // Transit the worker to RUNNING
+          rmContext.rmDispatcher.getEventHandler().handle(new WorkerEvent(workerId, WorkerEventType.STARTED));
+        } else {
+          LOG.info("Reconnect from the node at: " + workerId);
+          workerLivelinessMonitor.unregister(workerId);
+          rmContext.getDispatcher().getEventHandler().handle(new WorkerReconnectEvent(workerId, newWorker));
+        }
+
+        workerLivelinessMonitor.register(workerId);
+      }
+
+    } finally {
+      builder.setClusterResourceSummary(getClusterResourceSummary());
+      done.run(builder.build());
+    }
+  }
+
+  private static final String createWorkerId(NodeHeartbeat heartbeat) {
+    return heartbeat.getTajoWorkerHost() + ":" + heartbeat.getTajoQueryMasterPort() + ":" + heartbeat.getPeerRpcPort();
+  }
+
+  private Worker createWorkerResource(NodeHeartbeat request) {
+    boolean queryMasterMode = request.getServerStatus().getQueryMasterMode().getValue();
+    boolean taskRunnerMode = request.getServerStatus().getTaskRunnerMode().getValue();
+
+    WorkerResource workerResource = new WorkerResource();
+    workerResource.setQueryMasterMode(queryMasterMode);
+    workerResource.setTaskRunnerMode(taskRunnerMode);
+
+    if(request.getServerStatus() != null) {
+      workerResource.setMemoryMB(request.getServerStatus().getMemoryResourceMB());
+      workerResource.setCpuCoreSlots(request.getServerStatus().getSystem().getAvailableProcessors());
+      workerResource.setDiskSlots(request.getServerStatus().getDiskSlots());
+      workerResource.setNumRunningTasks(request.getServerStatus().getRunningTaskNum());
+      workerResource.setMaxHeap(request.getServerStatus().getJvmHeap().getMaxHeap());
+      workerResource.setFreeHeap(request.getServerStatus().getJvmHeap().getFreeHeap());
+      workerResource.setTotalHeap(request.getServerStatus().getJvmHeap().getTotalHeap());
+    } else {
+      workerResource.setMemoryMB(4096);
+      workerResource.setDiskSlots(4);
+      workerResource.setCpuCoreSlots(4);
+    }
+
+    Worker worker = new Worker(rmContext, workerResource);
+    worker.setHostName(request.getTajoWorkerHost());
+    worker.setHttpPort(request.getTajoWorkerHttpPort());
+    worker.setPeerRpcPort(request.getPeerRpcPort());
+    worker.setQueryMasterPort(request.getTajoQueryMasterPort());
+    worker.setClientPort(request.getTajoWorkerClientPort());
+    worker.setPullServerPort(request.getTajoWorkerPullServerPort());
+    return worker;
+  }
+
+  public TajoMasterProtocol.ClusterResourceSummary getClusterResourceSummary() {
+    int totalDiskSlots = 0;
+    int totalCpuCoreSlots = 0;
+    int totalMemoryMB = 0;
+
+    int totalAvailableDiskSlots = 0;
+    int totalAvailableCpuCoreSlots = 0;
+    int totalAvailableMemoryMB = 0;
+
+    synchronized(rmContext) {
+      for(String eachWorker: rmContext.getWorkers().keySet()) {
+        Worker worker = rmContext.getWorkers().get(eachWorker);
+        WorkerResource resource = worker.getResource();
+        if(worker != null) {
+          totalMemoryMB += resource.getMemoryMB();
+          totalAvailableMemoryMB += resource.getAvailableMemoryMB();
+
+          totalDiskSlots += resource.getDiskSlots();
+          totalAvailableDiskSlots += resource.getAvailableDiskSlots();
+
+          totalCpuCoreSlots += resource.getCpuCoreSlots();
+          totalAvailableCpuCoreSlots += resource.getAvailableCpuCoreSlots();
+        }
+      }
+    }
+
+    return TajoMasterProtocol.ClusterResourceSummary.newBuilder()
+        .setNumWorkers(rmContext.getWorkers().size())
+        .setTotalCpuCoreSlots(totalCpuCoreSlots)
+        .setTotalDiskSlots(totalDiskSlots)
+        .setTotalMemoryMB(totalMemoryMB)
+        .setTotalAvailableCpuCoreSlots(totalAvailableCpuCoreSlots)
+        .setTotalAvailableDiskSlots(totalAvailableDiskSlots)
+        .setTotalAvailableMemoryMB(totalAvailableMemoryMB)
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
new file mode 100644
index 0000000..4d6cbd2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
@@ -0,0 +1,100 @@
+/**
+ * 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.master.rm;
+
+import org.apache.hadoop.yarn.api.records.*;
+
+public class TajoWorkerContainer extends Container {
+  ContainerId id;
+  NodeId nodeId;
+  Worker worker;
+
+  public Worker getWorkerResource() {
+    return worker;
+  }
+
+  public void setWorkerResource(Worker workerResource) {
+    this.worker = workerResource;
+  }
+
+  @Override
+  public ContainerId getId() {
+    return id;
+  }
+
+  @Override
+  public void setId(ContainerId id) {
+    this.id = id;
+  }
+
+  @Override
+  public NodeId getNodeId() {
+    return nodeId;
+  }
+
+  @Override
+  public void setNodeId(NodeId nodeId) {
+    this.nodeId = nodeId;
+  }
+
+  @Override
+  public String getNodeHttpAddress() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setNodeHttpAddress(String nodeHttpAddress) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public Resource getResource() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setResource(Resource resource) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public Priority getPriority() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setPriority(Priority priority) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public Token getContainerToken() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setContainerToken(Token containerToken) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public int compareTo(Container container) {
+    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
new file mode 100644
index 0000000..634ad2b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
@@ -0,0 +1,93 @@
+/**
+ * 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.master.rm;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+
+public class TajoWorkerContainerId extends ContainerId {
+  ApplicationAttemptId applicationAttemptId;
+  int id;
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return applicationAttemptId;
+  }
+
+  @Override
+  public void setApplicationAttemptId(ApplicationAttemptId atId) {
+    this.applicationAttemptId = atId;
+  }
+
+  @Override
+  public int getId() {
+    return id;
+  }
+
+  @Override
+  public void setId(int id) {
+    this.id = id;
+  }
+
+  public YarnProtos.ContainerIdProto getProto() {
+    YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder()
+        .setClusterTimestamp(applicationAttemptId.getApplicationId().getClusterTimestamp())
+        .setId(applicationAttemptId.getApplicationId().getId())
+        .build();
+
+    YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder()
+        .setAttemptId(applicationAttemptId.getAttemptId())
+        .setApplicationId(appIdProto)
+        .build();
+
+    return YarnProtos.ContainerIdProto.newBuilder()
+        .setAppAttemptId(attemptIdProto)
+        .setAppId(appIdProto)
+        .setId(id)
+        .build();
+  }
+
+  public static YarnProtos.ContainerIdProto getContainerIdProto(ContainerId containerId) {
+    if(containerId instanceof TajoWorkerContainerId) {
+      return ((TajoWorkerContainerId)containerId).getProto();
+    } else {
+      YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder()
+          .setClusterTimestamp(containerId.getApplicationAttemptId().getApplicationId().getClusterTimestamp())
+          .setId(containerId.getApplicationAttemptId().getApplicationId().getId())
+          .build();
+
+      YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder()
+          .setAttemptId(containerId.getApplicationAttemptId().getAttemptId())
+          .setApplicationId(appIdProto)
+          .build();
+
+      return YarnProtos.ContainerIdProto.newBuilder()
+          .setAppAttemptId(attemptIdProto)
+          .setAppId(appIdProto)
+          .setId(containerId.getId())
+          .build();
+    }
+  }
+
+  @Override
+  protected void build() {
+
+  }
+}


[08/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
new file mode 100644
index 0000000..22a178f
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
@@ -0,0 +1,206 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.eval;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.exception.NoSuchFunctionException;
+import org.apache.tajo.datum.TimestampDatum;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+public class TestSQLExpression extends ExprTestBase {
+
+  @Test
+  public void testQuotedIdentifiers() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("컬럼1", TEXT);
+    schema.addColumn("컬럼2", TEXT);
+    testEval(schema, "테이블1", "123,234", "select \"컬럼1\"::float, cast (\"컬럼2\" as float4) as a from \"테이블1\"",
+        new String[]{"123.0", "234.0"});
+    testEval(schema,
+        "테이블1", "123,234", "select char_length(\"컬럼1\"), \"컬럼2\"::float4 as \"별명1\" from \"테이블1\"",
+        new String[]{"3", "234.0"});
+  }
+
+  @Test(expected = NoSuchFunctionException.class)
+  public void testNoSuchFunction() throws IOException {
+    testSimpleEval("select test123('abc') col1 ", new String[]{"abc"});
+  }
+
+  @Test
+  public void testCast() throws IOException {
+    testSimpleEval("select cast (1 as char)", new String[] {"1"});
+    testSimpleEval("select cast (119 as char)", new String[] {"1"});
+
+    testSimpleEval("select cast (1 as int2)", new String[ ]{"1"});
+    testSimpleEval("select cast (1 as int4)", new String[] {"1"});
+    testSimpleEval("select cast (1 as int8)", new String[] {"1"});
+    testSimpleEval("select cast (1 as float)", new String[] {"1.0"});
+    testSimpleEval("select cast (1 as double)", new String[] {"1.0"});
+    testSimpleEval("select cast (1 as text)", new String[] {"1"});
+
+    testSimpleEval("select cast ('123' as int2)", new String[] {"123"});
+    testSimpleEval("select cast ('123' as int4)", new String[] {"123"});
+    testSimpleEval("select cast ('123' as int8)", new String[] {"123"});
+    testSimpleEval("select cast ('123' as float)", new String[] {"123.0"});
+    testSimpleEval("select cast ('123' as double)", new String[] {"123.0"});
+    testSimpleEval("select cast ('123' as text)", new String[] {"123"});
+
+    testSimpleEval("select 123::int2", new String[] {"123"});
+    testSimpleEval("select 123::int4", new String[] {"123"});
+    testSimpleEval("select 123::int8", new String[] {"123"});
+    testSimpleEval("select 123::float", new String[] {"123.0"});
+    testSimpleEval("select 123::double", new String[] {"123.0"});
+    testSimpleEval("select 123::text", new String[] {"123"});
+
+    testSimpleEval("select 123.0::float", new String[] {"123.0"});
+    testSimpleEval("select 123.0::double", new String[] {"123.0"});
+
+    testSimpleEval("select '123'::int", new String[] {"123"});
+    testSimpleEval("select '123'::double", new String[] {"123.0"});
+  }
+
+  @Test
+  public void testCastWithNestedFunction() throws IOException {
+    int timestamp = (int) (System.currentTimeMillis() / 1000);
+    TimestampDatum expected = new TimestampDatum(timestamp);
+    testSimpleEval(String.format("select to_timestamp(CAST(split_part('%d.999', '.', 1) as INT8));", timestamp),
+        new String[] {expected.asChars()});
+  }
+
+  @Test
+  public void testCastFromTable() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    testEval(schema, "table1", "123,234", "select cast(col1 as float) as b, cast(col2 as float) as a from table1",
+        new String[]{"123.0", "234.0"});
+    testEval(schema, "table1", "123,234", "select col1::float, col2::float from table1",
+        new String[]{"123.0", "234.0"});
+    testEval(schema, "table1", "1980-04-01 01:50:01,234", "select col1::timestamp as t1, col2::float from table1 " +
+        "where t1 = '1980-04-01 01:50:01'::timestamp",
+        new String[]{"1980-04-01 01:50:01", "234.0"});
+
+    testSimpleEval("select '1980-04-01 01:50:01'::timestamp;", new String [] {"1980-04-01 01:50:01"});
+    testSimpleEval("select '1980-04-01 01:50:01'::timestamp::text", new String [] {"1980-04-01 01:50:01"});
+
+    testSimpleEval("select (cast ('99999'::int8 as text))::int4 + 1", new String [] {"100000"});
+  }
+
+  @Test
+  public void testBooleanLiteral() throws IOException {
+    testSimpleEval("select true", new String[] {"t"});
+    testSimpleEval("select false", new String[]{"f"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    testEval(schema, "table1", "123,234", "select col1, col2 from table1 where true", new String[]{"123", "234"});
+  }
+
+  @Test
+  public void testNullComparisons() throws IOException {
+    testSimpleEval("select null is null", new String[] {"t"});
+    testSimpleEval("select null is not null", new String[] {"f"});
+
+    testSimpleEval("select (1::int2 > null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int2 < null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int2 >= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int2 <= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int2 <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select (1::int4 > null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int4 < null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int4 >= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int4 <= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int4 <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select (1::int8 > null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int8 < null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int8 >= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int8 <= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::int8 <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select (1::float > null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float < null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float >= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float <= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select (1::float8 > null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float8 < null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float8 >= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float8 <= null) is null", new String[] {"t"});
+    testSimpleEval("select (1::float8 <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select ('abc' > null) is null", new String[] {"t"});
+    testSimpleEval("select ('abc' < null) is null", new String[] {"t"});
+    testSimpleEval("select ('abc' >= null) is null", new String[] {"t"});
+    testSimpleEval("select ('abc' <= null) is null", new String[] {"t"});
+    testSimpleEval("select ('abc' <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select ('1980-04-01'::date > null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01'::date < null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01'::date >= null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01'::date <= null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01'::date <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select ('09:08:50'::time > null) is null", new String[] {"t"});
+    testSimpleEval("select ('09:08:50'::time < null) is null", new String[] {"t"});
+    testSimpleEval("select ('09:08:50'::time >= null) is null", new String[] {"t"});
+    testSimpleEval("select ('09:08:50'::time <= null) is null", new String[] {"t"});
+    testSimpleEval("select ('09:08:50'::time <> null) is null", new String[] {"t"});
+
+    testSimpleEval("select ('1980-04-01 01:50:30'::timestamp > null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01 01:50:30'::timestamp < null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01 01:50:30'::timestamp >= null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01 01:50:30'::timestamp <= null) is null", new String[] {"t"});
+    testSimpleEval("select ('1980-04-01 01:50:30'::timestamp <> null) is null", new String[] {"t"});
+
+
+    // Three Valued Logic - AND
+    testSimpleEval("select (true AND true)", new String[] {"t"}); // true - true -> true
+    testSimpleEval("select (true AND 1 > null) is null", new String[] {"t"}); // true - unknown -> unknown
+    testSimpleEval("select (true AND false)", new String[] {"f"}); // true - false -> true
+
+    testSimpleEval("select (1 > null AND true) is null", new String[] {"t"}); // unknown - true -> true
+    testSimpleEval("select (1 > null AND 1 > null) is null", new String[] {"t"}); // unknown - unknown -> unknown
+    testSimpleEval("select (1 > null AND false)", new String[] {"f"}); // unknown - false -> false
+
+    testSimpleEval("select (false AND true)", new String[] {"f"}); // false - true -> true
+    testSimpleEval("select (false AND 1 > null) is null", new String[] {"f"}); // false - unknown -> unknown
+    testSimpleEval("select (false AND false)", new String[] {"f"}); // false - false -> false
+
+    // Three Valued Logic - OR
+    testSimpleEval("select (true OR true)", new String[] {"t"}); // true - true -> true
+    testSimpleEval("select (true OR 1 > null)", new String[] {"t"}); // true - unknown -> true
+    testSimpleEval("select (true OR false)", new String[] {"t"}); // true - false -> true
+
+    testSimpleEval("select (1 > null OR true)", new String[] {"t"}); // unknown - true -> true
+    testSimpleEval("select (1 > null OR 1 > null) is null", new String[] {"t"}); // unknown - unknown -> unknown
+    testSimpleEval("select (1 > null OR false) is null", new String[] {"t"}); // unknown - false -> false
+
+    testSimpleEval("select (false OR true)", new String[] {"t"}); // false - true -> true
+    testSimpleEval("select (false OR 1 > null) is null", new String[] {"t"}); // false - unknown -> unknown
+    testSimpleEval("select (false OR false)", new String[] {"f"}); // false - false -> false
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestAggFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestAggFunction.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestAggFunction.java
new file mode 100644
index 0000000..f635468
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestAggFunction.java
@@ -0,0 +1,64 @@
+/**
+ * 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.function;
+
+import org.junit.Test;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.builtin.AvgLong;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestAggFunction {
+
+  @Test
+  public void testAvgInt() {
+    Tuple [] tuples = new Tuple[5];
+
+    for (int i = 1; i <= 5; i++) {
+      tuples[i-1] = new VTuple(1);
+      tuples[i-1].put(0, DatumFactory.createInt4(i));
+    }
+
+    AvgLong avg = new AvgLong();
+    FunctionContext ctx = avg.newContext();
+    for (int i = 1; i <= 5; i++) {
+      avg.eval(ctx, tuples[i-1]);
+    }
+
+    assertTrue(15 / 5 == avg.terminate(ctx).asFloat8());
+
+
+    Tuple [] tuples2 = new Tuple[10];
+
+    FunctionContext ctx2 = avg.newContext();
+    for (int i = 1; i <= 10; i++) {
+      tuples2[i-1] = new VTuple(1);
+      tuples2[i-1].put(0, DatumFactory.createInt4(i));
+      avg.eval(ctx2, tuples2[i-1]);
+    }
+    assertTrue((double)55 / 10 == avg.terminate(ctx2).asFloat8());
+
+
+    avg.merge(ctx, new VTuple(new Datum[] {avg.getPartialResult(ctx2)}));
+    assertTrue((double)(15 + 55) / (5 + 10) == avg.terminate(ctx).asFloat8());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
new file mode 100644
index 0000000..65612f7
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
@@ -0,0 +1,123 @@
+/**
+ * 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.function;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+
+import static org.junit.Assert.assertTrue;
+
+@Category(IntegrationTest.class)
+public class TestBuiltinFunctions extends QueryTestCaseBase {
+
+  public TestBuiltinFunctions() {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test
+  public void testMaxLong() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testMinLong() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testMaxString() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testMinString() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCount() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testAvgDouble() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testAvgLong() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testAvgInt() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testRandom() throws Exception {
+    ResultSet res = executeQuery();
+    while(res.next()) {
+      assertTrue(res.getInt(2) >= 0 && res.getInt(2) < 3);
+    }
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testSplitPart() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testSplitPartByString() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testSplitPartNested() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
new file mode 100644
index 0000000..af86387
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
@@ -0,0 +1,82 @@
+/**
+ * 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.function;
+
+import org.apache.tajo.catalog.exception.NoSuchFunctionException;
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+public class TestConditionalExpressions extends ExprTestBase {
+  @Test
+  public void testCoalesceText() throws Exception {
+    testSimpleEval("select coalesce(null, 'value2');", new String[]{"value2"});
+    testSimpleEval("select coalesce(null, null, 'value3');", new String[]{"value3"});
+    testSimpleEval("select coalesce('value1', null, 'value3');", new String[]{"value1"});
+    testSimpleEval("select coalesce(null, 'value2', 'value3');", new String[]{"value2"});
+
+    //no matched function
+    try {
+      testSimpleEval("select coalesce(null, 2, 'value3');", new String[]{"2"});
+      fail("coalesce(NULL, INT, TEXT) not defined. So should throw exception.");
+    } catch (NoSuchFunctionException e) {
+      //success
+    }
+  }
+
+  @Test
+  public void testCoalesceLong() throws Exception {
+    testSimpleEval("select coalesce(null, 2);", new String[]{"2"});
+    testSimpleEval("select coalesce(null, null, 3);", new String[]{"3"});
+    testSimpleEval("select coalesce(1, null, 3);", new String[]{"1"});
+    testSimpleEval("select coalesce(null, 2, 3);", new String[]{"2"});
+
+    //no matched function
+    try {
+      testSimpleEval("select coalesce(null, 'value2', 3);", new String[]{"2"});
+      fail("coalesce(NULL, TEXT, INT) not defined. So should throw exception.");
+    } catch (NoSuchFunctionException e) {
+      //success
+    }
+  }
+
+  @Test
+  public void testCoalesceDouble() throws Exception {
+    testSimpleEval("select coalesce(null, 2.0);", new String[]{"2.0"});
+    testSimpleEval("select coalesce(null, null, 3.0);", new String[]{"3.0"});
+    testSimpleEval("select coalesce(1.0, null, 3.0);", new String[]{"1.0"});
+    testSimpleEval("select coalesce(null, 2.0, 3.0);", new String[]{"2.0"});
+
+    //no matched function
+    try {
+      testSimpleEval("select coalesce('value1', null, 3.0);", new String[]{"1.0"});
+      fail("coalesce(TEXT, NULL, FLOAT8) not defined. So should throw exception.");
+    } catch (NoSuchFunctionException e) {
+      //success
+    }
+
+    try {
+      testSimpleEval("select coalesce(null, 'value2', 3.0);", new String[]{"2.0"});
+      fail("coalesce(NULL, TEXT, FLOAT8) not defined. So should throw exception.");
+    } catch (NoSuchFunctionException e) {
+      //success
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
new file mode 100644
index 0000000..ac7a2b8
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
@@ -0,0 +1,253 @@
+/**
+ * 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.function;
+
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.datum.TimestampDatum;
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+
+public class TestDateTimeFunctions extends ExprTestBase {
+
+  @Test
+  public void testToTimestamp() throws IOException {
+    long expectedTimestamp = System.currentTimeMillis();
+    DateTime expectedDateTime = new DateTime(expectedTimestamp);
+
+    // (expectedTimestamp / 1000) means the translation from millis seconds to unix timestamp
+    String q1 = String.format("select to_timestamp(%d);", (expectedTimestamp / 1000));
+    testSimpleEval(q1, new String[]{expectedDateTime.toString(TimestampDatum.DEFAULT_FORMAT_STRING)});
+  }
+
+  @Test
+  public void testToChar() throws IOException {
+    long expectedTimestamp = System.currentTimeMillis();
+    DateTime expectedDateTime = new DateTime(expectedTimestamp);
+    String dateFormatStr = "yyyy-MM";
+    // (expectedTimestamp / 1000) means the translation from millis seconds to unix timestamp
+    String q = String.format("select to_char(to_timestamp(%d), 'yyyy-MM');", (expectedTimestamp / 1000));
+    testSimpleEval(q, new String[]{expectedDateTime.toString(dateFormatStr)});
+  }
+
+  @Test
+  public void testExtract() throws IOException {
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", TIMESTAMP);
+    testEval(schema2, "table1",
+        "1970-01-17 10:09:37",
+        "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
+
+    Schema schema3 = new Schema();
+    schema3.addColumn("col1", TIME);
+    testEval(schema3, "table1",
+        "10:09:37.5",
+        "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;",
+        new String[]{"10.0", "9.0", "37.5"});
+
+    Schema schema4 = new Schema();
+    schema4.addColumn("col1", DATE);
+    testEval(schema4, "table1",
+        "1970-01-17",
+        "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
+
+    testSimpleEval("select extract(century from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"19.0"});
+
+    testSimpleEval("select extract(century from DATE '1970-01-17');", new String[]{"19.0"});
+
+    testSimpleEval("select extract(decade from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"197.0"});
+
+    testSimpleEval("select extract(decade from DATE '1970-01-17');", new String[]{"197.0"});
+
+    testSimpleEval("select extract(millennium from TIMESTAMP '2001-02-16 10:09:37');", new String[]{"3.0"});
+    testSimpleEval("select extract(millennium from TIMESTAMP '2000-02-16 10:09:37');", new String[]{"2.0"});
+
+    testSimpleEval("select extract(millennium from DATE '2001-02-16');", new String[]{"3.0"});
+    testSimpleEval("select extract(millennium from DATE '2000-02-16');", new String[]{"2.0"});
+
+    testSimpleEval("select extract(year from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"1970.0"});
+    testSimpleEval("select extract(month from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"1.0"});
+    testSimpleEval("select extract(day from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"17.0"});
+
+    testSimpleEval("select extract(hour from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"10.0"});
+    testSimpleEval("select extract(minute from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"9.0"});
+    testSimpleEval("select extract(second from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"37.0"});
+    testSimpleEval("select extract(second from TIMESTAMP '1970-01-17 10:09:37.5');", new String[]{"37.5"});
+
+    testSimpleEval("select extract(hour from TIME '10:09:37');", new String[]{"10.0"});
+    testSimpleEval("select extract(minute from TIME '10:09:37');", new String[]{"9.0"});
+    testSimpleEval("select extract(second from TIME '10:09:37');", new String[]{"37.0"});
+    testSimpleEval("select extract(second from TIME '10:09:37.5');", new String[]{"37.5"});
+
+    testSimpleEval("select extract(year from DATE '1970-01-17');", new String[]{"1970.0"});
+    testSimpleEval("select extract(month from DATE '1970-01-17');", new String[]{"1.0"});
+    testSimpleEval("select extract(day from DATE '1970-01-17');", new String[]{"17.0"});
+
+    testSimpleEval("select extract(milliseconds from TIMESTAMP '1970-01-17 10:09:37.5');", new String[]{"37500.0"});
+    testSimpleEval("select extract(milliseconds from TIME '10:09:37.123');", new String[]{"37123.0"});
+
+    testSimpleEval("select extract(microseconds from TIMESTAMP '1970-01-17 10:09:37.5');", new String[]{"3.75E7"});
+    testSimpleEval("select extract(microseconds from TIME '10:09:37.123');", new String[]{"3.7123E7"});
+
+    testSimpleEval("select extract(dow from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"6.0"});
+    testSimpleEval("select extract(dow from TIMESTAMP '1970-01-18 10:09:37');", new String[]{"0.0"});
+    testSimpleEval("select extract(isodow from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"6.0"});
+    testSimpleEval("select extract(isodow from TIMESTAMP '1970-01-18 10:09:37');", new String[]{"7.0"});
+
+    testSimpleEval("select extract(year from TIMESTAMP '2006-01-02 10:09:37');", new String[]{"2006.0"});
+    testSimpleEval("select extract(year from TIMESTAMP '2006-01-01 10:09:37');", new String[]{"2006.0"});
+    testSimpleEval("select extract(isoyear from TIMESTAMP '2006-01-02 10:09:37');", new String[]{"2006.0"});
+    testSimpleEval("select extract(isoyear from TIMESTAMP '2006-01-01 10:09:37');", new String[]{"2005.0"});
+
+    testSimpleEval("select extract(quarter from TIMESTAMP '2006-02-01 10:09:37');", new String[]{"1.0"});
+    testSimpleEval("select extract(quarter from TIMESTAMP '2006-04-01 10:09:37');", new String[]{"2.0"});
+    testSimpleEval("select extract(quarter from TIMESTAMP '2006-07-01 10:09:37');", new String[]{"3.0"});
+    testSimpleEval("select extract(quarter from TIMESTAMP '2006-12-01 10:09:37');", new String[]{"4.0"});
+
+    testSimpleEval("select extract(week from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"3.0"});
+
+    testSimpleEval("select extract(dow from DATE '1970-01-17');", new String[]{"6.0"});
+    testSimpleEval("select extract(dow from DATE '1970-01-18');", new String[]{"0.0"});
+    testSimpleEval("select extract(isodow from DATE '1970-01-17');", new String[]{"6.0"});
+    testSimpleEval("select extract(isodow from DATE '1970-01-18');", new String[]{"7.0"});
+
+    testSimpleEval("select extract(year from DATE '2006-01-02');", new String[]{"2006.0"});
+    testSimpleEval("select extract(year from DATE '2006-01-01');", new String[]{"2006.0"});
+    testSimpleEval("select extract(isoyear from DATE '2006-01-02');", new String[]{"2006.0"});
+    testSimpleEval("select extract(isoyear from DATE '2006-01-01');", new String[]{"2005.0"});
+
+    testSimpleEval("select extract(quarter from DATE '2006-02-01');", new String[]{"1.0"});
+    testSimpleEval("select extract(quarter from DATE '2006-04-01');", new String[]{"2.0"});
+    testSimpleEval("select extract(quarter from DATE '2006-07-01');", new String[]{"3.0"});
+    testSimpleEval("select extract(quarter from DATE '2006-12-01');", new String[]{"4.0"});
+
+    testSimpleEval("select extract(week from DATE '1970-01-17');", new String[]{"3.0"});
+  }
+
+  @Test
+  public void testDatePart() throws IOException {
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", TIMESTAMP);
+    testEval(schema2, "table1",
+        "1970-01-17 10:09:37",
+        "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
+
+    Schema schema3 = new Schema();
+    schema3.addColumn("col1", TIME);
+    testEval(schema3, "table1", "10:09:37.5",
+        "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;",
+        new String[]{"10.0", "9.0", "37.5"});
+
+    Schema schema4 = new Schema();
+    schema4.addColumn("col1", DATE);
+    testEval(schema4, "table1",
+        "1970-01-17",
+        "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
+
+    testSimpleEval("select date_part('century', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"19.0"});
+
+    testSimpleEval("select date_part('century', DATE '1970-01-17');", new String[]{"19.0"});
+
+    testSimpleEval("select date_part('decade', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"197.0"});
+
+    testSimpleEval("select date_part('decade', DATE '1970-01-17');", new String[]{"197.0"});
+
+    testSimpleEval("select date_part('millennium', TIMESTAMP '2001-02-16 10:09:37');", new String[]{"3.0"});
+    testSimpleEval("select date_part('millennium', TIMESTAMP '2000-02-16 10:09:37');", new String[]{"2.0"});
+
+    testSimpleEval("select date_part('millennium', DATE '2001-02-16');", new String[]{"3.0"});
+    testSimpleEval("select date_part('millennium', DATE '2000-02-16');", new String[]{"2.0"});
+
+    testSimpleEval("select date_part('year', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"1970.0"});
+    testSimpleEval("select date_part('month', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"1.0"});
+    testSimpleEval("select date_part('day', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"17.0"});
+
+    testSimpleEval("select date_part('hour', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"10.0"});
+    testSimpleEval("select date_part('minute', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"9.0"});
+    testSimpleEval("select date_part('second', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"37.0"});
+    testSimpleEval("select date_part('second', TIMESTAMP '1970-01-17 10:09:37.5');", new String[]{"37.5"});
+
+    testSimpleEval("select date_part('hour', TIME '10:09:37');", new String[]{"10.0"});
+    testSimpleEval("select date_part('minute', TIME '10:09:37');", new String[]{"9.0"});
+    testSimpleEval("select date_part('second', TIME '10:09:37');", new String[]{"37.0"});
+    testSimpleEval("select date_part('second', TIME '10:09:37.5');", new String[]{"37.5"});
+
+    testSimpleEval("select date_part('year', DATE '1970-01-17');", new String[]{"1970.0"});
+    testSimpleEval("select date_part('month', DATE '1970-01-17');", new String[]{"1.0"});
+    testSimpleEval("select date_part('day', DATE '1970-01-17');", new String[]{"17.0"});
+
+    testSimpleEval("select date_part('milliseconds', TIMESTAMP '1970-01-17 10:09:37.5');", new String[]{"37500.0"});
+    testSimpleEval("select date_part('milliseconds', TIME '10:09:37.123');", new String[]{"37123.0"});
+
+    testSimpleEval("select date_part('microseconds', TIMESTAMP '1970-01-17 10:09:37.5');", new String[]{"3.75E7"});
+    testSimpleEval("select date_part('microseconds', TIME '10:09:37.123');", new String[]{"3.7123E7"});
+
+    testSimpleEval("select date_part('dow', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"6.0"});
+    testSimpleEval("select date_part('dow', TIMESTAMP '1970-01-18 10:09:37');", new String[]{"0.0"});
+    testSimpleEval("select date_part('isodow', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"6.0"});
+    testSimpleEval("select date_part('isodow', TIMESTAMP '1970-01-18 10:09:37');", new String[]{"7.0"});
+
+    testSimpleEval("select date_part('year', TIMESTAMP '2006-01-02 10:09:37');", new String[]{"2006.0"});
+    testSimpleEval("select date_part('year', TIMESTAMP '2006-01-01 10:09:37');", new String[]{"2006.0"});
+    testSimpleEval("select date_part('isoyear', TIMESTAMP '2006-01-02 10:09:37');", new String[]{"2006.0"});
+    testSimpleEval("select date_part('isoyear', TIMESTAMP '2006-01-01 10:09:37');", new String[]{"2005.0"});
+
+    testSimpleEval("select date_part('quarter', TIMESTAMP '2006-02-01 10:09:37');", new String[]{"1.0"});
+    testSimpleEval("select date_part('quarter', TIMESTAMP '2006-04-01 10:09:37');", new String[]{"2.0"});
+    testSimpleEval("select date_part('quarter', TIMESTAMP '2006-07-01 10:09:37');", new String[]{"3.0"});
+    testSimpleEval("select date_part('quarter', TIMESTAMP '2006-12-01 10:09:37');", new String[]{"4.0"});
+
+    testSimpleEval("select date_part('week', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"3.0"});
+
+    testSimpleEval("select date_part('dow', DATE '1970-01-17');", new String[]{"6.0"});
+    testSimpleEval("select date_part('dow', DATE '1970-01-18');", new String[]{"0.0"});
+    testSimpleEval("select date_part('isodow', DATE '1970-01-17');", new String[]{"6.0"});
+    testSimpleEval("select date_part('isodow', DATE '1970-01-18');", new String[]{"7.0"});
+
+    testSimpleEval("select date_part('year', DATE '2006-01-02');", new String[]{"2006.0"});
+    testSimpleEval("select date_part('year', DATE '2006-01-01');", new String[]{"2006.0"});
+    testSimpleEval("select date_part('isoyear', DATE '2006-01-02');", new String[]{"2006.0"});
+    testSimpleEval("select date_part('isoyear', DATE '2006-01-01');", new String[]{"2005.0"});
+
+    testSimpleEval("select date_part('quarter', DATE '2006-02-01');", new String[]{"1.0"});
+    testSimpleEval("select date_part('quarter', DATE '2006-04-01');", new String[]{"2.0"});
+    testSimpleEval("select date_part('quarter', DATE '2006-07-01');", new String[]{"3.0"});
+    testSimpleEval("select date_part('quarter', DATE '2006-12-01');", new String[]{"4.0"});
+
+    testSimpleEval("select date_part('week', DATE '1970-01-17');", new String[]{"3.0"});
+  }
+
+  @Test
+  public void testUtcUsecTo() throws IOException {
+     testSimpleEval("select utc_usec_to('day' ,1274259481071200);", new String[]{1274227200000000L+""});
+     testSimpleEval("select utc_usec_to('hour' ,1274259481071200);", new String[]{1274256000000000L+""});
+     testSimpleEval("select utc_usec_to('month' ,1274259481071200);", new String[]{1272672000000000L+""});
+     testSimpleEval("select utc_usec_to('year' ,1274259481071200);", new String[]{1262304000000000L+""});
+     testSimpleEval("select utc_usec_to('week' ,1207929480000000, 2);", new String[]{1207612800000000L+""});
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestGeneralFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestGeneralFunction.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestGeneralFunction.java
new file mode 100644
index 0000000..81ddba5
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestGeneralFunction.java
@@ -0,0 +1,49 @@
+/**
+ * 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.function;
+
+import org.junit.Test;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.function.builtin.Date;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+
+import java.util.Calendar;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestGeneralFunction {
+
+  @Test
+  public void testDate() {
+    Date date = new Date();
+    Tuple tuple = new VTuple(new Datum[] {new TextDatum("25/12/2012 00:00:00")});
+    Int8Datum unixtime = (Int8Datum) date.eval(tuple);
+    Calendar c = Calendar.getInstance();
+    c.setTimeInMillis(unixtime.asInt8());
+    assertEquals(2012, c.get(Calendar.YEAR));
+    assertEquals(11, c.get(Calendar.MONTH));
+    assertEquals(25, c.get(Calendar.DAY_OF_MONTH));
+    assertEquals(0, c.get(Calendar.HOUR_OF_DAY));
+    assertEquals(0, c.get(Calendar.MINUTE));
+    assertEquals(0, c.get(Calendar.SECOND));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
new file mode 100644
index 0000000..09f7f68
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
@@ -0,0 +1,428 @@
+/**
+ * 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.function;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+
+public class TestMathFunctions extends ExprTestBase {
+  @Test
+  public void testRound() throws IOException {
+    testSimpleEval("select round(5.1) as col1 ", new String[]{"5"});
+    testSimpleEval("select round(5.5) as col1 ", new String[]{"6"});
+    testSimpleEval("select round(5.6) as col1 ", new String[]{"6"});
+
+    testSimpleEval("select round(-5.1) as col1 ", new String[]{"-5"});
+    testSimpleEval("select round(-5.5) as col1 ", new String[]{"-6"});
+    testSimpleEval("select round(-5.6) as col1 ", new String[]{"-6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from table1",
+        new String[]{"2"});
+  }
+
+  @Test
+  public void testFloor() throws IOException {
+    testSimpleEval("select floor(5.1) as col1 ", new String[]{"5"});
+    testSimpleEval("select floor(5.5) as col1 ", new String[]{"5"});
+    testSimpleEval("select floor(5.6) as col1 ", new String[]{"5"});
+
+    testSimpleEval("select floor(-5.1) as col1 ", new String[]{"-6"});
+    testSimpleEval("select floor(-5.6) as col1 ", new String[]{"-6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from table1",
+        new String[]{"1"});
+  }
+
+  @Test
+  public void testCeil() throws IOException {
+    testSimpleEval("select ceil(5.0) as col1 ", new String[]{"5"});
+    testSimpleEval("select ceil(5.1) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceil(5.5) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceil(5.6) as col1 ", new String[]{"6"});
+
+    testSimpleEval("select ceil(-5.1) as col1 ", new String[]{"-5"});
+    testSimpleEval("select ceil(-5.6) as col1 ", new String[]{"-5"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from table1",
+        new String[]{"2"});
+  }
+
+  @Test
+  public void testCeiling() throws IOException {
+    testSimpleEval("select ceiling(5.0) as col1 ", new String[]{"5"});
+    testSimpleEval("select ceiling(5.1) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceiling(5.5) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceiling(5.6) as col1 ", new String[]{"6"});
+
+    testSimpleEval("select ceiling(-5.1) as col1 ", new String[]{"-5"});
+    testSimpleEval("select ceiling(-5.6) as col1 ", new String[]{"-5"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceiling(col1 + col2 + col3) from table1",
+        new String[]{"2"});
+  }
+
+  @Test
+  public void testSin() throws IOException {
+    testSimpleEval("select sin(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select sin(0.7) as col1 ", new String[]{"0.644217687237691"});
+    testSimpleEval("select sin(1.2) as col1 ", new String[]{"0.9320390859672263"});
+    testSimpleEval("select sin(-0.5) as col1 ", new String[]{"-0.479425538604203"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select sin(col1 + col2 + col3) from table1",
+        new String[]{"0.963558185417193"});
+  }
+
+
+  @Test
+  public void testCos() throws IOException {
+    testSimpleEval("select cos(0.0) as col1 ", new String[]{"1.0"});
+    testSimpleEval("select cos(0.7) as col1 ", new String[]{"0.7648421872844885"});
+    testSimpleEval("select cos(1.2) as col1 ", new String[]{"0.3623577544766736"});
+    testSimpleEval("select cos(-0.5) as col1 ", new String[]{"0.8775825618903728"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select cos(col1 + col2 + col3) from table1",
+        new String[]{"0.26749882862458735"});
+  }
+
+  @Test
+  public void testTan() throws IOException {
+    testSimpleEval("select tan(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select tan(0.3) as col1 ", new String[]{"0.30933624960962325"});
+    testSimpleEval("select tan(0.8) as col1 ", new String[]{"1.0296385570503641"});
+    testSimpleEval("select tan(-0.5) as col1 ", new String[]{"-0.5463024898437905"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select tan(col1 - col2 - col3) from table1",
+        new String[]{"0.8422883804630795"});
+  }
+
+  @Test
+  public void testAsin() throws IOException {
+    testSimpleEval("select asin(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select asin(0.3) as col1 ", new String[]{"0.3046926540153975"});
+    testSimpleEval("select asin(0.8) as col1 ", new String[]{"0.9272952180016123"});
+    testSimpleEval("select asin(-0.5) as col1 ", new String[]{"-0.5235987755982989"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select asin(col1 - col2 - col3) from table1",
+        new String[]{"0.7753974966107532"});
+  }
+
+  @Test
+  public void testAcos() throws IOException {
+    testSimpleEval("select acos(0.0) as col1 ", new String[]{"1.5707963267948966"});
+    testSimpleEval("select acos(0.3) as col1 ", new String[]{"1.2661036727794992"});
+    testSimpleEval("select acos(0.8) as col1 ", new String[]{"0.6435011087932843"});
+    testSimpleEval("select acos(-0.5) as col1 ", new String[]{"2.0943951023931957"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select acos(col1 - col2 - col3) from table1",
+        new String[]{"0.7953988301841435"});
+  }
+
+  @Test
+  public void testAtan() throws IOException {
+    testSimpleEval("select atan(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select atan(0.8) as col1 ", new String[]{"0.6747409422235527"});
+    testSimpleEval("select atan(1.2) as col1 ", new String[]{"0.8760580505981934"});
+    testSimpleEval("select atan(-0.5) as col1 ", new String[]{"-0.4636476090008061"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan(col1 + col2 + col3) from table1",
+        new String[]{"0.9151007005533605"});
+  }
+
+  @Test
+  public void testAtan2() throws IOException {
+    testSimpleEval("select atan2(0.8, 0.0) as col1 ", new String[]{"1.5707963267948966"});
+    testSimpleEval("select atan2(0.8, 1.1) as col1 ", new String[]{"0.628796286415433"});
+    testSimpleEval("select atan2(2.7, 0.3) as col1 ", new String[]{"1.460139105621001"});
+    testSimpleEval("select atan2(-0.5, 0.3) as col1 ", new String[]{"-1.0303768265243125"});
+    testSimpleEval("select atan2(-0.2, -1.3) as col1 ", new String[]{"-2.988943325194528"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan2(col1 + col2, col3) from table1",
+        new String[]{"1.4876550949064553"});
+  }
+
+  @Test
+  public void testMod() throws IOException {
+    testSimpleEval("select mod(9,4) as col1 ", new String[]{"1"});
+    testSimpleEval("select mod(200000000001,200000000000) as col1 ", new String[]{"1"});
+    testSimpleEval("select mod(200000000000,2) as col1 ", new String[]{"0"});
+    testSimpleEval("select mod(2,200000000000) as col1 ", new String[]{"2"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", INT8);
+    schema.addColumn("col2", INT8);
+    schema.addColumn("col3", INT8);
+
+    testEval(schema, "table1", "9,2,3", "select mod(col1 + col2, col3) from table1", 
+        new String[]{"2"});
+  }
+
+  @Test
+  public void testDiv() throws IOException {
+    testSimpleEval("select div(9,4) as col1 ", new String[]{"2"});
+    testSimpleEval("select div(200000000001,200000000000) as col1 ", new String[]{"1"});
+    testSimpleEval("select div(200000000000,2) as col1 ", new String[]{"100000000000"});
+    testSimpleEval("select div(2,200000000000) as col1 ", new String[]{"0"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", INT8);
+    schema.addColumn("col2", INT8);
+    schema.addColumn("col3", INT8);
+
+    testEval(schema, "table1", "9,2,3", "select div(col1 + col2, col3) from table1", 
+        new String[]{"3"});
+  }
+
+  @Test
+  public void testSign() throws IOException {
+    testSimpleEval("select sign(2) as col1 ", new String[]{"1.0"});
+    testSimpleEval("select sign(2.345) as col1 ", new String[]{"1.0"});
+    testSimpleEval("select sign(0.3) as col1 ", new String[]{"1.0"});
+
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+    schema.addColumn("col2", FLOAT4);
+    schema.addColumn("col3", FLOAT4);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select sign(col1 + col2 + col3) from table1",
+        new String[]{"1.0"});
+
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", FLOAT8);
+    schema2.addColumn("col2", FLOAT8);
+    schema2.addColumn("col3", FLOAT8);
+
+    testEval(schema2, "table1", "1.0, 0.2, 0.1", "select sign(col1 + col2 + col3) from table1",
+        new String[]{"1.0"});
+  }
+
+  @Test
+  public void testSqrt() throws IOException {
+    testSimpleEval("select sqrt(27.0) as col1 ", new String[]{"5.196152422706632"});
+    testSimpleEval("select sqrt(64.0) as col1 ", new String[]{"8.0"});
+    testSimpleEval("select sqrt(8.0) as col1 ", new String[]{"2.8284271247461903"});
+
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+    schema.addColumn("col2", FLOAT4);
+    schema.addColumn("col3", FLOAT4);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from table1",
+        new String[]{"1.1401754466632896"});
+
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", FLOAT8);
+    schema2.addColumn("col2", FLOAT8);
+    schema2.addColumn("col3", FLOAT8);
+
+    testEval(schema2, "table1", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from table1",
+        new String[]{"1.140175425099138"});
+  }
+
+  @Test
+  public void testExp() throws IOException {
+    testSimpleEval("select exp(1.0) as col1 ", new String[]{String.valueOf(Math.exp(1.0d))});
+    testSimpleEval("select exp(1.1) as col1 ", new String[]{String.valueOf(Math.exp(1.1d))});
+    testSimpleEval("select exp(1.2) as col1 ", new String[]{String.valueOf(Math.exp(1.2d))});
+
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+
+    testEval(schema, "table1", "1.123", "select exp(col1) from table1",
+        new String[]{String.valueOf(Math.exp(1.123f))});
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", FLOAT8);
+
+    testEval(schema2, "table1", "1.123", "select exp(col1) from table1",
+        new String[]{String.valueOf(Math.exp(1.123d))});
+  }
+
+
+  @Test
+  public void testAbs() throws IOException {
+    testSimpleEval("select abs(9) as col1 ", new String[]{"9"});
+    testSimpleEval("select abs(-9) as col1 ", new String[]{"9"});
+    testSimpleEval("select abs(200000000000) as col1 ", new String[]{"200000000000"});
+    testSimpleEval("select abs(-200000000000) as col1 ", new String[]{"200000000000"});
+    testSimpleEval("select abs(2.0) as col1 ", new String[]{"2.0"});
+    testSimpleEval("select abs(-2.0) as col1 ", new String[]{"2.0"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+    schema.addColumn("col2", FLOAT4);
+    testEval(schema, "table1", "0.39,-0.39", "select abs(col1), abs(col2) from table1", new String[]{"0.39", "0.39"});
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", FLOAT8);
+    schema2.addColumn("col2", FLOAT8);
+    testEval(schema2, "table1", "0.033312347,-0.033312347", "select abs(col1), abs(col2) from table1",
+        new String[]{"0.033312347", "0.033312347"});
+  }
+
+  @Test
+  public void testCbrt() throws IOException {
+    testSimpleEval("select cbrt(27.0) as col1 ", new String[]{"3.0"});
+    testSimpleEval("select cbrt(64.0) as col1 ", new String[]{"4.0"});
+    testSimpleEval("select cbrt(8.0) as col1 ", new String[]{"2.0"});
+
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+    schema.addColumn("col2", FLOAT4);
+    schema.addColumn("col3", FLOAT4);
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from table1",
+        new String[]{"1.0913928968221236"});
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", FLOAT8);
+    schema2.addColumn("col2", FLOAT8);
+    schema2.addColumn("col3", FLOAT8);
+    testEval(schema2, "table1", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from table1",
+        new String[]{"1.091392883061106"});
+  }
+
+  @Test
+  public void testDegrees() throws IOException {
+    testSimpleEval("select degrees(0.0) as col1 ", new String[]{String.valueOf(Math.toDegrees(0.0))});
+    testSimpleEval("select degrees(0.8) as col1 ", new String[]{String.valueOf(Math.toDegrees(0.8))});
+    testSimpleEval("select degrees(2.7) as col1 ", new String[]{String.valueOf(Math.toDegrees(2.7))});
+    testSimpleEval("select degrees(-0.8) as col1 ", new String[]{String.valueOf(Math.toDegrees(-0.8))});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "0.8,2.7,-0.8", "select degrees(col1), degrees(col2), degrees(col3) from table1",
+        new String[]{
+            String.valueOf(Math.toDegrees((float)0.8)),
+            String.valueOf(Math.toDegrees(2.7)),
+            String.valueOf(Math.toDegrees(-0.8))
+        });
+  }
+
+  @Test
+  public void testPow() throws IOException {
+    testSimpleEval("select pow(9,3) as col1 ", new String[]{String.valueOf(Math.pow(9,3))});
+    testSimpleEval("select pow(1.0,3) as col1 ", new String[]{String.valueOf(Math.pow(1.0,3))});
+    testSimpleEval("select pow(20.1,3.1) as col1 ", new String[]{String.valueOf(Math.pow(20.1,3.1))});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", INT4);
+    schema.addColumn("col4", INT8);
+
+    testEval(schema, "table1", "0.4,2.7,3,2", "select pow(col1, col2), pow(col3, col4) from table1",
+        new String[]{
+            String.valueOf(Math.pow((float)0.4, 2.7)),
+            String.valueOf(Math.pow(3,2))
+        });
+  }
+
+  @Test
+  public void testRadians() throws IOException {
+    testSimpleEval("select radians(0.0) as col1 ", new String[]{String.valueOf(Math.toRadians(0.0))});
+    testSimpleEval("select radians(0.8) as col1 ", new String[]{String.valueOf(Math.toRadians(0.8))});
+    testSimpleEval("select radians(2.7) as col1 ", new String[]{String.valueOf(Math.toRadians(2.7))});
+    testSimpleEval("select radians(-0.8) as col1 ", new String[]{String.valueOf(Math.toRadians(-0.8))});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT4);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "0.8,2.7,-0.8", "select radians(col1), radians(col2), radians(col3) from table1",
+        new String[]{
+            String.valueOf(Math.toRadians((float)0.8)),
+            String.valueOf(Math.toRadians(2.7)),
+            String.valueOf(Math.toRadians(-0.8))
+        });
+  }
+
+  @Test
+  public void testPi() throws IOException {
+    testSimpleEval("select pi() as col1 ", new String[]{String.valueOf(Math.PI)});
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java
new file mode 100644
index 0000000..7d8eba4
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java
@@ -0,0 +1,140 @@
+/**
+ * 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.function;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+public class TestPatternMatchingPredicates extends ExprTestBase {
+
+  @Test
+  public void testLike() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+
+    // test for null values
+    testEval(schema, "table1", ",", "select col1 like 'a%' from table1", new String[]{""});
+    //testSimpleEval("select null like 'a%'", new String[]{""});
+
+    testEval(schema, "table1", "abc", "select col1 like '%c' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 like 'a%' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 like '_bc' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 like 'ab_' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 like '_b_' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 like '%b%' from table1", new String[]{"t"});
+
+    // test for escaping regular expressions
+    testEval(schema, "table1", "abc", "select col1 not like '.bc' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 not like '.*bc' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 not like '.bc' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 not like '*bc' from table1", new String[]{"t"});
+
+    // test for case sensitive
+    testEval(schema, "table1", "abc", "select col1 not like '%C' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 not like 'A%' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 not like '_BC' from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc", "select col1 not like '_C_' from table1", new String[]{"t"});
+  }
+
+  @Test
+  public void testILike() throws IOException {
+    testSimpleEval("select 'abc' ilike '%c'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike 'a%'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike '_bc'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike 'ab_'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike '_b_'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike '%b%'", new String[]{"t"});
+
+    // test for escaping regular expressions
+    testSimpleEval("select 'abc' not like '.bc'", new String[]{"t"});
+    testSimpleEval("select 'abc' not like '.*bc'", new String[]{"t"});
+    testSimpleEval("select 'abc' not like '.bc'", new String[]{"t"});
+    testSimpleEval("select 'abc' not like '*bc'", new String[]{"t"});
+
+    // test for case insensitive
+    testSimpleEval("select 'abc' ilike '%C'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike 'A%'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike '_BC'", new String[]{"t"});
+    testSimpleEval("select 'abc' ilike '_B_'", new String[]{"t"});
+  }
+
+  @Test
+  public void testSimilarToLike() throws IOException {
+    testSimpleEval("select 'abc' similar to '%c'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to 'a%'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to '_bc'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to 'ab_'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to '_b_'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to '%b%'", new String[]{"t"});
+
+    // test for now allowed
+    testSimpleEval("select 'abc' not similar to '.bc'", new String[]{"t"});
+    testSimpleEval("select 'abc' not similar to 'ab.'", new String[]{"t"});
+
+    // test for escaping regular expressions
+    testSimpleEval("select 'abc' similar to '(a|f)b%'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to '[a-z]b%'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to '_+bc'", new String[]{"t"});
+    testSimpleEval("select 'abc' similar to 'abc'", new String[]{"t"});
+
+    // test for case sensitive
+    testSimpleEval("select 'abc' not similar to '%C'", new String[]{"t"});
+    testSimpleEval("select 'abc' not similar to '_Bc'", new String[]{"t"});
+  }
+
+  @Test
+  public void testRegexWithSimilarOperator() throws IOException {
+    testSimpleEval("select 'abc' ~ '.*c'", new String[]{"t"});
+    testSimpleEval("select 'abc' ~ '.*c$'", new String[]{"t"});
+    testSimpleEval("select 'aaabc' ~ '([a-z]){3}bc'", new String[]{"t"});
+
+    // for negative condition
+    testSimpleEval("select 'abc' !~ '.*c$'", new String[]{"f"});
+
+    // for case sensitivity
+    testSimpleEval("select 'abc' ~ '.*C'", new String[]{"f"});
+
+    // for case insensitivity
+    testSimpleEval("select 'abc' ~* '.*C'", new String[]{"t"});
+    testSimpleEval("select 'abc' !~* '.*C'", new String[]{"f"});
+  }
+
+  @Test
+  public void testRegexp() throws IOException {
+    testSimpleEval("select 'abc' regexp '.*c'", new String[]{"t"});
+    testSimpleEval("select 'abc' regexp '.*c$'", new String[]{"t"});
+
+    // for negative condition
+    testSimpleEval("select 'abc' not regexp '.*c$'", new String[]{"f"});
+  }
+
+  @Test
+  public void testRLike() throws IOException {
+    testSimpleEval("select 'abc' rlike '.*c'", new String[]{"t"});
+    testSimpleEval("select 'abc' rlike '.*c$'", new String[]{"t"});
+
+    // for negative condition
+    testSimpleEval("select 'abc' not rlike '.*c$'", new String[]{"f"});
+  }
+}


[03/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
new file mode 100644
index 0000000..cee0cb0
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -0,0 +1,1023 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.eval.AggregationFunctionCallEval;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.RangeRetrieverHandler;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.worker.dataserver.retriever.FileChunk;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.SortEnforce.SortAlgorithm;
+import static org.junit.Assert.*;
+
+public class TestPhysicalPlanner {
+  private static TajoTestingCluster util;
+  private static TajoConf conf;
+  private static CatalogService catalog;
+  private static SQLAnalyzer analyzer;
+  private static LogicalPlanner planner;
+  private static LogicalOptimizer optimizer;
+  private static AbstractStorageManager sm;
+  private static Path testDir;
+  private static Session session = LocalTajoTestingUtility.createDummySession();
+
+  private static TableDesc employee = null;
+  private static TableDesc score = null;
+
+  private static MasterPlan masterPlan;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    util = new TajoTestingCluster();
+
+    util.startCatalogCluster();
+    conf = util.getConfiguration();
+    testDir = CommonTestingUtil.getTestDir("target/test-data/TestPhysicalPlanner");
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+    catalog = util.getMiniCatalogCluster().getCatalog();
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+      catalog.createFunction(funcDesc);
+    }
+
+    Schema employeeSchema = new Schema();
+    employeeSchema.addColumn("name", Type.TEXT);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
+
+    Schema scoreSchema = new Schema();
+    scoreSchema.addColumn("deptname", Type.TEXT);
+    scoreSchema.addColumn("class", Type.TEXT);
+    scoreSchema.addColumn("score", Type.INT4);
+    scoreSchema.addColumn("nullable", Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+
+
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
+        employeePath);
+    appender.init();
+    Tuple tuple = new VTuple(employeeSchema.size());
+    for (int i = 0; i < 100; i++) {
+      tuple.put(new Datum[] {DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(i), DatumFactory.createText("dept_" + i)});
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+
+    employee = new TableDesc(
+        CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), employeeSchema, employeeMeta,
+        employeePath);
+    catalog.createTable(employee);
+
+    Path scorePath = new Path(testDir, "score");
+    TableMeta scoreMeta = CatalogUtil.newTableMeta(StoreType.CSV, new Options());
+    appender = StorageManagerFactory.getStorageManager(conf).getAppender(scoreMeta, scoreSchema, scorePath);
+    appender.init();
+    score = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "score"), scoreSchema, scoreMeta,
+        scorePath);
+    tuple = new VTuple(scoreSchema.size());
+    int m = 0;
+    for (int i = 1; i <= 5; i++) {
+      for (int k = 3; k < 5; k++) {
+        for (int j = 1; j <= 3; j++) {
+          tuple.put(
+              new Datum[] {
+                  DatumFactory.createText("name_" + i), // name_1 ~ 5 (cad: // 5)
+                  DatumFactory.createText(k + "rd"), // 3 or 4rd (cad: 2)
+                  DatumFactory.createInt4(j), // 1 ~ 3
+              m % 3 == 1 ? DatumFactory.createText("one") : NullDatum.get()});
+          appender.addTuple(tuple);
+          m++;
+        }
+      }
+    }
+    appender.flush();
+    appender.close();
+    catalog.createTable(score);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+    optimizer = new LogicalOptimizer(conf);
+
+    masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  private String[] QUERIES = {
+      "select name, empId, deptName from employee", // 0
+      "select name, empId, e.deptName, manager from employee as e, dept as dp", // 1
+      "select name, empId, e.deptName, manager, score from employee as e, dept, score", // 2
+      "select p.deptName, sum(score) from dept as p, score group by p.deptName having sum(score) > 30", // 3
+      "select p.deptName, score from dept as p, score order by score asc", // 4
+      "select name from employee where empId = 100", // 5
+      "select deptName, class, score from score", // 6
+      "select deptName, class, sum(score), max(score), min(score) from score group by deptName, class", // 7
+      "select count(*), max(score), min(score) from score", // 8
+      "select count(deptName) from score", // 9
+      "select managerId, empId, deptName from employee order by managerId, empId desc", // 10
+      "select deptName, nullable from score group by deptName, nullable", // 11
+      "select 3 < 4 as ineq, 3.5 * 2 as score", // 12
+      "select (1 > 0) and 3 > 1", // 13
+      "select sum(score), max(score), min(score) from score", // 14
+      "select deptname, sum(score), max(score), min(score) from score group by deptname", // 15
+      "select name from employee where empid >= 0", // 16
+  };
+
+  @Test
+  public final void testCreateScanPlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+        employee.getPath(), Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanPlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalNode rootNode =plan.getRootBlock().getRoot();
+    optimizer.optimize(plan);
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    Tuple tuple;
+    int i = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      assertTrue(tuple.contains(0));
+      assertTrue(tuple.contains(1));
+      assertTrue(tuple.contains(2));
+      i++;
+    }
+    exec.close();
+    assertEquals(100, i);
+  }
+
+  @Test
+  public final void testCreateScanWithFilterPlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+        employee.getPath(), Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanWithFilterPlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[16]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalNode rootNode =plan.getRootBlock().getRoot();
+    optimizer.optimize(plan);
+
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    Tuple tuple;
+    int i = 0;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      assertTrue(tuple.contains(0));
+      i++;
+    }
+    exec.close();
+    assertEquals(100, i);
+  }
+
+  @Test
+  public final void testGroupByPlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByPlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[7]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    optimizer.optimize(plan);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    int i = 0;
+    Tuple tuple;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
+      i++;
+    }
+    exec.close();
+    assertEquals(10, i);
+  }
+
+  @Test
+  public final void testHashGroupByPlanWithALLField() throws IOException, PlanningException {
+    // TODO - currently, this query does not use hash-based group operator.
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir(
+        "target/test-data/testHashGroupByPlanWithALLField");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[15]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    int i = 0;
+    Tuple tuple;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      assertEquals(12, tuple.get(1).asInt4()); // sum
+      assertEquals(3, tuple.get(2).asInt4()); // max
+      assertEquals(1, tuple.get(3).asInt4()); // min
+      i++;
+    }
+    exec.close();
+    assertEquals(5, i);
+  }
+
+  @Test
+  public final void testSortGroupByPlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortGroupByPlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[]{frags[0]}, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[7]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    optimizer.optimize(plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan.getRootBlock().getRoot());
+
+    /*HashAggregateExec hashAgg = (HashAggregateExec) exec;
+
+    SeqScanExec scan = (SeqScanExec) hashAgg.getSubOp();
+
+    Column [] grpColumns = hashAgg.getAnnotation().getGroupingColumns();
+    QueryBlock.SortSpec [] specs = new QueryBlock.SortSpec[grpColumns.length];
+    for (int i = 0; i < grpColumns.length; i++) {
+      specs[i] = new QueryBlock.SortSpec(grpColumns[i], true, false);
+    }
+    SortNode annotation = new SortNode(specs);
+    annotation.setInSchema(scan.getSchema());
+    annotation.setOutSchema(scan.getSchema());
+    SortExec sort = new SortExec(annotation, scan);
+    exec = new SortAggregateExec(hashAgg.getAnnotation(), sort);*/
+
+    int i = 0;
+    Tuple tuple;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
+      i++;
+    }
+    assertEquals(10, i);
+
+    exec.rescan();
+    i = 0;
+    while ((tuple = exec.next()) != null) {
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
+      i++;
+    }
+    exec.close();
+    assertEquals(10, i);
+  }
+
+  private String[] CreateTableAsStmts = {
+      "create table grouped1 as select deptName, class, sum(score), max(score), min(score) from score group by deptName, class", // 0
+      "create table grouped2 using rcfile as select deptName, class, sum(score), max(score), min(score) from score group by deptName, class", // 1
+      "create table grouped3 partition by column (dept text,  class text) as select sum(score), max(score), min(score), deptName, class from score group by deptName, class", // 2
+  };
+
+  @Test
+  public final void testStorePlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    ctx.setOutputPath(new Path(workDir, "grouped1"));
+
+    Expr context = analyzer.parse(CreateTableAsStmts[0]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+
+    TableMeta outputMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getFileScanner(outputMeta, rootNode.getOutSchema(),
+        ctx.getOutputPath());
+    scanner.init();
+    Tuple tuple;
+    int i = 0;
+    while ((tuple = scanner.next()) != null) {
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
+      i++;
+    }
+    assertEquals(10, i);
+    scanner.close();
+
+    // Examine the statistics information
+    assertEquals(10, ctx.getResultStats().getNumRows().longValue());
+  }
+
+  @Test
+  public final void testStorePlanWithRCFile() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlanWithRCFile");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    ctx.setOutputPath(new Path(workDir, "grouped2"));
+
+    Expr context = analyzer.parse(CreateTableAsStmts[1]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    TableMeta outputMeta = CatalogUtil.newTableMeta(StoreType.RCFILE);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getFileScanner(outputMeta, rootNode.getOutSchema(),
+        ctx.getOutputPath());
+    scanner.init();
+    Tuple tuple;
+    int i = 0;
+    while ((tuple = scanner.next()) != null) {
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
+      i++;
+    }
+    assertEquals(10, i);
+    scanner.close();
+
+    // Examine the statistics information
+    assertEquals(10, ctx.getResultStats().getNumRows().longValue());
+  }
+
+  @Test
+  public final void testEnforceForDefaultColumnPartitionStorePlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    ctx.setOutputPath(new Path(workDir, "grouped3"));
+
+    Expr context = analyzer.parse(CreateTableAsStmts[2]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    assertTrue(exec instanceof SortBasedColPartitionStoreExec);
+  }
+
+  @Test
+  public final void testEnforceForHashBasedColumnPartitionStorePlan() throws IOException, PlanningException {
+
+    Expr context = analyzer.parse(CreateTableAsStmts[2]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalRootNode rootNode = (LogicalRootNode) optimizer.optimize(plan);
+    CreateTableNode createTableNode = rootNode.getChild();
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.HASH_PARTITION);
+
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(enforcer);
+    ctx.setOutputPath(new Path(workDir, "grouped4"));
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    assertTrue(exec instanceof HashBasedColPartitionStoreExec);
+  }
+
+  @Test
+  public final void testEnforceForSortBasedColumnPartitionStorePlan() throws IOException, PlanningException {
+
+    Expr context = analyzer.parse(CreateTableAsStmts[2]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalRootNode rootNode = (LogicalRootNode) optimizer.optimize(plan);
+    CreateTableNode createTableNode = rootNode.getChild();
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.SORT_PARTITION);
+
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(enforcer);
+    ctx.setOutputPath(new Path(workDir, "grouped5"));
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    assertTrue(exec instanceof SortBasedColPartitionStoreExec);
+  }
+
+  @Test
+  public final void testPartitionedStorePlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, id, new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[7]);
+    LogicalPlan plan = planner.createPlan(session, context);
+
+    int numPartitions = 3;
+    Column key1 = new Column("default.score.deptname", Type.TEXT);
+    Column key2 = new Column("default.score.class", Type.TEXT);
+    DataChannel dataChannel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
+        ShuffleType.HASH_SHUFFLE, numPartitions);
+    dataChannel.setShuffleKeys(new Column[]{key1, key2});
+    ctx.setDataChannel(dataChannel);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    TableMeta outputMeta = CatalogUtil.newTableMeta(dataChannel.getStoreType());
+
+    FileSystem fs = sm.getFileSystem();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    Path path = new Path(workDir, "output");
+    FileStatus [] list = fs.listStatus(path);
+    assertEquals(numPartitions, list.length);
+
+    FileFragment[] fragments = new FileFragment[list.length];
+    int i = 0;
+    for (FileStatus status : list) {
+      fragments[i++] = new FileFragment("partition", status.getPath(), 0, status.getLen());
+    }
+    Scanner scanner = new MergeScanner(conf, rootNode.getOutSchema(), outputMeta, TUtil.newList(fragments));
+    scanner.init();
+
+    Tuple tuple;
+    i = 0;
+    while ((tuple = scanner.next()) != null) {
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
+      i++;
+    }
+    assertEquals(10, i);
+    scanner.close();
+
+    // Examine the statistics information
+    assertEquals(10, ctx.getResultStats().getNumRows().longValue());
+  }
+
+  @Test
+  public final void testPartitionedStorePlanWithEmptyGroupingSet()
+      throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
+
+    Path workDir = CommonTestingUtil.getTestDir(
+        "target/test-data/testPartitionedStorePlanWithEmptyGroupingSet");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, id, new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[14]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+    int numPartitions = 1;
+    DataChannel dataChannel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
+        ShuffleType.HASH_SHUFFLE, numPartitions);
+    dataChannel.setShuffleKeys(new Column[]{});
+    ctx.setDataChannel(dataChannel);
+    optimizer.optimize(plan);
+
+    TableMeta outputMeta = CatalogUtil.newTableMeta(dataChannel.getStoreType());
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    Path path = new Path(workDir, "output");
+    FileSystem fs = sm.getFileSystem();
+
+    FileStatus [] list = fs.listStatus(path);
+    assertEquals(numPartitions, list.length);
+
+    FileFragment[] fragments = new FileFragment[list.length];
+    int i = 0;
+    for (FileStatus status : list) {
+      fragments[i++] = new FileFragment("partition", status.getPath(), 0, status.getLen());
+    }
+    Scanner scanner = new MergeScanner(conf, rootNode.getOutSchema(), outputMeta, TUtil.newList(fragments));
+    scanner.init();
+    Tuple tuple;
+    i = 0;
+    while ((tuple = scanner.next()) != null) {
+      assertEquals(60, tuple.get(0).asInt4()); // sum
+      assertEquals(3, tuple.get(1).asInt4()); // max
+      assertEquals(1, tuple.get(2).asInt4()); // min
+      i++;
+    }
+    assertEquals(1, i);
+    scanner.close();
+
+    // Examine the statistics information
+    assertEquals(1, ctx.getResultStats().getNumRows().longValue());
+  }
+
+  @Test
+  public final void testAggregationFunction() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testAggregationFunction");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[8]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    // Set all aggregation functions to the first phase mode
+    GroupbyNode groupbyNode = PlannerUtil.findTopNode(rootNode, NodeType.GROUP_BY);
+    for (AggregationFunctionCallEval function : groupbyNode.getAggFunctions()) {
+      function.setFirstPhase();
+    }
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    exec.init();
+    Tuple tuple = exec.next();
+    assertEquals(30, tuple.get(0).asInt8());
+    assertEquals(3, tuple.get(1).asInt4());
+    assertEquals(1, tuple.get(2).asInt4());
+    assertNull(exec.next());
+    exec.close();
+  }
+
+  @Test
+  public final void testCountFunction() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCountFunction");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[9]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    // Set all aggregation functions to the first phase mode
+    GroupbyNode groupbyNode = PlannerUtil.findTopNode(rootNode, NodeType.GROUP_BY);
+    for (AggregationFunctionCallEval function : groupbyNode.getAggFunctions()) {
+      function.setFirstPhase();
+    }
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    Tuple tuple = exec.next();
+    assertEquals(30, tuple.get(0).asInt8());
+    assertNull(exec.next());
+    exec.close();
+  }
+
+  @Test
+  public final void testGroupByWithNullValue() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByWithNullValue");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[11]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    int count = 0;
+    exec.init();
+    while(exec.next() != null) {
+      count++;
+    }
+    exec.close();
+    assertEquals(10, count);
+  }
+
+  @Test
+  public final void testUnionPlan() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testUnionPlan");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr  context = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+    LogicalRootNode root = (LogicalRootNode) rootNode;
+    UnionNode union = plan.createNode(UnionNode.class);
+    union.setLeftChild(root.getChild());
+    union.setRightChild(root.getChild());
+    root.setChild(union);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, root);
+
+    int count = 0;
+    exec.init();
+    while(exec.next() != null) {
+      count++;
+    }
+    exec.close();
+    assertEquals(200, count);
+  }
+
+  @Test
+  public final void testEvalExpr() throws IOException, PlanningException {
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEvalExpr");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] { }, workDir);
+    Expr expr = analyzer.parse(QUERIES[12]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    Tuple tuple;
+    exec.init();
+    tuple = exec.next();
+    exec.close();
+    assertEquals(true, tuple.get(0).asBool());
+    assertTrue(7.0d == tuple.get(1).asFloat8());
+
+    expr = analyzer.parse(QUERIES[13]);
+    plan = planner.createPlan(session, expr);
+    rootNode = optimizer.optimize(plan);
+
+    phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    tuple = exec.next();
+    exec.close();
+    assertEquals(DatumFactory.createBool(true), tuple.get(0));
+  }
+
+  public final String [] createIndexStmt = {
+      "create index idx_employee on employee using bst (name null first, empId desc)"
+  };
+
+  //@Test
+  public final void testCreateIndex() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateIndex");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] {frags[0]}, workDir);
+    Expr context = analyzer.parse(createIndexStmt[0]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    while (exec.next() != null) {
+    }
+    exec.close();
+
+    FileStatus [] list = sm.getFileSystem().listStatus(StorageUtil.concatPath(workDir, "index"));
+    assertEquals(2, list.length);
+  }
+
+  final static String [] duplicateElimination = {
+      "select distinct deptname from score",
+  };
+
+  @Test
+  public final void testDuplicateEliminate() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
+        Integer.MAX_VALUE);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testDuplicateEliminate");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] {frags[0]}, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(duplicateElimination[0]);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    Tuple tuple;
+
+    int cnt = 0;
+    Set<String> expected = Sets.newHashSet(
+        "name_1", "name_2", "name_3", "name_4", "name_5");
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      assertTrue(expected.contains(tuple.get(0).asChars()));
+      cnt++;
+    }
+    exec.close();
+    assertEquals(5, cnt);
+  }
+
+  public String [] SORT_QUERY = {
+      "select name, empId from employee order by empId"
+  };
+
+  @Test
+  public final void testIndexedStoreExec() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+        employee.getPath(), Integer.MAX_VALUE);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testIndexedStoreExec");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] {frags[0]}, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(SORT_QUERY[0]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    SortNode sortNode = PlannerUtil.findTopNode(rootNode, NodeType.SORT);
+    DataChannel channel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
+        TajoWorkerProtocol.ShuffleType.RANGE_SHUFFLE);
+    channel.setShuffleKeys(PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()).toArray());
+    ctx.setDataChannel(channel);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    Tuple tuple;
+    exec.init();
+    exec.next();
+    exec.close();
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn("?empId", Type.INT4);
+    SortSpec[] sortSpec = new SortSpec[1];
+    sortSpec[0] = new SortSpec(keySchema.getColumn(0), true, false);
+    TupleComparator comp = new TupleComparator(keySchema, sortSpec);
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndex.BSTIndexReader reader = bst.getIndexReader(new Path(workDir, "output/index"),
+        keySchema, comp);
+    reader.open();
+    Path outputPath = StorageUtil.concatPath(workDir, "output", "output");
+    TableMeta meta = CatalogUtil.newTableMeta(channel.getStoreType(), new Options());
+    SeekableScanner scanner =
+        StorageManagerFactory.getSeekableScanner(conf, meta, exec.getSchema(), outputPath);
+    scanner.init();
+
+    int cnt = 0;
+
+    while(scanner.next() != null) {
+      cnt++;
+    }
+    scanner.reset();
+
+    assertEquals(100 ,cnt);
+
+    Tuple keytuple = new VTuple(1);
+    for(int i = 1 ; i < 100 ; i ++) {
+      keytuple.put(0, DatumFactory.createInt4(i));
+      long offsets = reader.find(keytuple);
+      scanner.seek(offsets);
+      tuple = scanner.next();
+
+      assertTrue("[seek check " + (i) + " ]", ("name_" + i).equals(tuple.get(0).asChars()));
+      assertTrue("[seek check " + (i) + " ]" , i == tuple.get(1).asInt4());
+    }
+
+
+    // The below is for testing RangeRetrieverHandler.
+    RowStoreEncoder encoder = RowStoreUtil.createEncoder(keySchema);
+    RangeRetrieverHandler handler = new RangeRetrieverHandler(
+        new File(new Path(workDir, "output").toUri()), keySchema, comp);
+    Map<String,List<String>> kvs = Maps.newHashMap();
+    Tuple startTuple = new VTuple(1);
+    startTuple.put(0, DatumFactory.createInt4(50));
+    kvs.put("start", Lists.newArrayList(
+        new String(Base64.encodeBase64(
+            encoder.toBytes(startTuple), false))));
+    Tuple endTuple = new VTuple(1);
+    endTuple.put(0, DatumFactory.createInt4(80));
+    kvs.put("end", Lists.newArrayList(
+        new String(Base64.encodeBase64(
+            encoder.toBytes(endTuple), false))));
+    FileChunk chunk = handler.get(kvs);
+
+    scanner.seek(chunk.startOffset());
+    keytuple = scanner.next();
+    assertEquals(50, keytuple.get(1).asInt4());
+
+    long endOffset = chunk.startOffset() + chunk.length();
+    while((keytuple = scanner.next()) != null && scanner.getNextOffset() <= endOffset) {
+      assertTrue(keytuple.get(1).asInt4() <= 80);
+    }
+
+    scanner.close();
+  }
+
+  @Test
+  public final void testSortEnforcer() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+        employee.getPath(), Integer.MAX_VALUE);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortEnforcer");
+    Expr context = analyzer.parse(SORT_QUERY[0]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    optimizer.optimize(plan);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+
+    SortNode sortNode = PlannerUtil.findTopNode(rootNode, NodeType.SORT);
+
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceSortAlgorithm(sortNode.getPID(), SortAlgorithm.IN_MEMORY_SORT);
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] {frags[0]}, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    assertTrue(exec instanceof MemSortExec);
+
+    context = analyzer.parse(SORT_QUERY[0]);
+    plan = planner.createPlan(session, context);
+    optimizer.optimize(plan);
+    rootNode = plan.getRootBlock().getRoot();
+
+    sortNode = PlannerUtil.findTopNode(rootNode, NodeType.SORT);
+
+    enforcer = new Enforcer();
+    enforcer.enforceSortAlgorithm(sortNode.getPID(), SortAlgorithm.MERGE_SORT);
+    ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] {frags[0]}, workDir);
+    ctx.setEnforcer(enforcer);
+
+    phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    assertTrue(exec instanceof ExternalSortExec);
+  }
+
+  @Test
+  public final void testGroupByEnforcer() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), Integer.MAX_VALUE);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByEnforcer");
+    Expr context = analyzer.parse(QUERIES[7]);
+    LogicalPlan plan = planner.createPlan(session, context);
+    optimizer.optimize(plan);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+
+    GroupbyNode groupByNode = PlannerUtil.findTopNode(rootNode, NodeType.GROUP_BY);
+
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceHashAggregation(groupByNode.getPID());
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] {frags[0]}, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    assertNotNull(PhysicalPlanUtil.findExecutor(exec, HashAggregateExec.class));
+
+    context = analyzer.parse(QUERIES[7]);
+    plan = planner.createPlan(session, context);
+    optimizer.optimize(plan);
+    rootNode = plan.getRootBlock().getRoot();
+
+    groupByNode = PlannerUtil.findTopNode(rootNode, NodeType.GROUP_BY);
+
+    enforcer = new Enforcer();
+    enforcer.enforceSortAggregation(groupByNode.getPID(), null);
+    ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+        new FileFragment[] {frags[0]}, workDir);
+    ctx.setEnforcer(enforcer);
+
+    phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    exec = phyPlanner.createPlan(ctx, rootNode);
+    exec.init();
+    exec.next();
+    exec.close();
+
+    assertTrue(exec instanceof SortAggregateExec);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
new file mode 100644
index 0000000..c60e05c
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
@@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Random;
+
+import static junit.framework.Assert.assertNotNull;
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestProgressExternalSortExec {
+  private TajoConf conf;
+  private TajoTestingCluster util;
+  private final String TEST_PATH = "target/test-data/TestProgressExternalSortExec";
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+
+  private final int numTuple = 100000;
+  private Random rnd = new Random(System.currentTimeMillis());
+
+  private TableDesc employee;
+
+  private TableStats testDataStats;
+  @Before
+  public void setUp() throws Exception {
+    this.conf = new TajoConf();
+    util = new TajoTestingCluster();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf.setVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR, testDir.toString());
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    Schema schema = new Schema();
+    schema.addColumn("managerid", TajoDataTypes.Type.INT4);
+    schema.addColumn("empid", TajoDataTypes.Type.INT4);
+    schema.addColumn("deptname", TajoDataTypes.Type.TEXT);
+
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.RAW);
+    Path employeePath = new Path(testDir, "employee.csv");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    appender.enableStats();
+    appender.init();
+    Tuple tuple = new VTuple(schema.size());
+    for (int i = 0; i < numTuple; i++) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(rnd.nextInt(50)),
+          DatumFactory.createInt4(rnd.nextInt(100)),
+          DatumFactory.createText("dept_" + i),
+      });
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+
+    System.out.println(appender.getStats().getNumRows() + " rows (" + appender.getStats().getNumBytes() + " Bytes)");
+
+    testDataStats = appender.getStats();
+    employee = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, employeeMeta,
+        employeePath);
+    catalog.createTable(employee);
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    CommonTestingUtil.cleanupTestDir(TEST_PATH);
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      "select managerId, empId from employee order by managerId, empId"
+  };
+
+  @Test
+  public void testExternalSortExecProgressWithMemTableScanner() throws Exception {
+    testProgress(testDataStats.getNumBytes().intValue() * 20);    //multiply 20 for memory fit
+  }
+
+  @Test
+  public void testExternalSortExecProgressWithPairWiseMerger() throws Exception {
+    testProgress(testDataStats.getNumBytes().intValue());
+  }
+
+  private void testProgress(int sortBufferBytesNum) throws Exception {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
+        Integer.MAX_VALUE);
+    Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalNode rootNode = plan.getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+
+    // TODO - should be planed with user's optimization hint
+    if (!(proj.getChild() instanceof ExternalSortExec)) {
+      UnaryPhysicalExec sortExec = proj.getChild();
+      SeqScanExec scan = sortExec.getChild();
+
+      ExternalSortExec extSort = new ExternalSortExec(ctx, sm,
+          ((MemSortExec)sortExec).getPlan(), scan);
+
+      extSort.setSortBufferBytesNum(sortBufferBytesNum);
+      proj.setChild(extSort);
+    } else {
+      ((ExternalSortExec)proj.getChild()).setSortBufferBytesNum(sortBufferBytesNum);
+    }
+
+    Tuple tuple;
+    Tuple preVal = null;
+    Tuple curVal;
+    int cnt = 0;
+    exec.init();
+    TupleComparator comparator = new TupleComparator(proj.getSchema(),
+        new SortSpec[]{
+            new SortSpec(new Column("managerid", TajoDataTypes.Type.INT4)),
+            new SortSpec(new Column("empid", TajoDataTypes.Type.INT4))
+        });
+
+    float initProgress = 0.0f;
+    while ((tuple = exec.next()) != null) {
+      if (cnt == 0) {
+        initProgress = exec.getProgress();
+        assertTrue(initProgress > 0.5f && initProgress < 1.0f);
+      }
+
+      if (cnt == testDataStats.getNumRows() / 2) {
+        float progress = exec.getProgress();
+
+        assertTrue(progress > initProgress);
+      }
+      curVal = tuple;
+      if (preVal != null) {
+        assertTrue("prev: " + preVal + ", but cur: " + curVal, comparator.compare(preVal, curVal) <= 0);
+      }
+      preVal = curVal;
+      cnt++;
+    }
+
+    assertEquals(1.0f, exec.getProgress(), 0);
+    assertEquals(numTuple, cnt);
+
+    TableStats tableStats = exec.getInputStats();
+    assertNotNull(tableStats);
+    assertEquals(testDataStats.getNumBytes().longValue(), tableStats.getNumBytes().longValue());
+    assertEquals(cnt, testDataStats.getNumRows().longValue());
+    assertEquals(cnt, tableStats.getNumRows().longValue());
+    assertEquals(testDataStats.getNumBytes().longValue(), tableStats.getReadBytes().longValue());
+
+    // for rescan test
+    preVal = null;
+    exec.rescan();
+
+    cnt = 0;
+    while ((tuple = exec.next()) != null) {
+      curVal = tuple;
+      if (preVal != null) {
+        assertTrue("prev: " + preVal + ", but cur: " + curVal, comparator.compare(preVal, curVal) <= 0);
+      }
+      preVal = curVal;
+      cnt++;
+    }
+    assertEquals(1.0f, exec.getProgress(), 0);
+    assertEquals(numTuple, cnt);
+    exec.close();
+    assertEquals(1.0f, exec.getProgress(), 0);
+
+    tableStats = exec.getInputStats();
+    assertNotNull(tableStats);
+    assertEquals(testDataStats.getNumBytes().longValue(), tableStats.getNumBytes().longValue());
+    assertEquals(cnt, testDataStats.getNumRows().longValue());
+    assertEquals(cnt, tableStats.getNumRows().longValue());
+    assertEquals(testDataStats.getNumBytes().longValue(), tableStats.getReadBytes().longValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
new file mode 100644
index 0000000..a45e397
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
@@ -0,0 +1,350 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlanner;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+
+// this is not a physical operator in itself, but it uses the HashLeftOuterJoinExec with switched inputs order
+public class TestRightOuterHashJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestRightOuterHashJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+  private static Session session = LocalTajoTestingUtility.createDummySession();
+
+  private TableDesc dep3;
+  private TableDesc job3;
+  private TableDesc emp3;
+
+  private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3");
+  private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3");
+  private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3");
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    //----------------- dep3 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    Schema dep3Schema = new Schema();
+    dep3Schema.addColumn("dep_id", Type.INT4);
+    dep3Schema.addColumn("dep_name", Type.TEXT);
+    dep3Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep3Path = new Path(testDir, "dep3.csv");
+    Appender appender1 = StorageManagerFactory.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    appender1.init();
+    Tuple tuple = new VTuple(dep3Schema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+                    DatumFactory.createText("dept_" + i),
+                    DatumFactory.createInt4(1000 + i) });
+      appender1.addTuple(tuple);
+    }
+
+    appender1.flush();
+    appender1.close();
+    dep3 = CatalogUtil.newTableDesc(DEP3_NAME, dep3Schema, dep3Meta, dep3Path);
+    catalog.createTable(dep3);
+
+    //----------------- job3 ------------------------------
+    //  job_id  | job_title
+    // ----------------------
+    //   101    |  job_101
+    //   102    |  job_102
+    //   103    |  job_103
+
+    Schema job3Schema = new Schema();
+    job3Schema.addColumn("job_id", Type.INT4);
+    job3Schema.addColumn("job_title", Type.TEXT);
+
+
+    TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path job3Path = new Path(testDir, "job3.csv");
+    Appender appender2 = StorageManagerFactory.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    appender2.init();
+    Tuple tuple2 = new VTuple(job3Schema.size());
+    for (int i = 1; i < 4; i++) {
+      int x = 100 + i;
+      tuple2.put(new Datum[] { DatumFactory.createInt4(100 + i),
+                    DatumFactory.createText("job_" + x) });
+      appender2.addTuple(tuple2);
+    }
+
+    appender2.flush();
+    appender2.close();
+    job3 = CatalogUtil.newTableDesc(JOB3_NAME, job3Schema, job3Meta, job3Path);
+    catalog.createTable(job3);
+
+
+
+    //---------------------emp3 --------------------
+    // emp_id  | first_name | last_name | dep_id | salary | job_id
+    // ------------------------------------------------------------
+    //  11     |  fn_11     |  ln_11    |  1     | 123    | 101
+    //  13     |  fn_13     |  ln_13    |  3     | 369    | 103
+    //  15     |  fn_15     |  ln_15    |  5     | 615    | null
+    //  17     |  fn_17     |  ln_17    |  7     | 861    | null
+    //  19     |  fn_19     |  ln_19    |  9     | 1107   | null
+    //  21     |  fn_21     |  ln_21    |  1     | 123    | 101
+    //  23     |  fn_23     |  ln_23    |  3     | 369    | 103
+
+    Schema emp3Schema = new Schema();
+    emp3Schema.addColumn("emp_id", Type.INT4);
+    emp3Schema.addColumn("first_name", Type.TEXT);
+    emp3Schema.addColumn("last_name", Type.TEXT);
+    emp3Schema.addColumn("dep_id", Type.INT4);
+    emp3Schema.addColumn("salary", Type.FLOAT4);
+    emp3Schema.addColumn("job_id", Type.INT4);
+
+
+    TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path emp3Path = new Path(testDir, "emp3.csv");
+    Appender appender3 = StorageManagerFactory.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    appender3.init();
+    Tuple tuple3 = new VTuple(emp3Schema.size());
+
+    for (int i = 1; i < 4; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+
+      int y = 20 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(20 + i),
+          DatumFactory.createText("firstname_" + y),
+          DatumFactory.createText("lastname_" + y),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+    }
+
+    for (int i = 5; i < 10; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createNullDatum() });
+      appender3.addTuple(tuple3);
+    }
+
+    appender3.flush();
+    appender3.close();
+    emp3 = CatalogUtil.newTableDesc(EMP3_NAME, emp3Schema, emp3Meta, emp3Path);
+    catalog.createTable(emp3);
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      "select dep3.dep_id, dep_name, emp_id, salary from emp3 right outer join dep3 on dep3.dep_id = emp3.dep_id", //0 no nulls
+      "select job3.job_id, job_title, emp_id, salary from emp3 right outer join job3 on job3.job_id=emp3.job_id", //1 nulls on the left operand
+      "select job3.job_id, job_title, emp_id, salary from job3 right outer join emp3 on job3.job_id=emp3.job_id" //2 nulls on the right side
+  };
+
+  @Test
+  public final void testRightOuter_HashJoinExec0() throws IOException, PlanningException {
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestRightOuter_HashJoinExec0");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof RightOuterMergeJoinExec) {
+       //for this small data set this is not likely to happen
+      
+      assertEquals(1, 0);
+    }
+    else{
+       Tuple tuple;
+       int count = 0;
+       int i = 1;
+       exec.init();
+  
+       while ((tuple = exec.next()) != null) {
+         //TODO check contents
+         count = count + 1;
+       }
+       exec.close();
+       assertEquals(12, count);
+    }
+  }
+
+
+  @Test
+  public final void testRightOuter_HashJoinExec1() throws IOException, PlanningException {
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestRightOuter_HashJoinExec1");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof RightOuterMergeJoinExec) {
+       //for this small data set this is not likely to happen
+      
+      assertEquals(1, 0);
+    }
+    else{
+       Tuple tuple;
+       int count = 0;
+       int i = 1;
+       exec.init();
+  
+       while ((tuple = exec.next()) != null) {
+         //TODO check contents
+         count = count + 1;
+       }
+       exec.close();
+       assertEquals(5, count);
+    }
+  }
+
+    @Test
+  public final void testRightOuter_HashJoinExec2() throws IOException, PlanningException {
+    
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(),
+        Integer.MAX_VALUE);
+
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestRightOuter_HashJoinExec2");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr expr = analyzer.parse(QUERIES[2]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    if (proj.getChild() instanceof RightOuterMergeJoinExec) {
+      //for this small data set this is not likely to happen
+      
+      assertEquals(1, 0);
+    }
+    else{
+       Tuple tuple;
+       int count = 0;
+       int i = 1;
+       exec.init();
+  
+       while ((tuple = exec.next()) != null) {
+         //TODO check contents
+         count = count + 1;
+       }
+       exec.close();
+       assertEquals(7, count);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
new file mode 100644
index 0000000..5b504b2
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
@@ -0,0 +1,520 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.junit.Assert.*;
+
+public class TestRightOuterMergeJoinExec {
+  private TajoConf conf;
+  private final String TEST_PATH = "target/test-data/TestRightOuterMergeJoinExec";
+  private TajoTestingCluster util;
+  private CatalogService catalog;
+  private SQLAnalyzer analyzer;
+  private LogicalPlanner planner;
+  private AbstractStorageManager sm;
+  private Path testDir;
+  private static final Session session = LocalTajoTestingUtility.createDummySession();
+
+  private TableDesc dep3;
+  private TableDesc dep4;
+  private TableDesc job3;
+  private TableDesc emp3;
+  private TableDesc phone3;
+
+  private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3");
+  private final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4");
+  private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3");
+  private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3");
+  private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3");
+
+  @Before
+  public void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.initTestDir();
+    catalog = util.startCatalogCluster().getCatalog();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    conf = util.getConfiguration();
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+
+    //----------------- dep3 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    Schema dep3Schema = new Schema();
+    dep3Schema.addColumn("dep_id", Type.INT4);
+    dep3Schema.addColumn("dep_name", Type.TEXT);
+    dep3Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep3Path = new Path(testDir, "dep3.csv");
+    Appender appender1 = StorageManagerFactory.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    appender1.init();
+    Tuple tuple = new VTuple(dep3Schema.size());
+    for (int i = 0; i < 10; i++) {
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createText("dept_" + i),
+          DatumFactory.createInt4(1000 + i) });
+      appender1.addTuple(tuple);
+    }
+
+    appender1.flush();
+    appender1.close();
+    dep3 = CatalogUtil.newTableDesc(DEP3_NAME, dep3Schema, dep3Meta, dep3Path);
+    catalog.createTable(dep3);
+
+
+    //----------------- dep4 ------------------------------
+    // dep_id | dep_name  | loc_id
+    //--------------------------------
+    //  0     | dep_0     | 1000
+    //  1     | dep_1     | 1001
+    //  2     | dep_2     | 1002
+    //  3     | dep_3     | 1003
+    //  4     | dep_4     | 1004
+    //  5     | dep_5     | 1005
+    //  6     | dep_6     | 1006
+    //  7     | dep_7     | 1007
+    //  8     | dep_8     | 1008
+    //  9     | dep_9     | 1009
+    // 10     | dep_10    | 1010
+    Schema dep4Schema = new Schema();
+    dep4Schema.addColumn("dep_id", Type.INT4);
+    dep4Schema.addColumn("dep_name", Type.TEXT);
+    dep4Schema.addColumn("loc_id", Type.INT4);
+
+
+    TableMeta dep4Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path dep4Path = new Path(testDir, "dep4.csv");
+    Appender appender4 = StorageManagerFactory.getStorageManager(conf).getAppender(dep4Meta, dep4Schema, dep4Path);
+    appender4.init();
+    Tuple tuple4 = new VTuple(dep4Schema.size());
+    for (int i = 0; i < 11; i++) {
+      tuple4.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createText("dept_" + i),
+          DatumFactory.createInt4(1000 + i) });
+      appender4.addTuple(tuple4);
+    }
+
+    appender4.flush();
+    appender4.close();
+    dep4 = CatalogUtil.newTableDesc(DEP4_NAME, dep4Schema, dep4Meta, dep4Path);
+    catalog.createTable(dep4);
+
+
+
+    //----------------- job3 ------------------------------
+    //  job_id  | job_title
+    // ----------------------
+    //   101    |  job_101
+    //   102    |  job_102
+    //   103    |  job_103
+
+    Schema job3Schema = new Schema();
+    job3Schema.addColumn("job_id", Type.INT4);
+    job3Schema.addColumn("job_title", Type.TEXT);
+
+
+    TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path job3Path = new Path(testDir, "job3.csv");
+    Appender appender2 = StorageManagerFactory.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    appender2.init();
+    Tuple tuple2 = new VTuple(job3Schema.size());
+    for (int i = 1; i < 4; i++) {
+      int x = 100 + i;
+      tuple2.put(new Datum[] { DatumFactory.createInt4(100 + i),
+          DatumFactory.createText("job_" + x) });
+      appender2.addTuple(tuple2);
+    }
+
+    appender2.flush();
+    appender2.close();
+    job3 = CatalogUtil.newTableDesc(JOB3_NAME, job3Schema, job3Meta, job3Path);
+    catalog.createTable(job3);
+
+
+
+    //---------------------emp3 --------------------
+    // emp_id  | first_name | last_name | dep_id | salary | job_id
+    // ------------------------------------------------------------
+    //  11     |  fn_11     |  ln_11    |  1     | 123    | 101
+    //  13     |  fn_13     |  ln_13    |  3     | 369    | 103
+    //  15     |  fn_15     |  ln_15    |  5     | 615    | null
+    //  17     |  fn_17     |  ln_17    |  7     | 861    | null
+    //  19     |  fn_19     |  ln_19    |  9     | 1107   | null
+    //  21     |  fn_21     |  ln_21    |  1     | 123    | 101
+    //  23     |  fn_23     |  ln_23    |  3     | 369    | 103
+
+    Schema emp3Schema = new Schema();
+    emp3Schema.addColumn("emp_id", Type.INT4);
+    emp3Schema.addColumn("first_name", Type.TEXT);
+    emp3Schema.addColumn("last_name", Type.TEXT);
+    emp3Schema.addColumn("dep_id", Type.INT4);
+    emp3Schema.addColumn("salary", Type.FLOAT4);
+    emp3Schema.addColumn("job_id", Type.INT4);
+
+
+    TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path emp3Path = new Path(testDir, "emp3.csv");
+    Appender appender3 = StorageManagerFactory.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    appender3.init();
+    Tuple tuple3 = new VTuple(emp3Schema.size());
+
+    for (int i = 1; i < 4; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+
+      int y = 20 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(20 + i),
+          DatumFactory.createText("firstname_" + y),
+          DatumFactory.createText("lastname_" + y),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createInt4(100 + i) });
+      appender3.addTuple(tuple3);
+    }
+
+    for (int i = 5; i < 10; i += 2) {
+      int x = 10 + i;
+      tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("firstname_" + x),
+          DatumFactory.createText("lastname_" + x),
+          DatumFactory.createInt4(i),
+          DatumFactory.createFloat4(123 * i),
+          DatumFactory.createNullDatum() });
+      appender3.addTuple(tuple3);
+    }
+
+    appender3.flush();
+    appender3.close();
+    emp3 = CatalogUtil.newTableDesc(EMP3_NAME, emp3Schema, emp3Meta, emp3Path);
+    catalog.createTable(emp3);
+
+    //---------------------phone3 --------------------
+    // emp_id  | phone_number
+    // -----------------------------------------------
+    // this table is empty, no rows
+
+    Schema phone3Schema = new Schema();
+    phone3Schema.addColumn("emp_id", Type.INT4);
+    phone3Schema.addColumn("phone_number", Type.TEXT);
+
+
+    TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    Path phone3Path = new Path(testDir, "phone3.csv");
+    Appender appender5 = StorageManagerFactory.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
+        phone3Path);
+    appender5.init();
+
+    appender5.flush();
+    appender5.close();
+    phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path);
+    catalog.createTable(phone3);
+
+
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  String[] QUERIES = {
+      // [0] no nulls
+      "select dep3.dep_id, dep_name, emp_id, salary from emp3 right outer join dep3 on dep3.dep_id = emp3.dep_id",
+      // [1] nulls on the left operand
+      "select job3.job_id, job_title, emp_id, salary from emp3 right outer join job3 on job3.job_id=emp3.job_id",
+      // [2] nulls on the right side
+      "select job3.job_id, job_title, emp_id, salary from job3 right outer join emp3 on job3.job_id=emp3.job_id",
+      // [3] no nulls, right continues after left
+      "select dep4.dep_id, dep_name, emp_id, salary from emp3 right outer join dep4 on dep4.dep_id = emp3.dep_id",
+      // [4] one operand is empty
+      "select emp3.emp_id, first_name, phone_number from emp3 right outer join phone3 on emp3.emp_id = phone3.emp_id",
+      // [5] one operand is empty
+      "select phone_number, emp3.emp_id, first_name from phone3 right outer join emp3 on emp3.emp_id = phone3.emp_id"
+  };
+
+  @Test
+  public final void testRightOuterMergeJoin0() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin0");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
+
+    int count = 0;
+    exec.init();
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(12, count);
+  }
+
+
+  @Test
+  public final void testRightOuter_MergeJoin1() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] job3Frags =
+        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin1");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
+
+    int count = 0;
+    exec.init();
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(5, count);
+  }
+
+  @Test
+  public final void testRightOuterMergeJoin2() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[2]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] job3Frags =
+        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin2");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
+
+    int count = 0;
+    exec.init();
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(7, count);
+  }
+
+
+  @Test
+  public final void testRightOuter_MergeJoin3() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[3]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] dep4Frags =
+        StorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), dep4.getPath(), Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuter_MergeJoin3");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(13, count);
+  }
+
+  @Test
+  public final void testRightOuter_MergeJoin4() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[4]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] phone3Frags =
+        StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuter_MergeJoin4");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    assertNull(exec.next());
+    exec.close();
+    assertEquals(0, count);
+  }
+
+  @Test
+  public final void testRightOuterMergeJoin5() throws IOException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[5]);
+    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
+    Enforcer enforcer = new Enforcer();
+    enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
+
+    FileFragment[] emp3Frags =
+        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(),
+        Integer.MAX_VALUE);
+    FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags);
+
+
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin5");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf,
+        LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
+    ctx.setEnforcer(enforcer);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
+    ProjectionExec proj = (ProjectionExec) exec;
+    assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
+
+    int count = 0;
+    exec.init();
+
+    while (exec.next() != null) {
+      //TODO check contents
+      count = count + 1;
+    }
+    exec.close();
+    assertEquals(7, count);
+  }
+}


[37/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
new file mode 100644
index 0000000..2053e36
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
@@ -0,0 +1,1057 @@
+/**
+ * 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.planner;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.ObjectArrays;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.planner.physical.*;
+import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.IndexUtil;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Stack;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType;
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.GroupbyEnforce.GroupbyAlgorithm;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.SortEnforce;
+
+public class PhysicalPlannerImpl implements PhysicalPlanner {
+  private static final Log LOG = LogFactory.getLog(PhysicalPlannerImpl.class);
+  private static final int UNGENERATED_PID = -1;
+  private final long INNER_JOIN_INMEMORY_HASH_THRESHOLD;
+
+  protected final TajoConf conf;
+  protected final AbstractStorageManager sm;
+
+  public PhysicalPlannerImpl(final TajoConf conf, final AbstractStorageManager sm) {
+    this.conf = conf;
+    this.sm = sm;
+
+    this.INNER_JOIN_INMEMORY_HASH_THRESHOLD = conf.getLongVar(ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD);
+  }
+
+  public PhysicalExec createPlan(final TaskAttemptContext context, final LogicalNode logicalPlan)
+      throws InternalException {
+
+    PhysicalExec execPlan;
+
+    try {
+      execPlan = createPlanRecursive(context, logicalPlan, new Stack<LogicalNode>());
+      if (execPlan instanceof StoreTableExec
+          || execPlan instanceof RangeShuffleFileWriteExec
+          || execPlan instanceof HashShuffleFileWriteExec
+          || execPlan instanceof ColPartitionStoreExec) {
+        return execPlan;
+      } else if (context.getDataChannel() != null) {
+        return buildOutputOperator(context, logicalPlan, execPlan);
+      } else {
+        return execPlan;
+      }
+    } catch (IOException ioe) {
+      throw new InternalException(ioe);
+    }
+  }
+
+  private PhysicalExec buildOutputOperator(TaskAttemptContext context, LogicalNode plan,
+                                           PhysicalExec execPlan) throws IOException {
+    DataChannel channel = context.getDataChannel();
+    ShuffleFileWriteNode shuffleFileWriteNode = LogicalPlan.createNodeWithoutPID(ShuffleFileWriteNode.class);
+    shuffleFileWriteNode.setStorageType(context.getDataChannel().getStoreType());
+    shuffleFileWriteNode.setInSchema(plan.getOutSchema());
+    shuffleFileWriteNode.setOutSchema(plan.getOutSchema());
+    shuffleFileWriteNode.setShuffle(channel.getShuffleType(), channel.getShuffleKeys(), channel.getShuffleOutputNum());
+    shuffleFileWriteNode.setChild(plan);
+
+    PhysicalExec outExecPlan = createShuffleFileWritePlan(context, shuffleFileWriteNode, execPlan);
+    return outExecPlan;
+  }
+
+  private PhysicalExec createPlanRecursive(TaskAttemptContext ctx, LogicalNode logicalNode, Stack<LogicalNode> stack)
+      throws IOException {
+    PhysicalExec leftExec;
+    PhysicalExec rightExec;
+
+    switch (logicalNode.getType()) {
+
+      case ROOT:
+        LogicalRootNode rootNode = (LogicalRootNode) logicalNode;
+        stack.push(rootNode);
+        leftExec = createPlanRecursive(ctx, rootNode.getChild(), stack);
+        stack.pop();
+        return leftExec;
+
+      case EXPRS:
+        EvalExprNode evalExpr = (EvalExprNode) logicalNode;
+        return new EvalExprExec(ctx, evalExpr);
+
+      case CREATE_TABLE:
+      case INSERT:
+      case STORE:
+        StoreTableNode storeNode = (StoreTableNode) logicalNode;
+        stack.push(storeNode);
+        leftExec = createPlanRecursive(ctx, storeNode.getChild(), stack);
+        stack.pop();
+        return createStorePlan(ctx, storeNode, leftExec);
+
+      case SELECTION:
+        SelectionNode selNode = (SelectionNode) logicalNode;
+        stack.push(selNode);
+        leftExec = createPlanRecursive(ctx, selNode.getChild(), stack);
+        stack.pop();
+        return new SelectionExec(ctx, selNode, leftExec);
+
+      case PROJECTION:
+        ProjectionNode prjNode = (ProjectionNode) logicalNode;
+        stack.push(prjNode);
+        leftExec = createPlanRecursive(ctx, prjNode.getChild(), stack);
+        stack.pop();
+        return new ProjectionExec(ctx, prjNode, leftExec);
+
+      case TABLE_SUBQUERY: {
+        TableSubQueryNode subQueryNode = (TableSubQueryNode) logicalNode;
+        stack.push(subQueryNode);
+        leftExec = createPlanRecursive(ctx, subQueryNode.getSubQuery(), stack);
+        stack.pop();
+        return new ProjectionExec(ctx, subQueryNode, leftExec);
+
+      }
+
+      case PARTITIONS_SCAN:
+      case SCAN:
+        leftExec = createScanPlan(ctx, (ScanNode) logicalNode, stack);
+        return leftExec;
+
+      case GROUP_BY:
+        GroupbyNode grpNode = (GroupbyNode) logicalNode;
+        stack.push(grpNode);
+        leftExec = createPlanRecursive(ctx, grpNode.getChild(), stack);
+        stack.pop();
+        return createGroupByPlan(ctx, grpNode, leftExec);
+
+      case HAVING:
+        HavingNode havingNode = (HavingNode) logicalNode;
+        stack.push(havingNode);
+        leftExec = createPlanRecursive(ctx, havingNode.getChild(), stack);
+        stack.pop();
+        return new HavingExec(ctx, havingNode, leftExec);
+
+      case SORT:
+        SortNode sortNode = (SortNode) logicalNode;
+        stack.push(sortNode);
+        leftExec = createPlanRecursive(ctx, sortNode.getChild(), stack);
+        stack.pop();
+        return createSortPlan(ctx, sortNode, leftExec);
+
+      case JOIN:
+        JoinNode joinNode = (JoinNode) logicalNode;
+        stack.push(joinNode);
+        leftExec = createPlanRecursive(ctx, joinNode.getLeftChild(), stack);
+        rightExec = createPlanRecursive(ctx, joinNode.getRightChild(), stack);
+        stack.pop();
+        return createJoinPlan(ctx, joinNode, leftExec, rightExec);
+
+      case UNION:
+        UnionNode unionNode = (UnionNode) logicalNode;
+        stack.push(unionNode);
+        leftExec = createPlanRecursive(ctx, unionNode.getLeftChild(), stack);
+        rightExec = createPlanRecursive(ctx, unionNode.getRightChild(), stack);
+        stack.pop();
+        return new UnionExec(ctx, leftExec, rightExec);
+
+      case LIMIT:
+        LimitNode limitNode = (LimitNode) logicalNode;
+        stack.push(limitNode);
+        leftExec = createPlanRecursive(ctx, limitNode.getChild(), stack);
+        stack.pop();
+        return new LimitExec(ctx, limitNode.getInSchema(),
+            limitNode.getOutSchema(), leftExec, limitNode);
+
+      case BST_INDEX_SCAN:
+        IndexScanNode indexScanNode = (IndexScanNode) logicalNode;
+        leftExec = createIndexScanExec(ctx, indexScanNode);
+        return leftExec;
+
+      default:
+        return null;
+    }
+  }
+
+  @VisibleForTesting
+  public long estimateSizeRecursive(TaskAttemptContext ctx, String [] tableIds) throws IOException {
+    long size = 0;
+    for (String tableId : tableIds) {
+      // TODO - CSV is a hack.
+      List<FileFragment> fragments = FragmentConvertor.convert(ctx.getConf(), CatalogProtos.StoreType.CSV,
+          ctx.getTables(tableId));
+      for (FileFragment frag : fragments) {
+        size += frag.getEndKey();
+      }
+    }
+    return size;
+  }
+
+  @VisibleForTesting
+  public boolean checkIfInMemoryInnerJoinIsPossible(TaskAttemptContext context, LogicalNode node, boolean left)
+      throws IOException {
+    String [] lineage = PlannerUtil.getRelationLineage(node);
+    long volume = estimateSizeRecursive(context, lineage);
+    boolean inMemoryInnerJoinFlag = volume <= INNER_JOIN_INMEMORY_HASH_THRESHOLD;
+    LOG.info(String.format("[%s] the volume of %s relations (%s) is %s and is %sfit to main maemory.",
+        context.getTaskId().toString(),
+        (left ? "Left" : "Right"),
+        TUtil.arrayToString(lineage),
+        FileUtil.humanReadableByteCount(volume, false),
+        (inMemoryInnerJoinFlag ? "" : "not ")));
+    return inMemoryInnerJoinFlag;
+  }
+
+  public PhysicalExec createJoinPlan(TaskAttemptContext context, JoinNode joinNode, PhysicalExec leftExec,
+                                     PhysicalExec rightExec) throws IOException {
+
+    switch (joinNode.getJoinType()) {
+      case CROSS:
+        return createCrossJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case INNER:
+        return createInnerJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case LEFT_OUTER:
+        return createLeftOuterJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case RIGHT_OUTER:
+        return createRightOuterJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case FULL_OUTER:
+        return createFullOuterJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case LEFT_SEMI:
+        return createLeftSemiJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case RIGHT_SEMI:
+        return createRightSemiJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case LEFT_ANTI:
+        return createLeftAntiJoinPlan(context, joinNode, leftExec, rightExec);
+
+      case RIGHT_ANTI:
+        return createRightAntiJoinPlan(context, joinNode, leftExec, rightExec);
+
+      default:
+        throw new PhysicalPlanningException("Cannot support join type: " + joinNode.getJoinType().name());
+    }
+  }
+
+  private PhysicalExec createCrossJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                           PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+
+      switch (algorithm) {
+        case NESTED_LOOP_JOIN:
+          LOG.info("Join (" + plan.getPID() +") chooses [Nested Loop Join]");
+          return new NLJoinExec(context, plan, leftExec, rightExec);
+        case BLOCK_NESTED_LOOP_JOIN:
+          LOG.info("Join (" + plan.getPID() +") chooses [Block Nested Loop Join]");
+          return new BNLJoinExec(context, plan, leftExec, rightExec);
+        default:
+          // fallback algorithm
+          LOG.error("Invalid Cross Join Algorithm Enforcer: " + algorithm.name());
+          return new BNLJoinExec(context, plan, leftExec, rightExec);
+      }
+
+    } else {
+      return new BNLJoinExec(context, plan, leftExec, rightExec);
+    }
+  }
+
+  private PhysicalExec createInnerJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                           PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+
+      switch (algorithm) {
+        case NESTED_LOOP_JOIN:
+          LOG.info("Join (" + plan.getPID() +") chooses [Nested Loop Join]");
+          return new NLJoinExec(context, plan, leftExec, rightExec);
+        case BLOCK_NESTED_LOOP_JOIN:
+          LOG.info("Join (" + plan.getPID() +") chooses [Block Nested Loop Join]");
+          return new BNLJoinExec(context, plan, leftExec, rightExec);
+        case IN_MEMORY_HASH_JOIN:
+          LOG.info("Join (" + plan.getPID() +") chooses [In-memory Hash Join]");
+          // returns two PhysicalExec. smaller one is 0, and larger one is 1.
+          PhysicalExec [] orderedChilds = switchJoinSidesIfNecessary(context, plan, leftExec, rightExec);
+          return new HashJoinExec(context, plan, orderedChilds[1], orderedChilds[0]);
+        case MERGE_JOIN:
+          LOG.info("Join (" + plan.getPID() +") chooses [Sort Merge Join]");
+          return createMergeInnerJoin(context, plan, leftExec, rightExec);
+        case HYBRID_HASH_JOIN:
+
+        default:
+          LOG.error("Invalid Inner Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback inner join algorithm: " + JoinAlgorithm.MERGE_JOIN.name());
+          return createMergeInnerJoin(context, plan, leftExec, rightExec);
+      }
+    } else {
+      return createBestInnerJoinPlan(context, plan, leftExec, rightExec);
+    }
+  }
+
+  /**
+   * It returns two {@link org.apache.tajo.engine.planner.physical.PhysicalExec}s sorted in an ascending order of
+   * their child relations' total volume. In other words, the smaller side is returned as 0's PhysicalExec, and
+   * the larger side is returned as 1's PhysicalExec.
+   */
+  @VisibleForTesting
+  public PhysicalExec [] switchJoinSidesIfNecessary(TaskAttemptContext context, JoinNode plan,
+                                                     PhysicalExec left, PhysicalExec right) throws IOException {
+    String [] leftLineage = PlannerUtil.getRelationLineage(plan.getLeftChild());
+    String [] rightLineage = PlannerUtil.getRelationLineage(plan.getRightChild());
+    long leftSize = estimateSizeRecursive(context, leftLineage);
+    long rightSize = estimateSizeRecursive(context, rightLineage);
+
+    PhysicalExec smaller;
+    PhysicalExec larger;
+    if (leftSize <= rightSize) {
+      smaller = left;
+      larger = right;
+      LOG.info(String.format("[%s] Left relations %s (%s) is smaller than Right relations %s (%s).",
+          context.getTaskId().toString(),
+          TUtil.arrayToString(leftLineage),
+          FileUtil.humanReadableByteCount(leftSize, false),
+          TUtil.arrayToString(rightLineage),
+          FileUtil.humanReadableByteCount(rightSize, false)));
+    } else {
+      smaller = right;
+      larger = left;
+      LOG.info(String.format("[%s] Right relations %s (%s) is smaller than Left relations %s (%s).",
+          context.getTaskId().toString(),
+          TUtil.arrayToString(rightLineage),
+          FileUtil.humanReadableByteCount(rightSize, false),
+          TUtil.arrayToString(leftLineage),
+          FileUtil.humanReadableByteCount(leftSize, false)));
+    }
+
+    return new PhysicalExec [] {smaller, larger};
+  }
+
+  private PhysicalExec createBestInnerJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                               PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    boolean inMemoryHashJoin = false;
+    if (checkIfInMemoryInnerJoinIsPossible(context, plan.getLeftChild(), true)
+        || checkIfInMemoryInnerJoinIsPossible(context, plan.getRightChild(), false)) {
+      inMemoryHashJoin = true;
+    }
+
+    if (inMemoryHashJoin) {
+      LOG.info("Join (" + plan.getPID() +") chooses [In-memory Hash Join]");
+      // returns two PhysicalExec. smaller one is 0, and larger one is 1.
+      PhysicalExec [] orderedChilds = switchJoinSidesIfNecessary(context, plan, leftExec, rightExec);
+      return new HashJoinExec(context, plan, orderedChilds[1], orderedChilds[0]);
+    } else {
+      return createMergeInnerJoin(context, plan, leftExec, rightExec);
+    }
+  }
+
+  private MergeJoinExec createMergeInnerJoin(TaskAttemptContext context, JoinNode plan,
+                                             PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    SortSpec[][] sortSpecs = PlannerUtil.getSortKeysFromJoinQual(
+        plan.getJoinQual(), leftExec.getSchema(), rightExec.getSchema());
+
+    SortNode leftSortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    leftSortNode.setSortSpecs(sortSpecs[0]);
+    leftSortNode.setInSchema(leftExec.getSchema());
+    leftSortNode.setOutSchema(leftExec.getSchema());
+    ExternalSortExec outerSort = new ExternalSortExec(context, sm, leftSortNode, leftExec);
+
+    SortNode rightSortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    rightSortNode.setSortSpecs(sortSpecs[1]);
+    rightSortNode.setInSchema(rightExec.getSchema());
+    rightSortNode.setOutSchema(rightExec.getSchema());
+    ExternalSortExec innerSort = new ExternalSortExec(context, sm, rightSortNode, rightExec);
+
+    LOG.info("Join (" + plan.getPID() +") chooses [Merge Join]");
+    return new MergeJoinExec(context, plan, outerSort, innerSort, sortSpecs[0], sortSpecs[1]);
+  }
+
+  private PhysicalExec createLeftOuterJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                               PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+      switch (algorithm) {
+        case IN_MEMORY_HASH_JOIN:
+          LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Hash Join].");
+          return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec);
+        case NESTED_LOOP_JOIN:
+          //the right operand is too large, so we opt for NL implementation of left outer join
+          LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Nested Loop Join].");
+          return new NLLeftOuterJoinExec(context, plan, leftExec, rightExec);
+        default:
+          LOG.error("Invalid Left Outer Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback inner join algorithm: " + JoinAlgorithm.IN_MEMORY_HASH_JOIN.name());
+          return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec);
+      }
+    } else {
+      return createBestLeftOuterJoinPlan(context, plan, leftExec, rightExec);
+    }
+  }
+
+  private PhysicalExec createBestLeftOuterJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                                   PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    String [] rightLineage = PlannerUtil.getRelationLineage(plan.getRightChild());
+    long rightTableVolume = estimateSizeRecursive(context, rightLineage);
+
+    if (rightTableVolume < conf.getLongVar(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD)) {
+      // we can implement left outer join using hash join, using the right operand as the build relation
+      LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Hash Join].");
+      return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec);
+    }
+    else {
+      //the right operand is too large, so we opt for NL implementation of left outer join
+      LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Nested Loop Join].");
+      return new NLLeftOuterJoinExec(context, plan, leftExec, rightExec);
+    }
+  }
+
+  private PhysicalExec createBestRightJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                               PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    //if the left operand is small enough => implement it as a left outer hash join with exchanged operators (note:
+    // blocking, but merge join is blocking as well)
+    String [] outerLineage4 = PlannerUtil.getRelationLineage(plan.getLeftChild());
+    long outerSize = estimateSizeRecursive(context, outerLineage4);
+    if (outerSize < conf.getLongVar(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD)){
+      LOG.info("Right Outer Join (" + plan.getPID() +") chooses [Hash Join].");
+      return new HashLeftOuterJoinExec(context, plan, rightExec, leftExec);
+    } else {
+      return createRightOuterMergeJoinPlan(context, plan, leftExec, rightExec);
+    }
+  }
+
+  private PhysicalExec createRightOuterMergeJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                                     PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    //the left operand is too large, so opt for merge join implementation
+    LOG.info("Right Outer Join (" + plan.getPID() +") chooses [Merge Join].");
+    SortSpec[][] sortSpecs2 = PlannerUtil.getSortKeysFromJoinQual(
+        plan.getJoinQual(), leftExec.getSchema(), rightExec.getSchema());
+
+    SortNode leftSortNode2 = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    leftSortNode2.setSortSpecs(sortSpecs2[0]);
+    leftSortNode2.setInSchema(leftExec.getSchema());
+    leftSortNode2.setOutSchema(leftExec.getSchema());
+    ExternalSortExec outerSort2 = new ExternalSortExec(context, sm, leftSortNode2, leftExec);
+
+    SortNode rightSortNode2 = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    rightSortNode2.setSortSpecs(sortSpecs2[1]);
+    rightSortNode2.setInSchema(rightExec.getSchema());
+    rightSortNode2.setOutSchema(rightExec.getSchema());
+    ExternalSortExec innerSort2 = new ExternalSortExec(context, sm, rightSortNode2, rightExec);
+
+    return new RightOuterMergeJoinExec(context, plan, outerSort2, innerSort2, sortSpecs2[0], sortSpecs2[1]);
+  }
+
+  private PhysicalExec createRightOuterJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                                PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+      switch (algorithm) {
+        case IN_MEMORY_HASH_JOIN:
+          LOG.info("Right Outer Join (" + plan.getPID() +") chooses [Hash Join].");
+          return new HashLeftOuterJoinExec(context, plan, rightExec, leftExec);
+        case MERGE_JOIN:
+          return createRightOuterMergeJoinPlan(context, plan, leftExec, rightExec);
+        default:
+          LOG.error("Invalid Right Outer Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback merge join algorithm: " + JoinAlgorithm.MERGE_JOIN.name());
+          return createRightOuterMergeJoinPlan(context, plan, leftExec, rightExec);
+      }
+    } else {
+      return createBestRightJoinPlan(context, plan, leftExec, rightExec);
+    }
+  }
+
+  private PhysicalExec createFullOuterJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                               PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+      switch (algorithm) {
+        case IN_MEMORY_HASH_JOIN:
+          return createFullOuterHashJoinPlan(context, plan, leftExec, rightExec);
+
+        case MERGE_JOIN:
+          return createFullOuterMergeJoinPlan(context, plan, leftExec, rightExec);
+
+        default:
+          LOG.error("Invalid Full Outer Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback merge join algorithm: " + JoinAlgorithm.MERGE_JOIN.name());
+          return createFullOuterMergeJoinPlan(context, plan, leftExec, rightExec);
+      }
+    } else {
+      return createBestFullOuterJoinPlan(context, plan, leftExec, rightExec);
+    }
+  }
+
+  private HashFullOuterJoinExec createFullOuterHashJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                                            PhysicalExec leftExec, PhysicalExec rightExec)
+      throws IOException {
+    String [] leftLineage = PlannerUtil.getRelationLineage(plan.getLeftChild());
+    String [] rightLineage = PlannerUtil.getRelationLineage(plan.getRightChild());
+    long outerSize2 = estimateSizeRecursive(context, leftLineage);
+    long innerSize2 = estimateSizeRecursive(context, rightLineage);
+
+    PhysicalExec selectedRight;
+    PhysicalExec selectedLeft;
+
+    // HashJoinExec loads the smaller relation to memory.
+    if (outerSize2 <= innerSize2) {
+      selectedLeft = leftExec;
+      selectedRight = rightExec;
+    } else {
+      selectedLeft = rightExec;
+      selectedRight = leftExec;
+    }
+    LOG.info("Full Outer Join (" + plan.getPID() +") chooses [Hash Join]");
+    return new HashFullOuterJoinExec(context, plan, selectedRight, selectedLeft);
+  }
+
+  private MergeFullOuterJoinExec createFullOuterMergeJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                                              PhysicalExec leftExec, PhysicalExec rightExec)
+      throws IOException {
+    // if size too large, full outer merge join implementation
+    LOG.info("Full Outer Join (" + plan.getPID() +") chooses [Merge Join]");
+    SortSpec[][] sortSpecs3 = PlannerUtil.getSortKeysFromJoinQual(plan.getJoinQual(),
+        leftExec.getSchema(), rightExec.getSchema());
+
+    SortNode leftSortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    leftSortNode.setSortSpecs(sortSpecs3[0]);
+    leftSortNode.setInSchema(leftExec.getSchema());
+    leftSortNode.setOutSchema(leftExec.getSchema());
+    ExternalSortExec outerSort3 = new ExternalSortExec(context, sm, leftSortNode, leftExec);
+
+    SortNode rightSortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    rightSortNode.setSortSpecs(sortSpecs3[1]);
+    rightSortNode.setInSchema(rightExec.getSchema());
+    rightSortNode.setOutSchema(rightExec.getSchema());
+    ExternalSortExec innerSort3 = new ExternalSortExec(context, sm, rightSortNode, rightExec);
+
+    return new MergeFullOuterJoinExec(context, plan, outerSort3, innerSort3, sortSpecs3[0], sortSpecs3[1]);
+  }
+
+  private PhysicalExec createBestFullOuterJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                                   PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    String [] leftLineage = PlannerUtil.getRelationLineage(plan.getLeftChild());
+    String [] rightLineage = PlannerUtil.getRelationLineage(plan.getRightChild());
+    long outerSize2 = estimateSizeRecursive(context, leftLineage);
+    long innerSize2 = estimateSizeRecursive(context, rightLineage);
+    final long threshold = 1048576 * 128;
+    if (outerSize2 < threshold || innerSize2 < threshold) {
+      return createFullOuterHashJoinPlan(context, plan, leftExec, rightExec);
+    } else {
+      return createFullOuterMergeJoinPlan(context, plan, leftExec, rightExec);
+    }
+  }
+
+  /**
+   *  Left semi join means that the left side is the IN side table, and the right side is the FROM side table.
+   */
+  private PhysicalExec createLeftSemiJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                              PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+      switch (algorithm) {
+        case IN_MEMORY_HASH_JOIN:
+          LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+          return new HashLeftSemiJoinExec(context, plan, leftExec, rightExec);
+
+        default:
+          LOG.error("Invalid Left Semi Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback inner join algorithm: " + JoinAlgorithm.IN_MEMORY_HASH_JOIN.name());
+          return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec);
+      }
+    } else {
+      LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+      return new HashLeftSemiJoinExec(context, plan, leftExec, rightExec);
+    }
+  }
+
+  /**
+   *  Left semi join means that the left side is the FROM side table, and the right side is the IN side table.
+   */
+  private PhysicalExec createRightSemiJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                               PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+      switch (algorithm) {
+        case IN_MEMORY_HASH_JOIN:
+          LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+          return new HashLeftSemiJoinExec(context, plan, rightExec, leftExec);
+
+        default:
+          LOG.error("Invalid Left Semi Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback inner join algorithm: " + JoinAlgorithm.IN_MEMORY_HASH_JOIN.name());
+          return new HashLeftOuterJoinExec(context, plan, rightExec, leftExec);
+      }
+    } else {
+      LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+      return new HashLeftSemiJoinExec(context, plan, rightExec, leftExec);
+    }
+  }
+
+  /**
+   *  Left semi join means that the left side is the FROM side table, and the right side is the IN side table.
+   */
+  private PhysicalExec createLeftAntiJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                              PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+      switch (algorithm) {
+        case IN_MEMORY_HASH_JOIN:
+          LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+          return new HashLeftAntiJoinExec(context, plan, leftExec, rightExec);
+
+        default:
+          LOG.error("Invalid Left Semi Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback inner join algorithm: " + JoinAlgorithm.IN_MEMORY_HASH_JOIN.name());
+          return new HashLeftAntiJoinExec(context, plan, leftExec, rightExec);
+      }
+    } else {
+      LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+      return new HashLeftAntiJoinExec(context, plan, leftExec, rightExec);
+    }
+  }
+
+  /**
+   *  Left semi join means that the left side is the FROM side table, and the right side is the IN side table.
+   */
+  private PhysicalExec createRightAntiJoinPlan(TaskAttemptContext context, JoinNode plan,
+                                               PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan);
+    if (property != null) {
+      JoinAlgorithm algorithm = property.getJoin().getAlgorithm();
+      switch (algorithm) {
+        case IN_MEMORY_HASH_JOIN:
+          LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+          return new HashLeftSemiJoinExec(context, plan, rightExec, leftExec);
+
+        default:
+          LOG.error("Invalid Left Semi Join Algorithm Enforcer: " + algorithm.name());
+          LOG.error("Choose a fallback inner join algorithm: " + JoinAlgorithm.IN_MEMORY_HASH_JOIN.name());
+          return new HashLeftOuterJoinExec(context, plan, rightExec, leftExec);
+      }
+    } else {
+      LOG.info("Left Semi Join (" + plan.getPID() +") chooses [In Memory Hash Join].");
+      return new HashLeftSemiJoinExec(context, plan, rightExec, leftExec);
+    }
+  }
+
+
+  /**
+   * Create a shuffle file write executor to store intermediate data into local disks.
+   */
+  public PhysicalExec createShuffleFileWritePlan(TaskAttemptContext ctx,
+                                                 ShuffleFileWriteNode plan, PhysicalExec subOp) throws IOException {
+    switch (plan.getShuffleType()) {
+    case HASH_SHUFFLE:
+      return new HashShuffleFileWriteExec(ctx, sm, plan, subOp);
+
+    case RANGE_SHUFFLE:
+      SortExec sortExec = PhysicalPlanUtil.findExecutor(subOp, SortExec.class);
+
+      SortSpec [] sortSpecs = null;
+      if (sortExec != null) {
+        sortSpecs = sortExec.getSortSpecs();
+      } else {
+        Column[] columns = ctx.getDataChannel().getShuffleKeys();
+        SortSpec specs[] = new SortSpec[columns.length];
+        for (int i = 0; i < columns.length; i++) {
+          specs[i] = new SortSpec(columns[i]);
+        }
+      }
+      return new RangeShuffleFileWriteExec(ctx, sm, subOp, plan.getInSchema(), plan.getInSchema(), sortSpecs);
+
+    case NONE_SHUFFLE:
+      return new StoreTableExec(ctx, plan, subOp);
+
+    default:
+      throw new IllegalStateException(ctx.getDataChannel().getShuffleType() + " is not supported yet.");
+    }
+  }
+
+  /**
+   * Create a executor to store a table into HDFS. This is used for CREATE TABLE ..
+   * AS or INSERT (OVERWRITE) INTO statement.
+   */
+  public PhysicalExec createStorePlan(TaskAttemptContext ctx,
+                                      StoreTableNode plan, PhysicalExec subOp) throws IOException {
+
+    if (plan.getPartitionMethod() != null) {
+      switch (plan.getPartitionMethod().getPartitionType()) {
+      case COLUMN:
+        return createColumnPartitionStorePlan(ctx, plan, subOp);
+      default:
+        throw new IllegalStateException(plan.getPartitionMethod().getPartitionType() + " is not supported yet.");
+      }
+    } else {
+      return new StoreTableExec(ctx, plan, subOp);
+    }
+  }
+
+  private PhysicalExec createColumnPartitionStorePlan(TaskAttemptContext context,
+                                                      StoreTableNode storeTableNode,
+                                                      PhysicalExec child) throws IOException {
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, storeTableNode);
+    if (property != null) {
+      ColumnPartitionAlgorithm algorithm = property.getColumnPartition().getAlgorithm();
+      switch (algorithm) {
+      case HASH_PARTITION:
+        return createHashColumnPartitionStorePlan(context, storeTableNode, child);
+      case SORT_PARTITION: // default algorithm
+      default:
+        return createSortBasedColumnPartitionStorePlan(context, storeTableNode, child);
+      }
+    } else { // default algorithm is sorted-based column partition
+      return createSortBasedColumnPartitionStorePlan(context, storeTableNode, child);
+    }
+  }
+
+  private PhysicalExec createHashColumnPartitionStorePlan(TaskAttemptContext context,
+                                                          StoreTableNode storeTableNode,
+                                                          PhysicalExec child) throws IOException {
+    LOG.info("The planner chooses [Hash-based Column Partitioned Store] algorithm");
+    return new HashBasedColPartitionStoreExec(context, storeTableNode, child);
+  }
+
+  private PhysicalExec createSortBasedColumnPartitionStorePlan(TaskAttemptContext context,
+                                                               StoreTableNode storeTableNode,
+                                                               PhysicalExec child) throws IOException {
+
+    Column[] partitionKeyColumns = storeTableNode.getPartitionMethod().getExpressionSchema().toArray();
+    SortSpec[] sortSpecs = new SortSpec[partitionKeyColumns.length];
+
+    if (storeTableNode.getType() == NodeType.INSERT) {
+      InsertNode insertNode = (InsertNode) storeTableNode;
+      for (int i = 0; i < partitionKeyColumns.length; i++) {
+        for (Column column : partitionKeyColumns) {
+          int id = insertNode.getTableSchema().getColumnId(column.getQualifiedName());
+          sortSpecs[i++] = new SortSpec(insertNode.getProjectedSchema().getColumn(id), true, false);
+        }
+      }
+    } else {
+      for (int i = 0; i < partitionKeyColumns.length; i++) {
+        sortSpecs[i] = new SortSpec(partitionKeyColumns[i], true, false);
+      }
+    }
+
+    SortNode sortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    sortNode.setSortSpecs(sortSpecs);
+    sortNode.setInSchema(child.getSchema());
+    sortNode.setOutSchema(child.getSchema());
+
+    ExternalSortExec sortExec = new ExternalSortExec(context, sm, sortNode, child);
+    LOG.info("The planner chooses [Sort-based Column Partitioned Store] algorithm");
+    return new SortBasedColPartitionStoreExec(context, storeTableNode, sortExec);
+  }
+
+  private boolean checkIfSortEquivalance(TaskAttemptContext ctx, ScanNode scanNode, Stack<LogicalNode> node) {
+    Enforcer enforcer = ctx.getEnforcer();
+    List<EnforceProperty> property = enforcer.getEnforceProperties(EnforceType.SORTED_INPUT);
+    if (property != null && property.size() > 0 && node.peek().getType() == NodeType.SORT) {
+      SortNode sortNode = (SortNode) node.peek();
+      TajoWorkerProtocol.SortedInputEnforce sortEnforcer = property.get(0).getSortedInput();
+
+      boolean condition = scanNode.getTableName().equals(sortEnforcer.getTableName());
+      SortSpec [] sortSpecs = PlannerUtil.convertSortSpecs(sortEnforcer.getSortSpecsList());
+      return condition && TUtil.checkEquals(sortNode.getSortKeys(), sortSpecs);
+    } else {
+      return false;
+    }
+  }
+
+  public PhysicalExec createScanPlan(TaskAttemptContext ctx, ScanNode scanNode, Stack<LogicalNode> node)
+      throws IOException {
+    if (ctx.getTable(scanNode.getCanonicalName()) == null) {
+      return new SeqScanExec(ctx, sm, scanNode, null);
+    }
+    Preconditions.checkNotNull(ctx.getTable(scanNode.getCanonicalName()),
+        "Error: There is no table matched to %s", scanNode.getCanonicalName() + "(" + scanNode.getTableName() + ")");    
+
+    // check if an input is sorted in the same order to the subsequence sort operator.
+    // TODO - it works only if input files are raw files. We should check the file format.
+    // Since the default intermediate file format is raw file, it is not problem right now.
+    if (checkIfSortEquivalance(ctx, scanNode, node)) {
+      FragmentProto [] fragments = ctx.getTables(scanNode.getCanonicalName());
+      return new ExternalSortExec(ctx, sm, (SortNode) node.peek(), fragments);
+    } else {
+      Enforcer enforcer = ctx.getEnforcer();
+
+      // check if this table is broadcasted one or not.
+      boolean broadcastFlag = false;
+      if (enforcer != null && enforcer.hasEnforceProperty(EnforceType.BROADCAST)) {
+        List<EnforceProperty> properties = enforcer.getEnforceProperties(EnforceType.BROADCAST);
+        for (EnforceProperty property : properties) {
+          broadcastFlag |= scanNode.getCanonicalName().equals(property.getBroadcast().getTableName());
+        }
+      }
+
+      if (scanNode instanceof PartitionedTableScanNode
+          && ((PartitionedTableScanNode)scanNode).getInputPaths() != null &&
+          ((PartitionedTableScanNode)scanNode).getInputPaths().length > 0) {
+
+        if (scanNode instanceof PartitionedTableScanNode) {
+          if (broadcastFlag) {
+            PartitionedTableScanNode partitionedTableScanNode = (PartitionedTableScanNode) scanNode;
+            List<FileFragment> fileFragments = TUtil.newList();
+            for (Path path : partitionedTableScanNode.getInputPaths()) {
+              fileFragments.addAll(TUtil.newList(sm.split(scanNode.getCanonicalName(), path)));
+            }
+
+            return new PartitionMergeScanExec(ctx, sm, scanNode,
+                FragmentConvertor.toFragmentProtoArray(fileFragments.toArray(new FileFragment[fileFragments.size()])));
+          }
+        }
+      }
+
+      FragmentProto [] fragments = ctx.getTables(scanNode.getCanonicalName());
+      return new SeqScanExec(ctx, sm, scanNode, fragments);
+    }
+  }
+
+  public PhysicalExec createGroupByPlan(TaskAttemptContext context,GroupbyNode groupbyNode, PhysicalExec subOp)
+      throws IOException {
+
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, groupbyNode);
+    if (property != null) {
+      GroupbyAlgorithm algorithm = property.getGroupby().getAlgorithm();
+      if (algorithm == GroupbyAlgorithm.HASH_AGGREGATION) {
+        return createInMemoryHashAggregation(context, groupbyNode, subOp);
+      } else {
+        return createSortAggregation(context, property, groupbyNode, subOp);
+      }
+    }
+    return createBestAggregationPlan(context, groupbyNode, subOp);
+  }
+
+  private PhysicalExec createInMemoryHashAggregation(TaskAttemptContext ctx,GroupbyNode groupbyNode, PhysicalExec subOp)
+      throws IOException {
+    LOG.info("The planner chooses [Hash Aggregation]");
+    return new HashAggregateExec(ctx, groupbyNode, subOp);
+  }
+
+  private PhysicalExec createSortAggregation(TaskAttemptContext ctx, EnforceProperty property, GroupbyNode groupbyNode,
+                                             PhysicalExec subOp) throws IOException {
+
+    Column[] grpColumns = groupbyNode.getGroupingColumns();
+    SortSpec[] sortSpecs = new SortSpec[grpColumns.length];
+    for (int i = 0; i < grpColumns.length; i++) {
+      sortSpecs[i] = new SortSpec(grpColumns[i], true, false);
+    }
+
+    if (property != null) {
+      List<CatalogProtos.SortSpecProto> sortSpecProtos = property.getGroupby().getSortSpecsList();
+
+      List<SortSpec> enforcedSortSpecList = Lists.newArrayList();
+      int i = 0;
+      outer:
+      for (int j = 0; j < sortSpecProtos.size(); j++) {
+        SortSpec enforcedSortSpecs = new SortSpec(sortSpecProtos.get(j));
+
+        for (Column grpKey : grpColumns) { // if this sort key is included in grouping columns, skip it.
+          if (enforcedSortSpecs.getSortKey().equals(grpKey)) {
+            continue outer;
+          }
+        }
+
+        enforcedSortSpecList.add(enforcedSortSpecs);
+      }
+
+      sortSpecs = ObjectArrays.concat(sortSpecs, TUtil.toArray(enforcedSortSpecList, SortSpec.class), SortSpec.class);
+    }
+
+    SortNode sortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
+    sortNode.setSortSpecs(sortSpecs);
+    sortNode.setInSchema(subOp.getSchema());
+    sortNode.setOutSchema(subOp.getSchema());
+    ExternalSortExec sortExec = new ExternalSortExec(ctx, sm, sortNode, subOp);
+    LOG.info("The planner chooses [Sort Aggregation] in (" + TUtil.arrayToString(sortSpecs) + ")");
+    return new SortAggregateExec(ctx, groupbyNode, sortExec);
+  }
+
+  private PhysicalExec createBestAggregationPlan(TaskAttemptContext context, GroupbyNode groupbyNode,
+                                                 PhysicalExec subOp) throws IOException {
+    Column[] grpColumns = groupbyNode.getGroupingColumns();
+    if (grpColumns.length == 0) {
+      return createInMemoryHashAggregation(context, groupbyNode, subOp);
+    }
+
+    String [] outerLineage = PlannerUtil.getRelationLineage(groupbyNode.getChild());
+    long estimatedSize = estimateSizeRecursive(context, outerLineage);
+    final long threshold = conf.getLongVar(ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD);
+
+    // if the relation size is less than the threshold,
+    // the hash aggregation will be used.
+    if (estimatedSize <= threshold) {
+      LOG.info("The planner chooses [Hash Aggregation]");
+      return createInMemoryHashAggregation(context, groupbyNode, subOp);
+    } else {
+      return createSortAggregation(context, null, groupbyNode, subOp);
+    }
+  }
+
+  public PhysicalExec createSortPlan(TaskAttemptContext context, SortNode sortNode,
+                                     PhysicalExec child) throws IOException {
+
+    // check if it is a distributed merge sort
+    // If so, it does need to create a sort executor because
+    // the sort executor is created at the scan planning
+    if (child instanceof SortExec) {
+      SortExec childSortExec = (SortExec) child;
+      if (TUtil.checkEquals(sortNode.getSortKeys(), childSortExec.getSortSpecs())) {
+        return child;
+      }
+    }
+
+    Enforcer enforcer = context.getEnforcer();
+    EnforceProperty property = getAlgorithmEnforceProperty(enforcer, sortNode);
+    if (property != null) {
+      SortEnforce.SortAlgorithm algorithm = property.getSort().getAlgorithm();
+      if (algorithm == SortEnforce.SortAlgorithm.IN_MEMORY_SORT) {
+        return new MemSortExec(context, sortNode, child);
+      } else {
+        return new ExternalSortExec(context, sm, sortNode, child);
+      }
+    }
+
+    return createBestSortPlan(context, sortNode, child);
+  }
+
+  public SortExec createBestSortPlan(TaskAttemptContext context, SortNode sortNode,
+                                     PhysicalExec child) throws IOException {
+    return new ExternalSortExec(context, sm, sortNode, child);
+  }
+
+  public PhysicalExec createIndexScanExec(TaskAttemptContext ctx,
+                                          IndexScanNode annotation)
+      throws IOException {
+    //TODO-general Type Index
+    Preconditions.checkNotNull(ctx.getTable(annotation.getCanonicalName()),
+        "Error: There is no table matched to %s", annotation.getCanonicalName());
+
+    FragmentProto [] fragmentProtos = ctx.getTables(annotation.getTableName());
+    List<FileFragment> fragments =
+        FragmentConvertor.convert(ctx.getConf(), ctx.getDataChannel().getStoreType(), fragmentProtos);
+
+    String indexName = IndexUtil.getIndexNameOfFrag(fragments.get(0), annotation.getSortKeys());
+    Path indexPath = new Path(sm.getTablePath(annotation.getTableName()), "index");
+
+    TupleComparator comp = new TupleComparator(annotation.getKeySchema(),
+        annotation.getSortKeys());
+    return new BSTIndexScanExec(ctx, sm, annotation, fragments.get(0), new Path(indexPath, indexName),
+        annotation.getKeySchema(), comp, annotation.getDatum());
+
+  }
+
+  private EnforceProperty getAlgorithmEnforceProperty(Enforcer enforcer, LogicalNode node) {
+    if (enforcer == null) {
+      return null;
+    }
+
+    EnforceType type;
+    if (node.getType() == NodeType.JOIN) {
+      type = EnforceType.JOIN;
+    } else if (node.getType() == NodeType.GROUP_BY) {
+      type = EnforceType.GROUP_BY;
+    } else if (node.getType() == NodeType.SORT) {
+      type = EnforceType.SORT;
+    } else if (node instanceof StoreTableNode
+        && ((StoreTableNode)node).hasPartition()
+        && ((StoreTableNode)node).getPartitionMethod().getPartitionType() == PartitionType.COLUMN) {
+      type = EnforceType.COLUMN_PARTITION;
+    } else {
+      return null;
+    }
+
+    if (enforcer.hasEnforceProperty(type)) {
+      List<EnforceProperty> properties = enforcer.getEnforceProperties(type);
+      EnforceProperty found = null;
+      for (EnforceProperty property : properties) {
+        if (type == EnforceType.JOIN && property.getJoin().getPid() == node.getPID()) {
+          found = property;
+        } else if (type == EnforceType.GROUP_BY && property.getGroupby().getPid() == node.getPID()) {
+          found = property;
+        } else if (type == EnforceType.SORT && property.getSort().getPid() == node.getPID()) {
+          found = property;
+        } else if (type == EnforceType.COLUMN_PARTITION && property.getColumnPartition().getPid() == node.getPID()) {
+          found = property;
+        }
+      }
+      return found;
+    } else {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanningException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanningException.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanningException.java
new file mode 100644
index 0000000..1b0a7c3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanningException.java
@@ -0,0 +1,31 @@
+/**
+ * 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.planner;
+
+import java.io.IOException;
+
+public class PhysicalPlanningException extends IOException {
+  public PhysicalPlanningException(String message) {
+    super(message);
+  }
+
+  public PhysicalPlanningException(Throwable t) {
+    super(t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanString.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanString.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanString.java
new file mode 100644
index 0000000..98f921b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanString.java
@@ -0,0 +1,119 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+public class PlanString {
+  final StringBuilder title;
+
+  final List<String> explanations = new ArrayList<String>();
+  final List<String> details = new ArrayList<String>();
+
+  StringBuilder currentExplanation;
+  StringBuilder currentDetail;
+
+  public PlanString(LogicalNode node) {
+    this.title = new StringBuilder(node.getType().name() + "(" + node.getPID() + ")");
+  }
+
+  public PlanString(String title) {
+    this.title = new StringBuilder(title);
+  }
+
+  public PlanString appendTitle(String str) {
+    title.append(str);
+    return this;
+  }
+
+  public PlanString addExplan(String explain) {
+    flushCurrentExplanation();
+    currentExplanation = new StringBuilder(explain);
+    return this;
+  }
+
+  public PlanString appendExplain(String explain) {
+    if (currentExplanation == null) {
+      currentExplanation = new StringBuilder();
+    }
+    currentExplanation.append(explain);
+    return this;
+  }
+
+  public PlanString addDetail(String detail) {
+    flushCurrentDetail();
+    currentDetail = new StringBuilder(detail);
+    return this;
+  }
+
+  public PlanString appendDetail(String detail) {
+    if (currentDetail == null) {
+      currentDetail = new StringBuilder();
+    }
+    currentDetail.append(detail);
+    return this;
+
+  }
+
+  public String getTitle() {
+    return title.toString();
+  }
+
+  public List<String> getExplanations() {
+    flushCurrentExplanation();
+    return explanations;
+  }
+
+  public List<String> getDetails() {
+    flushCurrentDetail();
+    return details;
+  }
+
+  private void flushCurrentExplanation() {
+    if (currentExplanation != null) {
+      explanations.add(currentExplanation.toString());
+      currentExplanation = null;
+    }
+  }
+
+  private void flushCurrentDetail() {
+    if (currentDetail != null) {
+      details.add(currentDetail.toString());
+      currentDetail = null;
+    }
+  }
+
+  public String toString() {
+    StringBuilder output = new StringBuilder();
+    output.append(getTitle()).append("\n");
+
+    for (String str : getExplanations()) {
+      output.append("  => ").append(str).append("\n");
+    }
+
+    for (String str : getDetails()) {
+      output.append("  => ").append(str).append("\n");
+    }
+    return output.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java
new file mode 100644
index 0000000..9f988bd
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java
@@ -0,0 +1,762 @@
+/**
+ * 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.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.exception.InvalidQueryException;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+public class PlannerUtil {
+
+  public static boolean checkIfDDLPlan(LogicalNode node) {
+    LogicalNode baseNode = node;
+    if (node instanceof LogicalRootNode) {
+      baseNode = ((LogicalRootNode) node).getChild();
+    }
+
+    NodeType type = baseNode.getType();
+
+    return
+        type == NodeType.CREATE_DATABASE ||
+            type == NodeType.DROP_DATABASE ||
+            (type == NodeType.CREATE_TABLE && !((CreateTableNode) baseNode).hasSubQuery()) ||
+            baseNode.getType() == NodeType.DROP_TABLE ||
+            baseNode.getType() == NodeType.ALTER_TABLESPACE ||
+            baseNode.getType() == NodeType.ALTER_TABLE;
+  }
+
+  /**
+   * Checks whether the query is simple or not.
+   * The simple query can be defined as 'select * from tb_name [LIMIT X]'.
+   *
+   * @param plan The logical plan
+   * @return True if the query is a simple query.
+   */
+  public static boolean checkIfSimpleQuery(LogicalPlan plan) {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+
+    // one block, without where clause, no group-by, no-sort, no-join
+    boolean isOneQueryBlock = plan.getQueryBlocks().size() == 1;
+    boolean simpleOperator = rootNode.getChild().getType() == NodeType.LIMIT
+        || rootNode.getChild().getType() == NodeType.SCAN;
+    boolean noOrderBy = !plan.getRootBlock().hasNode(NodeType.SORT);
+    boolean noGroupBy = !plan.getRootBlock().hasNode(NodeType.GROUP_BY);
+    boolean noWhere = !plan.getRootBlock().hasNode(NodeType.SELECTION);
+    boolean noJoin = !plan.getRootBlock().hasNode(NodeType.JOIN);
+    boolean singleRelation = plan.getRootBlock().hasNode(NodeType.SCAN)
+        && PlannerUtil.getRelationLineage(plan.getRootBlock().getRoot()).length == 1;
+
+    boolean noComplexComputation = false;
+    if (singleRelation) {
+      ScanNode scanNode = plan.getRootBlock().getNode(NodeType.SCAN);
+      if (!scanNode.getTableDesc().hasPartition() && scanNode.hasTargets()
+          && scanNode.getTargets().length == scanNode.getInSchema().size()) {
+        noComplexComputation = true;
+        for (int i = 0; i < scanNode.getTargets().length; i++) {
+          noComplexComputation = noComplexComputation && scanNode.getTargets()[i].getEvalTree().getType() == EvalType.FIELD;
+          if (noComplexComputation) {
+            noComplexComputation = noComplexComputation && scanNode.getTargets()[i].getNamedColumn().equals(scanNode.getInSchema().getColumn(i));
+          }
+          if (!noComplexComputation) {
+            return noComplexComputation;
+          }
+        }
+      }
+    }
+
+    return !checkIfDDLPlan(rootNode) &&
+        (simpleOperator && noComplexComputation  && isOneQueryBlock && noOrderBy && noGroupBy && noWhere && noJoin && singleRelation);
+  }
+
+  /**
+   * Checks whether the query has 'from clause' or not.
+   *
+   * @param plan The logical plan
+   * @return True if a query does not have 'from clause'.
+   */
+  public static boolean checkIfNonFromQuery(LogicalPlan plan) {
+    LogicalNode node = plan.getRootBlock().getRoot();
+
+    // one block, without where clause, no group-by, no-sort, no-join
+    boolean isOneQueryBlock = plan.getQueryBlocks().size() == 1;
+    boolean noRelation = !plan.getRootBlock().hasAlgebraicExpr(OpType.Relation);
+
+    return !checkIfDDLPlan(node) && noRelation && isOneQueryBlock;
+  }
+
+  /**
+   * Get all RelationNodes which are descendant of a given LogicalNode.
+   *
+   * @param from The LogicalNode to start visiting LogicalNodes.
+   * @return an array of all descendant RelationNode of LogicalNode.
+   */
+  public static String[] getRelationLineage(LogicalNode from) {
+    LogicalNode[] scans = findAllNodes(from, NodeType.SCAN, NodeType.PARTITIONS_SCAN);
+    String[] tableNames = new String[scans.length];
+    ScanNode scan;
+    for (int i = 0; i < scans.length; i++) {
+      scan = (ScanNode) scans[i];
+      tableNames[i] = scan.getCanonicalName();
+    }
+    return tableNames;
+  }
+
+  /**
+   * Get all RelationNodes which are descendant of a given LogicalNode.
+   * The finding is restricted within a query block.
+   *
+   * @param from The LogicalNode to start visiting LogicalNodes.
+   * @return an array of all descendant RelationNode of LogicalNode.
+   */
+  public static Collection<String> getRelationLineageWithinQueryBlock(LogicalPlan plan, LogicalNode from)
+      throws PlanningException {
+    RelationFinderVisitor visitor = new RelationFinderVisitor();
+    visitor.visit(null, plan, null, from, new Stack<LogicalNode>());
+    return visitor.getFoundRelations();
+  }
+
+  public static class RelationFinderVisitor extends BasicLogicalPlanVisitor<Object, LogicalNode> {
+    private Set<String> foundRelNameSet = Sets.newHashSet();
+
+    public Set<String> getFoundRelations() {
+      return foundRelNameSet;
+    }
+
+    @Override
+    public LogicalNode visit(Object context, LogicalPlan plan, @Nullable LogicalPlan.QueryBlock block, LogicalNode node,
+                             Stack<LogicalNode> stack) throws PlanningException {
+      if (node.getType() != NodeType.TABLE_SUBQUERY) {
+        super.visit(context, plan, block, node, stack);
+      }
+
+      if (node instanceof RelationNode) {
+        foundRelNameSet.add(((RelationNode) node).getCanonicalName());
+      }
+
+      return node;
+    }
+  }
+
+  /**
+   * Delete the logical node from a plan.
+   *
+   * @param parent      this node must be a parent node of one node to be removed.
+   * @param tobeRemoved this node must be a child node of the parent.
+   */
+  public static LogicalNode deleteNode(LogicalNode parent, LogicalNode tobeRemoved) {
+    Preconditions.checkArgument(tobeRemoved instanceof UnaryNode,
+        "ERROR: the logical node to be removed must be unary node.");
+
+    UnaryNode child = (UnaryNode) tobeRemoved;
+    LogicalNode grandChild = child.getChild();
+    if (parent instanceof UnaryNode) {
+      UnaryNode unaryParent = (UnaryNode) parent;
+
+      Preconditions.checkArgument(unaryParent.getChild() == child,
+          "ERROR: both logical node must be parent and child nodes");
+      unaryParent.setChild(grandChild);
+
+    } else if (parent instanceof BinaryNode) {
+      BinaryNode binaryParent = (BinaryNode) parent;
+      if (binaryParent.getLeftChild().deepEquals(child)) {
+        binaryParent.setLeftChild(grandChild);
+      } else if (binaryParent.getRightChild().deepEquals(child)) {
+        binaryParent.setRightChild(grandChild);
+      } else {
+        throw new IllegalStateException("ERROR: both logical node must be parent and child nodes");
+      }
+    } else {
+      throw new InvalidQueryException("Unexpected logical plan: " + parent);
+    }
+    return child;
+  }
+
+  public static void replaceNode(LogicalPlan plan, LogicalNode startNode, LogicalNode oldNode, LogicalNode newNode) {
+    LogicalNodeReplaceVisitor replacer = new LogicalNodeReplaceVisitor(oldNode, newNode);
+    try {
+      replacer.visit(new ReplacerContext(), plan, null, startNode, new Stack<LogicalNode>());
+    } catch (PlanningException e) {
+      e.printStackTrace();
+    }
+  }
+
+  static class ReplacerContext {
+    boolean updateSchemaFlag = false;
+  }
+
+  public static class LogicalNodeReplaceVisitor extends BasicLogicalPlanVisitor<ReplacerContext, LogicalNode> {
+    private LogicalNode target;
+    private LogicalNode tobeReplaced;
+
+    public LogicalNodeReplaceVisitor(LogicalNode target, LogicalNode tobeReplaced) {
+      this.target = target;
+      this.tobeReplaced = tobeReplaced;
+    }
+
+    /**
+     * If this node can have child, it returns TRUE. Otherwise, it returns FALSE.
+     */
+    private static boolean checkIfVisitable(LogicalNode node) {
+      return node instanceof UnaryNode || node instanceof BinaryNode;
+    }
+
+    @Override
+    public LogicalNode visit(ReplacerContext context, LogicalPlan plan, @Nullable LogicalPlan.QueryBlock block,
+                             LogicalNode node, Stack<LogicalNode> stack) throws PlanningException {
+      LogicalNode left = null;
+      LogicalNode right = null;
+
+      if (node instanceof UnaryNode) {
+        UnaryNode unaryNode = (UnaryNode) node;
+        if (unaryNode.getChild().deepEquals(target)) {
+          unaryNode.setChild(tobeReplaced);
+          left = tobeReplaced;
+          context.updateSchemaFlag = true;
+        } else if (checkIfVisitable(unaryNode.getChild())) {
+          left = visit(context, plan, null, unaryNode.getChild(), stack);
+        }
+      } else if (node instanceof BinaryNode) {
+        BinaryNode binaryNode = (BinaryNode) node;
+        if (binaryNode.getLeftChild().deepEquals(target)) {
+          binaryNode.setLeftChild(tobeReplaced);
+          left = tobeReplaced;
+          context.updateSchemaFlag = true;
+        } else if (checkIfVisitable(binaryNode.getLeftChild())) {
+          left = visit(context, plan, null, binaryNode.getLeftChild(), stack);
+        } else {
+          left = binaryNode.getLeftChild();
+        }
+
+        if (binaryNode.getRightChild().deepEquals(target)) {
+          binaryNode.setRightChild(tobeReplaced);
+          right = tobeReplaced;
+          context.updateSchemaFlag = true;
+        } else if (checkIfVisitable(binaryNode.getRightChild())) {
+          right = visit(context, plan, null, binaryNode.getRightChild(), stack);
+        } else {
+          right = binaryNode.getRightChild();
+        }
+      }
+
+      // update schemas of nodes except for leaf node (i.e., RelationNode)
+      if (context.updateSchemaFlag) {
+        if (node instanceof Projectable) {
+          if (node instanceof BinaryNode) {
+            node.setInSchema(SchemaUtil.merge(left.getOutSchema(), right.getOutSchema()));
+          } else {
+            node.setInSchema(left.getOutSchema());
+          }
+          context.updateSchemaFlag = false;
+        } else {
+          node.setInSchema(left.getOutSchema());
+          node.setOutSchema(left.getOutSchema());
+        }
+      }
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitScan(ReplacerContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode node,
+                                 Stack<LogicalNode> stack) throws PlanningException {
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitPartitionedTableScan(ReplacerContext context, LogicalPlan plan, LogicalPlan.
+        QueryBlock block, PartitionedTableScanNode node, Stack<LogicalNode> stack)
+
+        throws PlanningException {
+      return node;
+    }
+  }
+
+  public static void replaceNode(LogicalNode plan, LogicalNode newNode, NodeType type) {
+    LogicalNode parent = findTopParentNode(plan, type);
+    Preconditions.checkArgument(parent instanceof UnaryNode);
+    Preconditions.checkArgument(!(newNode instanceof BinaryNode));
+    UnaryNode parentNode = (UnaryNode) parent;
+    LogicalNode child = parentNode.getChild();
+    if (child instanceof UnaryNode) {
+      ((UnaryNode) newNode).setChild(((UnaryNode) child).getChild());
+    }
+    parentNode.setChild(newNode);
+  }
+
+  /**
+   * Find the top logical node matched to type from the given node
+   *
+   * @param node start node
+   * @param type to find
+   * @return a found logical node
+   */
+  public static <T extends LogicalNode> T findTopNode(LogicalNode node, NodeType type) {
+    Preconditions.checkNotNull(node);
+    Preconditions.checkNotNull(type);
+
+    LogicalNodeFinder finder = new LogicalNodeFinder(type);
+    node.preOrder(finder);
+
+    if (finder.getFoundNodes().size() == 0) {
+      return null;
+    }
+    return (T) finder.getFoundNodes().get(0);
+  }
+
+  /**
+   * Find the most bottom logical node matched to type from the given node
+   *
+   * @param node start node
+   * @param type to find
+   * @return a found logical node
+   */
+  public static <T extends LogicalNode> T findMostBottomNode(LogicalNode node, NodeType type) {
+    Preconditions.checkNotNull(node);
+    Preconditions.checkNotNull(type);
+
+    LogicalNodeFinder finder = new LogicalNodeFinder(type);
+    node.preOrder(finder);
+
+    if (finder.getFoundNodes().size() == 0) {
+      return null;
+    }
+    return (T) finder.getFoundNodes().get(finder.getFoundNodes().size() - 1);
+  }
+
+  /**
+   * Find the all logical node matched to type from the given node
+   *
+   * @param node start node
+   * @param type to find
+   * @return a found logical node
+   */
+  public static LogicalNode[] findAllNodes(LogicalNode node, NodeType... type) {
+    Preconditions.checkNotNull(node);
+    Preconditions.checkNotNull(type);
+
+    LogicalNodeFinder finder = new LogicalNodeFinder(type);
+    node.postOrder(finder);
+
+    if (finder.getFoundNodes().size() == 0) {
+      return new LogicalNode[]{};
+    }
+    List<LogicalNode> founds = finder.getFoundNodes();
+    return founds.toArray(new LogicalNode[founds.size()]);
+  }
+
+  /**
+   * Find a parent node of a given-typed operator.
+   *
+   * @param node start node
+   * @param type to find
+   * @return the parent node of a found logical node
+   */
+  public static <T extends LogicalNode> T findTopParentNode(LogicalNode node, NodeType type) {
+    Preconditions.checkNotNull(node);
+    Preconditions.checkNotNull(type);
+
+    ParentNodeFinder finder = new ParentNodeFinder(type);
+    node.postOrder(finder);
+
+    if (finder.getFoundNodes().size() == 0) {
+      return null;
+    }
+    return (T) finder.getFoundNodes().get(0);
+  }
+
+  private static class LogicalNodeFinder implements LogicalNodeVisitor {
+    private List<LogicalNode> list = new ArrayList<LogicalNode>();
+    private final NodeType[] tofind;
+    private boolean topmost = false;
+    private boolean finished = false;
+
+    public LogicalNodeFinder(NodeType... type) {
+      this.tofind = type;
+    }
+
+    public LogicalNodeFinder(NodeType[] type, boolean topmost) {
+      this(type);
+      this.topmost = topmost;
+    }
+
+    @Override
+    public void visit(LogicalNode node) {
+      if (!finished) {
+        for (NodeType type : tofind) {
+          if (node.getType() == type) {
+            list.add(node);
+          }
+          if (topmost && list.size() > 0) {
+            finished = true;
+          }
+        }
+      }
+    }
+
+    public List<LogicalNode> getFoundNodes() {
+      return list;
+    }
+
+    public LogicalNode[] getFoundNodeArray() {
+      return list.toArray(new LogicalNode[list.size()]);
+    }
+  }
+
+  private static class ParentNodeFinder implements LogicalNodeVisitor {
+    private List<LogicalNode> list = new ArrayList<LogicalNode>();
+    private NodeType tofind;
+
+    public ParentNodeFinder(NodeType type) {
+      this.tofind = type;
+    }
+
+    @Override
+    public void visit(LogicalNode node) {
+      if (node instanceof UnaryNode) {
+        UnaryNode unary = (UnaryNode) node;
+        if (unary.getChild().getType() == tofind) {
+          list.add(node);
+        }
+      } else if (node instanceof BinaryNode) {
+        BinaryNode bin = (BinaryNode) node;
+        if (bin.getLeftChild().getType() == tofind ||
+            bin.getRightChild().getType() == tofind) {
+          list.add(node);
+        }
+      }
+    }
+
+    public List<LogicalNode> getFoundNodes() {
+      return list;
+    }
+  }
+
+  /**
+   * fill targets with FieldEvals from a given schema
+   *
+   * @param schema  to be transformed to targets
+   * @param targets to be filled
+   */
+  public static void schemaToTargets(Schema schema, Target[] targets) {
+    FieldEval eval;
+    for (int i = 0; i < schema.size(); i++) {
+      eval = new FieldEval(schema.getColumn(i));
+      targets[i] = new Target(eval);
+    }
+  }
+
+  public static Target[] schemaToTargets(Schema schema) {
+    Target[] targets = new Target[schema.size()];
+
+    FieldEval eval;
+    for (int i = 0; i < schema.size(); i++) {
+      eval = new FieldEval(schema.getColumn(i));
+      targets[i] = new Target(eval);
+    }
+    return targets;
+  }
+
+  public static Target[] schemaToTargetsWithGeneratedFields(Schema schema) {
+    List<Target> targets = TUtil.newList();
+
+    FieldEval eval;
+    for (int i = 0; i < schema.size(); i++) {
+      eval = new FieldEval(schema.getColumn(i));
+      targets.add(new Target(eval));
+    }
+    return targets.toArray(new Target[targets.size()]);
+  }
+
+  public static SortSpec[] schemaToSortSpecs(Schema schema) {
+    return schemaToSortSpecs(schema.toArray());
+  }
+
+  public static SortSpec[] schemaToSortSpecs(Column[] columns) {
+    SortSpec[] specs = new SortSpec[columns.length];
+
+    for (int i = 0; i < columns.length; i++) {
+      specs[i] = new SortSpec(columns[i], true, false);
+    }
+
+    return specs;
+  }
+
+  public static SortSpec[] columnsToSortSpec(Collection<Column> columns) {
+    SortSpec[] specs = new SortSpec[columns.size()];
+    int i = 0;
+    for (Column column : columns) {
+      specs[i++] = new SortSpec(column, true, false);
+    }
+
+    return specs;
+  }
+
+  public static Schema sortSpecsToSchema(SortSpec[] sortSpecs) {
+    Schema schema = new Schema();
+    for (SortSpec spec : sortSpecs) {
+      schema.addColumn(spec.getSortKey());
+    }
+
+    return schema;
+  }
+
+  public static SortSpec[][] getSortKeysFromJoinQual(EvalNode joinQual, Schema outer, Schema inner) {
+    // It is used for the merge join executor. The merge join only considers the equi-join.
+    // So, theta-join flag must be false.
+    List<Column[]> joinKeyPairs = getJoinKeyPairs(joinQual, outer, inner, false);
+    SortSpec[] outerSortSpec = new SortSpec[joinKeyPairs.size()];
+    SortSpec[] innerSortSpec = new SortSpec[joinKeyPairs.size()];
+
+    for (int i = 0; i < joinKeyPairs.size(); i++) {
+      outerSortSpec[i] = new SortSpec(joinKeyPairs.get(i)[0]);
+      innerSortSpec[i] = new SortSpec(joinKeyPairs.get(i)[1]);
+    }
+
+    return new SortSpec[][]{outerSortSpec, innerSortSpec};
+  }
+
+  public static TupleComparator[] getComparatorsFromJoinQual(EvalNode joinQual, Schema leftSchema, Schema rightSchema) {
+    SortSpec[][] sortSpecs = getSortKeysFromJoinQual(joinQual, leftSchema, rightSchema);
+    TupleComparator[] comparators = new TupleComparator[2];
+    comparators[0] = new TupleComparator(leftSchema, sortSpecs[0]);
+    comparators[1] = new TupleComparator(rightSchema, sortSpecs[1]);
+    return comparators;
+  }
+
+  /**
+   * @return the first array contains left table's columns, and the second array contains right table's columns.
+   */
+  public static Column[][] joinJoinKeyForEachTable(EvalNode joinQual, Schema leftSchema, Schema rightSchema) {
+    List<Column[]> joinKeys = getJoinKeyPairs(joinQual, leftSchema, rightSchema, true);
+    Column[] leftColumns = new Column[joinKeys.size()];
+    Column[] rightColumns = new Column[joinKeys.size()];
+    for (int i = 0; i < joinKeys.size(); i++) {
+      leftColumns[i] = joinKeys.get(i)[0];
+      rightColumns[i] = joinKeys.get(i)[1];
+    }
+
+    return new Column[][]{leftColumns, rightColumns};
+  }
+
+  public static List<Column[]> getJoinKeyPairs(EvalNode joinQual, Schema leftSchema, Schema rightSchema,
+                                               boolean includeThetaJoin) {
+    JoinKeyPairFinder finder = new JoinKeyPairFinder(includeThetaJoin, leftSchema, rightSchema);
+    joinQual.preOrder(finder);
+    return finder.getPairs();
+  }
+
+  public static class JoinKeyPairFinder implements EvalNodeVisitor {
+    private boolean includeThetaJoin;
+    private final List<Column[]> pairs = Lists.newArrayList();
+    private Schema[] schemas = new Schema[2];
+
+    public JoinKeyPairFinder(boolean includeThetaJoin, Schema outer, Schema inner) {
+      this.includeThetaJoin = includeThetaJoin;
+      schemas[0] = outer;
+      schemas[1] = inner;
+    }
+
+    @Override
+    public void visit(EvalNode node) {
+      if (EvalTreeUtil.isJoinQual(node, includeThetaJoin)) {
+        Column[] pair = new Column[2];
+
+        for (int i = 0; i <= 1; i++) { // access left, right sub expression
+          Column column = EvalTreeUtil.findAllColumnRefs(node.getExpr(i)).get(0);
+          for (int j = 0; j < schemas.length; j++) {
+            // check whether the column is for either outer or inner
+            // 0 is outer, and 1 is inner
+            if (schemas[j].containsByQualifiedName(column.getQualifiedName())) {
+              pair[j] = column;
+            }
+          }
+        }
+
+        if (pair[0] == null || pair[1] == null) {
+          throw new IllegalStateException("Wrong join key: " + node);
+        }
+        pairs.add(pair);
+      }
+    }
+
+    public List<Column[]> getPairs() {
+      return this.pairs;
+    }
+  }
+
+  public static Schema targetToSchema(Collection<Target> targets) {
+    return targetToSchema(targets.toArray(new Target[targets.size()]));
+  }
+
+  public static Schema targetToSchema(Target[] targets) {
+    Schema schema = new Schema();
+    for (Target t : targets) {
+      DataType type = t.getEvalTree().getValueType();
+      String name;
+      if (t.hasAlias()) {
+        name = t.getAlias();
+      } else {
+        name = t.getEvalTree().getName();
+      }
+      if (!schema.containsByQualifiedName(name)) {
+        schema.addColumn(name, type);
+      }
+    }
+
+    return schema;
+  }
+
+  /**
+   * It removes all table names from FieldEvals in targets
+   *
+   * @param sourceTargets The targets to be stripped
+   * @return The stripped targets
+   */
+  public static Target[] stripTarget(Target[] sourceTargets) {
+    Target[] copy = new Target[sourceTargets.length];
+    for (int i = 0; i < sourceTargets.length; i++) {
+      try {
+        copy[i] = (Target) sourceTargets[i].clone();
+      } catch (CloneNotSupportedException e) {
+        throw new InternalError(e.getMessage());
+      }
+      if (copy[i].getEvalTree().getType() == EvalType.FIELD) {
+        FieldEval fieldEval = copy[i].getEvalTree();
+        if (fieldEval.getColumnRef().hasQualifier()) {
+          fieldEval.replaceColumnRef(fieldEval.getColumnName());
+        }
+      }
+    }
+
+    return copy;
+  }
+
+  public static <T extends LogicalNode> T clone(LogicalPlan plan, LogicalNode node) {
+    try {
+      T copy = (T) node.clone();
+      if (plan == null) {
+        copy.setPID(-1);
+      } else {
+        copy.setPID(plan.newPID());
+      }
+      return copy;
+    } catch (CloneNotSupportedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public static boolean isCommutativeJoin(JoinType joinType) {
+    return joinType == JoinType.INNER;
+  }
+
+  public static boolean existsAggregationFunction(Expr expr) throws PlanningException {
+    AggregationFunctionFinder finder = new AggregationFunctionFinder();
+    AggFunctionFoundResult result = new AggFunctionFoundResult();
+    finder.visit(result, new Stack<Expr>(), expr);
+    return result.generalSetFunction;
+  }
+
+  public static boolean existsDistinctAggregationFunction(Expr expr) throws PlanningException {
+    AggregationFunctionFinder finder = new AggregationFunctionFinder();
+    AggFunctionFoundResult result = new AggFunctionFoundResult();
+    finder.visit(result, new Stack<Expr>(), expr);
+    return result.distinctSetFunction;
+  }
+
+  static class AggFunctionFoundResult {
+    boolean generalSetFunction;
+    boolean distinctSetFunction;
+  }
+
+  static class AggregationFunctionFinder extends SimpleAlgebraVisitor<AggFunctionFoundResult, Object> {
+    @Override
+    public Object visitCountRowsFunction(AggFunctionFoundResult ctx, Stack<Expr> stack, CountRowsFunctionExpr expr)
+        throws PlanningException {
+      ctx.generalSetFunction = true;
+      return super.visitCountRowsFunction(ctx, stack, expr);
+    }
+
+    @Override
+    public Object visitGeneralSetFunction(AggFunctionFoundResult ctx, Stack<Expr> stack, GeneralSetFunctionExpr expr)
+        throws PlanningException {
+      ctx.generalSetFunction = true;
+      ctx.distinctSetFunction = expr.isDistinct();
+      return super.visitGeneralSetFunction(ctx, stack, expr);
+    }
+  }
+
+  public static Collection<String> toQualifiedFieldNames(Collection<String> fieldNames, String qualifier) {
+    List<String> names = TUtil.newList();
+    for (String n : fieldNames) {
+      String[] parts = n.split("\\.");
+      if (parts.length == 1) {
+        names.add(qualifier + "." + parts[0]);
+      } else {
+        names.add(qualifier + "." + parts[1]);
+      }
+    }
+    return names;
+  }
+
+  public static SortSpec[] convertSortSpecs(Collection<CatalogProtos.SortSpecProto> sortSpecProtos) {
+    SortSpec[] sortSpecs = new SortSpec[sortSpecProtos.size()];
+    int i = 0;
+    for (CatalogProtos.SortSpecProto proto : sortSpecProtos) {
+      sortSpecs[i++] = new SortSpec(proto);
+    }
+    return sortSpecs;
+  }
+
+  /**
+   * Generate an explain string of a LogicalNode and its descendant nodes.
+   *
+   * @param node The LogicalNode instance to be started
+   * @return A pretty print explain string
+   */
+  public static String buildExplainString(LogicalNode node) {
+    ExplainLogicalPlanVisitor explain = new ExplainLogicalPlanVisitor();
+
+    StringBuilder explains = new StringBuilder();
+    try {
+      ExplainLogicalPlanVisitor.Context explainContext = explain.getBlockPlanStrings(null, node);
+      while (!explainContext.explains.empty()) {
+        explains.append(
+            ExplainLogicalPlanVisitor.printDepthString(explainContext.getMaxDepth(), explainContext.explains.pop()));
+      }
+    } catch (PlanningException e) {
+      throw new RuntimeException(e);
+    }
+
+    return explains.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanningException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanningException.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanningException.java
new file mode 100644
index 0000000..4fa88ee
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PlanningException.java
@@ -0,0 +1,29 @@
+/**
+ * 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.planner;
+
+public class PlanningException extends Exception {
+  public PlanningException(String message) {
+    super(message);
+  }
+
+  public PlanningException(Exception e) {
+    super(e);
+  }
+}


[21/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
new file mode 100644
index 0000000..15ac6b6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -0,0 +1,537 @@
+/**
+ * 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.master.rm;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.RpcCallback;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.util.ApplicationIdUtils;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import static org.apache.tajo.ipc.TajoMasterProtocol.*;
+
+
+/**
+ * It manages all resources of tajo workers.
+ */
+public class TajoWorkerResourceManager extends CompositeService implements WorkerResourceManager {
+  /** class logger */
+  private static final Log LOG = LogFactory.getLog(TajoWorkerResourceManager.class);
+
+  static AtomicInteger containerIdSeq = new AtomicInteger(0);
+
+  private TajoMaster.MasterContext masterContext;
+
+  private TajoRMContext rmContext;
+
+  private String queryIdSeed;
+
+  private WorkerResourceAllocationThread workerResourceAllocator;
+
+  /**
+   * Worker Liveliness monitor
+   */
+  private WorkerLivelinessMonitor workerLivelinessMonitor;
+
+  private BlockingQueue<WorkerResourceRequest> requestQueue;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private TajoConf systemConf;
+
+  private ConcurrentMap<ContainerIdProto, AllocatedWorkerResource> allocatedResourceMap = Maps.newConcurrentMap();
+
+  /** It receives status messages from workers and their resources. */
+  private TajoResourceTracker resourceTracker;
+
+  public TajoWorkerResourceManager(TajoMaster.MasterContext masterContext) {
+    super(TajoWorkerResourceManager.class.getSimpleName());
+    this.masterContext = masterContext;
+  }
+
+  public TajoWorkerResourceManager(TajoConf systemConf) {
+    super(TajoWorkerResourceManager.class.getSimpleName());
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    Preconditions.checkArgument(conf instanceof TajoConf);
+    this.systemConf = (TajoConf) conf;
+
+    AsyncDispatcher dispatcher = new AsyncDispatcher();
+    addIfService(dispatcher);
+
+    rmContext = new TajoRMContext(dispatcher);
+
+    this.queryIdSeed = String.valueOf(System.currentTimeMillis());
+
+    requestQueue = new LinkedBlockingDeque<WorkerResourceRequest>();
+
+    workerResourceAllocator = new WorkerResourceAllocationThread();
+    workerResourceAllocator.start();
+
+    this.workerLivelinessMonitor = new WorkerLivelinessMonitor(this.rmContext.getDispatcher());
+    addIfService(this.workerLivelinessMonitor);
+
+    // Register event handler for Workers
+    rmContext.getDispatcher().register(WorkerEventType.class, new WorkerEventDispatcher(rmContext));
+
+    resourceTracker = new TajoResourceTracker(rmContext, workerLivelinessMonitor);
+    addIfService(resourceTracker);
+
+    super.serviceInit(systemConf);
+  }
+
+  @InterfaceAudience.Private
+  public static final class WorkerEventDispatcher implements EventHandler<WorkerEvent> {
+
+    private final TajoRMContext rmContext;
+
+    public WorkerEventDispatcher(TajoRMContext rmContext) {
+      this.rmContext = rmContext;
+    }
+
+    @Override
+    public void handle(WorkerEvent event) {
+      String workerId = event.getWorkerId();
+      Worker node = this.rmContext.getWorkers().get(workerId);
+      if (node != null) {
+        try {
+          node.handle(event);
+        } catch (Throwable t) {
+          LOG.error("Error in handling event type " + event.getType() + " for node " + workerId, t);
+        }
+      }
+    }
+  }
+
+  @Override
+  public Map<String, Worker> getWorkers() {
+    return ImmutableMap.copyOf(rmContext.getWorkers());
+  }
+
+  @Override
+  public Map<String, Worker> getInactiveWorkers() {
+    return ImmutableMap.copyOf(rmContext.getInactiveWorkers());
+  }
+
+  public Collection<String> getQueryMasters() {
+    return Collections.unmodifiableSet(rmContext.getQueryMasterWorker());
+  }
+
+  @Override
+  public TajoMasterProtocol.ClusterResourceSummary getClusterResourceSummary() {
+    return resourceTracker.getClusterResourceSummary();
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    if(stopped.get()) {
+      return;
+    }
+    stopped.set(true);
+    if(workerResourceAllocator != null) {
+      workerResourceAllocator.interrupt();
+    }
+
+    super.serviceStop();
+  }
+
+  /**
+   *
+   * @return The prefix of queryId. It is generated when a TajoMaster starts up.
+   */
+  @Override
+  public String getSeedQueryId() throws IOException {
+    return queryIdSeed;
+  }
+
+  @VisibleForTesting
+  TajoResourceTracker getResourceTracker() {
+    return resourceTracker;
+  }
+
+  private WorkerResourceAllocationRequest createQMResourceRequest(QueryId queryId) {
+    float queryMasterDefaultDiskSlot = masterContext.getConf().getFloatVar(
+        TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT);
+    int queryMasterDefaultMemoryMB = masterContext.getConf().getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB);
+
+    WorkerResourceAllocationRequest.Builder builder = WorkerResourceAllocationRequest.newBuilder();
+    builder.setQueryId(queryId.getProto());
+    builder.setMaxMemoryMBPerContainer(queryMasterDefaultMemoryMB);
+    builder.setMinMemoryMBPerContainer(queryMasterDefaultMemoryMB);
+    builder.setMaxDiskSlotPerContainer(queryMasterDefaultDiskSlot);
+    builder.setMinDiskSlotPerContainer(queryMasterDefaultDiskSlot);
+    builder.setResourceRequestPriority(TajoMasterProtocol.ResourceRequestPriority.MEMORY);
+    builder.setNumContainers(1);
+    return builder.build();
+  }
+
+  @Override
+  public WorkerAllocatedResource allocateQueryMaster(QueryInProgress queryInProgress) {
+    // Create a resource request for a query master
+    WorkerResourceAllocationRequest qmResourceRequest = createQMResourceRequest(queryInProgress.getQueryId());
+
+    // call future for async call
+    CallFuture<WorkerResourceAllocationResponse> callFuture = new CallFuture<WorkerResourceAllocationResponse>();
+    allocateWorkerResources(qmResourceRequest, callFuture);
+
+    // Wait for 3 seconds
+    WorkerResourceAllocationResponse response = null;
+    try {
+      response = callFuture.get(3, TimeUnit.SECONDS);
+    } catch (Throwable t) {
+      LOG.error(t);
+      return null;
+    }
+
+    if (response.getWorkerAllocatedResourceList().size() == 0) {
+      return null;
+    }
+
+    WorkerAllocatedResource resource = response.getWorkerAllocatedResource(0);
+    registerQueryMaster(queryInProgress.getQueryId(), resource.getContainerId());
+    return resource;
+  }
+
+  private void registerQueryMaster(QueryId queryId, ContainerIdProto containerId) {
+    rmContext.getQueryMasterContainer().putIfAbsent(queryId, containerId);
+  }
+
+  @Override
+  public void allocateWorkerResources(WorkerResourceAllocationRequest request,
+                                      RpcCallback<WorkerResourceAllocationResponse> callBack) {
+    try {
+      //TODO checking queue size
+      requestQueue.put(new WorkerResourceRequest(new QueryId(request.getQueryId()), false, request, callBack));
+    } catch (InterruptedException e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  static class WorkerResourceRequest {
+    boolean queryMasterRequest;
+    QueryId queryId;
+    WorkerResourceAllocationRequest request;
+    RpcCallback<WorkerResourceAllocationResponse> callBack;
+    WorkerResourceRequest(
+        QueryId queryId,
+        boolean queryMasterRequest, WorkerResourceAllocationRequest request,
+        RpcCallback<WorkerResourceAllocationResponse> callBack) {
+      this.queryId = queryId;
+      this.queryMasterRequest = queryMasterRequest;
+      this.request = request;
+      this.callBack = callBack;
+    }
+  }
+
+  static class AllocatedWorkerResource {
+    Worker worker;
+    int allocatedMemoryMB;
+    float allocatedDiskSlots;
+  }
+
+  class WorkerResourceAllocationThread extends Thread {
+    @Override
+    public void run() {
+      LOG.info("WorkerResourceAllocationThread start");
+      while(!stopped.get()) {
+        try {
+          WorkerResourceRequest resourceRequest = requestQueue.take();
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("allocateWorkerResources:" +
+                (new QueryId(resourceRequest.request.getQueryId())) +
+                ", requiredMemory:" + resourceRequest.request.getMinMemoryMBPerContainer() +
+                "~" + resourceRequest.request.getMaxMemoryMBPerContainer() +
+                ", requiredContainers:" + resourceRequest.request.getNumContainers() +
+                ", requiredDiskSlots:" + resourceRequest.request.getMinDiskSlotPerContainer() +
+                "~" + resourceRequest.request.getMaxDiskSlotPerContainer() +
+                ", queryMasterRequest=" + resourceRequest.queryMasterRequest +
+                ", liveWorkers=" + rmContext.getWorkers().size());
+          }
+
+          List<AllocatedWorkerResource> allocatedWorkerResources = chooseWorkers(resourceRequest);
+
+          if(allocatedWorkerResources.size() > 0) {
+            List<WorkerAllocatedResource> allocatedResources =
+                new ArrayList<WorkerAllocatedResource>();
+
+            for(AllocatedWorkerResource allocatedResource: allocatedWorkerResources) {
+              NodeId nodeId = NodeId.newInstance(allocatedResource.worker.getHostName(),
+                  allocatedResource.worker.getPeerRpcPort());
+
+              TajoWorkerContainerId containerId = new TajoWorkerContainerId();
+
+              containerId.setApplicationAttemptId(
+                  ApplicationIdUtils.createApplicationAttemptId(resourceRequest.queryId));
+              containerId.setId(containerIdSeq.incrementAndGet());
+
+              ContainerIdProto containerIdProto = containerId.getProto();
+              allocatedResources.add(WorkerAllocatedResource.newBuilder()
+                  .setContainerId(containerIdProto)
+                  .setNodeId(nodeId.toString())
+                  .setWorkerHost(allocatedResource.worker.getHostName())
+                  .setQueryMasterPort(allocatedResource.worker.getQueryMasterPort())
+                  .setClientPort(allocatedResource.worker.getClientPort())
+                  .setPeerRpcPort(allocatedResource.worker.getPeerRpcPort())
+                  .setWorkerPullServerPort(allocatedResource.worker.getPullServerPort())
+                  .setAllocatedMemoryMB(allocatedResource.allocatedMemoryMB)
+                  .setAllocatedDiskSlots(allocatedResource.allocatedDiskSlots)
+                  .build());
+
+
+              allocatedResourceMap.putIfAbsent(containerIdProto, allocatedResource);
+            }
+
+            resourceRequest.callBack.run(WorkerResourceAllocationResponse.newBuilder()
+                .setQueryId(resourceRequest.request.getQueryId())
+                .addAllWorkerAllocatedResource(allocatedResources)
+                .build()
+            );
+
+          } else {
+            if(LOG.isDebugEnabled()) {
+              LOG.debug("=========================================");
+              LOG.debug("Available Workers");
+              for(String liveWorker: rmContext.getWorkers().keySet()) {
+                LOG.debug(rmContext.getWorkers().get(liveWorker).toString());
+              }
+              LOG.debug("=========================================");
+            }
+            requestQueue.put(resourceRequest);
+            Thread.sleep(100);
+          }
+        } catch(InterruptedException ie) {
+          LOG.error(ie);
+        }
+      }
+    }
+  }
+
+  private List<AllocatedWorkerResource> chooseWorkers(WorkerResourceRequest resourceRequest) {
+    List<AllocatedWorkerResource> selectedWorkers = new ArrayList<AllocatedWorkerResource>();
+
+    int allocatedResources = 0;
+
+    TajoMasterProtocol.ResourceRequestPriority resourceRequestPriority
+        = resourceRequest.request.getResourceRequestPriority();
+
+    if(resourceRequestPriority == TajoMasterProtocol.ResourceRequestPriority.MEMORY) {
+      synchronized(rmContext) {
+        List<String> randomWorkers = new ArrayList<String>(rmContext.getWorkers().keySet());
+        Collections.shuffle(randomWorkers);
+
+        int numContainers = resourceRequest.request.getNumContainers();
+        int minMemoryMB = resourceRequest.request.getMinMemoryMBPerContainer();
+        int maxMemoryMB = resourceRequest.request.getMaxMemoryMBPerContainer();
+        float diskSlot = Math.max(resourceRequest.request.getMaxDiskSlotPerContainer(),
+            resourceRequest.request.getMinDiskSlotPerContainer());
+
+        int liveWorkerSize = randomWorkers.size();
+        Set<String> insufficientWorkers = new HashSet<String>();
+        boolean stop = false;
+        boolean checkMax = true;
+        while(!stop) {
+          if(allocatedResources >= numContainers) {
+            break;
+          }
+
+          if(insufficientWorkers.size() >= liveWorkerSize) {
+            if(!checkMax) {
+              break;
+            }
+            insufficientWorkers.clear();
+            checkMax = false;
+          }
+          int compareAvailableMemory = checkMax ? maxMemoryMB : minMemoryMB;
+
+          for(String eachWorker: randomWorkers) {
+            if(allocatedResources >= numContainers) {
+              stop = true;
+              break;
+            }
+
+            if(insufficientWorkers.size() >= liveWorkerSize) {
+              break;
+            }
+
+            Worker worker = rmContext.getWorkers().get(eachWorker);
+            WorkerResource workerResource = worker.getResource();
+            if(workerResource.getAvailableMemoryMB() >= compareAvailableMemory) {
+              int workerMemory;
+              if(workerResource.getAvailableMemoryMB() >= maxMemoryMB) {
+                workerMemory = maxMemoryMB;
+              } else {
+                workerMemory = workerResource.getAvailableMemoryMB();
+              }
+              AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource();
+              allocatedWorkerResource.worker = worker;
+              allocatedWorkerResource.allocatedMemoryMB = workerMemory;
+              if(workerResource.getAvailableDiskSlots() >= diskSlot) {
+                allocatedWorkerResource.allocatedDiskSlots = diskSlot;
+              } else {
+                allocatedWorkerResource.allocatedDiskSlots = workerResource.getAvailableDiskSlots();
+              }
+
+              workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots,
+                  allocatedWorkerResource.allocatedMemoryMB);
+
+              selectedWorkers.add(allocatedWorkerResource);
+
+              allocatedResources++;
+            } else {
+              insufficientWorkers.add(eachWorker);
+            }
+          }
+        }
+      }
+    } else {
+      synchronized(rmContext) {
+        List<String> randomWorkers = new ArrayList<String>(rmContext.getWorkers().keySet());
+        Collections.shuffle(randomWorkers);
+
+        int numContainers = resourceRequest.request.getNumContainers();
+        float minDiskSlots = resourceRequest.request.getMinDiskSlotPerContainer();
+        float maxDiskSlots = resourceRequest.request.getMaxDiskSlotPerContainer();
+        int memoryMB = Math.max(resourceRequest.request.getMaxMemoryMBPerContainer(),
+            resourceRequest.request.getMinMemoryMBPerContainer());
+
+        int liveWorkerSize = randomWorkers.size();
+        Set<String> insufficientWorkers = new HashSet<String>();
+        boolean stop = false;
+        boolean checkMax = true;
+        while(!stop) {
+          if(allocatedResources >= numContainers) {
+            break;
+          }
+
+          if(insufficientWorkers.size() >= liveWorkerSize) {
+            if(!checkMax) {
+              break;
+            }
+            insufficientWorkers.clear();
+            checkMax = false;
+          }
+          float compareAvailableDisk = checkMax ? maxDiskSlots : minDiskSlots;
+
+          for(String eachWorker: randomWorkers) {
+            if(allocatedResources >= numContainers) {
+              stop = true;
+              break;
+            }
+
+            if(insufficientWorkers.size() >= liveWorkerSize) {
+              break;
+            }
+
+            Worker worker = rmContext.getWorkers().get(eachWorker);
+            WorkerResource workerResource = worker.getResource();
+            if(workerResource.getAvailableDiskSlots() >= compareAvailableDisk) {
+              float workerDiskSlots;
+              if(workerResource.getAvailableDiskSlots() >= maxDiskSlots) {
+                workerDiskSlots = maxDiskSlots;
+              } else {
+                workerDiskSlots = workerResource.getAvailableDiskSlots();
+              }
+              AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource();
+              allocatedWorkerResource.worker = worker;
+              allocatedWorkerResource.allocatedDiskSlots = workerDiskSlots;
+
+              if(workerResource.getAvailableMemoryMB() >= memoryMB) {
+                allocatedWorkerResource.allocatedMemoryMB = memoryMB;
+              } else {
+                allocatedWorkerResource.allocatedMemoryMB = workerResource.getAvailableMemoryMB();
+              }
+              workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots,
+                  allocatedWorkerResource.allocatedMemoryMB);
+
+              selectedWorkers.add(allocatedWorkerResource);
+
+              allocatedResources++;
+            } else {
+              insufficientWorkers.add(eachWorker);
+            }
+          }
+        }
+      }
+    }
+    return selectedWorkers;
+  }
+
+  /**
+   * Release allocated resource.
+   *
+   * @param containerId ContainerIdProto to be released
+   */
+  @Override
+  public void releaseWorkerResource(ContainerIdProto containerId) {
+    AllocatedWorkerResource allocated = allocatedResourceMap.get(containerId);
+    if(allocated != null) {
+      LOG.info("Release Resource: " + allocated.allocatedDiskSlots + "," + allocated.allocatedMemoryMB);
+      allocated.worker.getResource().releaseResource( allocated.allocatedDiskSlots, allocated.allocatedMemoryMB);
+    } else {
+      LOG.warn("No AllocatedWorkerResource data for [" + containerId + "]");
+      return;
+    }
+  }
+
+  @Override
+  public boolean isQueryMasterStopped(QueryId queryId) {
+    return !rmContext.getQueryMasterContainer().containsKey(queryId);
+  }
+
+  @Override
+  public void stopQueryMaster(QueryId queryId) {
+    WorkerResource resource = null;
+    if(!rmContext.getQueryMasterContainer().containsKey(queryId)) {
+      LOG.warn("No QueryMaster resource info for " + queryId);
+      return;
+    } else {
+      ContainerIdProto containerId = rmContext.getQueryMasterContainer().remove(queryId);
+      releaseWorkerResource(containerId);
+      LOG.info(String.format("Released QueryMaster (%s) resource:" + resource, queryId.toString()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java
new file mode 100644
index 0000000..0d6b5ee
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java
@@ -0,0 +1,296 @@
+/**
+ * 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.master.rm;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+
+import java.util.EnumSet;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * It contains resource and various information for a worker.
+ */
+public class Worker implements EventHandler<WorkerEvent>, Comparable<Worker> {
+  /** class logger */
+  private static final Log LOG = LogFactory.getLog(Worker.class);
+
+  private final ReentrantReadWriteLock.ReadLock readLock;
+  private final ReentrantReadWriteLock.WriteLock writeLock;
+
+  /** context of {@link org.apache.tajo.master.rm.TajoWorkerResourceManager} */
+  private final TajoRMContext rmContext;
+
+  /** Hostname */
+  private String hostName;
+  /** QueryMaster rpc port */
+  private int qmRpcPort;
+  /** Peer rpc port */
+  private int peerRpcPort;
+  /** http info port */
+  private int httpInfoPort;
+  /** the port of QueryMaster client rpc which provides an client API */
+  private int qmClientPort;
+  /** pull server port */
+  private int pullServerPort;
+  /** last heartbeat time */
+  private long lastHeartbeatTime;
+
+  /** Resource capability */
+  private WorkerResource resource;
+
+  private static final ReconnectNodeTransition RECONNECT_NODE_TRANSITION = new ReconnectNodeTransition();
+  private static final StatusUpdateTransition STATUS_UPDATE_TRANSITION = new StatusUpdateTransition();
+
+  private static final StateMachineFactory<Worker,
+      WorkerState,
+      WorkerEventType,
+      WorkerEvent> stateMachineFactory
+      = new StateMachineFactory<Worker,
+      WorkerState,
+      WorkerEventType,
+      WorkerEvent>(WorkerState.NEW)
+
+      // Transition from NEW
+      .addTransition(WorkerState.NEW, WorkerState.RUNNING,
+          WorkerEventType.STARTED,
+          new AddNodeTransition())
+
+      // Transition from RUNNING
+      .addTransition(WorkerState.RUNNING, EnumSet.of(WorkerState.RUNNING, WorkerState.UNHEALTHY),
+          WorkerEventType.STATE_UPDATE,
+          STATUS_UPDATE_TRANSITION)
+      .addTransition(WorkerState.RUNNING, WorkerState.LOST,
+          WorkerEventType.EXPIRE,
+          new DeactivateNodeTransition(WorkerState.LOST))
+      .addTransition(WorkerState.RUNNING, WorkerState.RUNNING,
+          WorkerEventType.RECONNECTED,
+          RECONNECT_NODE_TRANSITION)
+
+      // Transitions from UNHEALTHY state
+      .addTransition(WorkerState.UNHEALTHY, EnumSet.of(WorkerState.RUNNING, WorkerState.UNHEALTHY),
+          WorkerEventType.STATE_UPDATE,
+          STATUS_UPDATE_TRANSITION)
+      .addTransition(WorkerState.UNHEALTHY, WorkerState.LOST,
+          WorkerEventType.EXPIRE,
+          new DeactivateNodeTransition(WorkerState.LOST))
+      .addTransition(WorkerState.UNHEALTHY, WorkerState.UNHEALTHY,
+          WorkerEventType.RECONNECTED,
+          RECONNECT_NODE_TRANSITION);
+
+  private final StateMachine<WorkerState, WorkerEventType, WorkerEvent> stateMachine =
+      stateMachineFactory.make(this, WorkerState.NEW);
+
+  public Worker(TajoRMContext rmContext, WorkerResource resource) {
+    this.rmContext = rmContext;
+
+    this.lastHeartbeatTime = System.currentTimeMillis();
+    this.resource = resource;
+
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    this.readLock = lock.readLock();
+    this.writeLock = lock.writeLock();
+  }
+
+  public String getWorkerId() {
+    return hostName + ":" + qmRpcPort + ":" + peerRpcPort;
+  }
+
+  public String getHostName() {
+    return hostName;
+  }
+
+  public void setHostName(String allocatedHost) {
+    this.hostName = allocatedHost;
+  }
+
+  public int getPeerRpcPort() {
+    return peerRpcPort;
+  }
+
+  public void setPeerRpcPort(int peerRpcPort) {
+    this.peerRpcPort = peerRpcPort;
+  }
+
+  public int getQueryMasterPort() {
+    return qmRpcPort;
+  }
+
+  public void setQueryMasterPort(int queryMasterPort) {
+    this.qmRpcPort = queryMasterPort;
+  }
+
+  public int getClientPort() {
+    return qmClientPort;
+  }
+
+  public void setClientPort(int clientPort) {
+    this.qmClientPort = clientPort;
+  }
+
+  public int getPullServerPort() {
+    return pullServerPort;
+  }
+
+  public void setPullServerPort(int pullServerPort) {
+    this.pullServerPort = pullServerPort;
+  }
+
+  public int getHttpPort() {
+    return httpInfoPort;
+  }
+
+  public void setHttpPort(int port) {
+    this.httpInfoPort = port;
+  }
+
+  public void setLastHeartbeatTime(long lastheartbeatReportTime) {
+    this.writeLock.lock();
+
+    try {
+      this.lastHeartbeatTime = lastheartbeatReportTime;
+    } finally {
+      this.writeLock.unlock();
+    }
+  }
+
+  public long getLastHeartbeatTime() {
+    this.readLock.lock();
+
+    try {
+      return this.lastHeartbeatTime;
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
+   *
+   * @return the current state of worker
+   */
+  public WorkerState getState() {
+    this.readLock.lock();
+
+    try {
+      return this.stateMachine.getCurrentState();
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
+   *
+   * @return the current resource capability of worker
+   */
+  public WorkerResource getResource() {
+    return this.resource;
+  }
+
+  @Override
+  public int compareTo(Worker o) {
+    if(o == null) {
+      return 1;
+    }
+    return getWorkerId().compareTo(o.getWorkerId());
+  }
+
+  public static class AddNodeTransition implements SingleArcTransition<Worker, WorkerEvent> {
+    @Override
+    public void transition(Worker worker, WorkerEvent workerEvent) {
+
+      if(worker.getResource().isQueryMasterMode()) {
+        worker.rmContext.getQueryMasterWorker().add(worker.getWorkerId());
+      }
+      LOG.info("Worker with " + worker.getResource() + " is joined to Tajo cluster");
+    }
+  }
+
+  public static class StatusUpdateTransition implements
+      MultipleArcTransition<Worker, WorkerEvent, WorkerState> {
+
+    @Override
+    public WorkerState transition(Worker worker, WorkerEvent event) {
+      WorkerStatusEvent statusEvent = (WorkerStatusEvent) event;
+
+      // TODO - the synchronization scope using rmContext is too coarsen.
+      synchronized (worker.rmContext) {
+        worker.setLastHeartbeatTime(System.currentTimeMillis());
+        worker.getResource().setNumRunningTasks(statusEvent.getRunningTaskNum());
+        worker.getResource().setMaxHeap(statusEvent.maxHeap());
+        worker.getResource().setFreeHeap(statusEvent.getFreeHeap());
+        worker.getResource().setTotalHeap(statusEvent.getTotalHeap());
+      }
+
+      return WorkerState.RUNNING;
+    }
+  }
+
+  public static class DeactivateNodeTransition implements SingleArcTransition<Worker, WorkerEvent> {
+    private final WorkerState finalState;
+
+    public DeactivateNodeTransition(WorkerState finalState) {
+      this.finalState = finalState;
+    }
+
+    @Override
+    public void transition(Worker worker, WorkerEvent workerEvent) {
+
+      worker.rmContext.getWorkers().remove(worker.getWorkerId());
+      LOG.info("Deactivating Node " + worker.getWorkerId() + " as it is now " + finalState);
+      worker.rmContext.getInactiveWorkers().putIfAbsent(worker.getWorkerId(), worker);
+    }
+  }
+
+  public static class ReconnectNodeTransition implements SingleArcTransition<Worker, WorkerEvent> {
+
+    @Override
+    public void transition(Worker worker, WorkerEvent workerEvent) {
+      WorkerReconnectEvent castedEvent = (WorkerReconnectEvent) workerEvent;
+
+      Worker newWorker = castedEvent.getWorker();
+      worker.rmContext.getWorkers().put(castedEvent.getWorkerId(), newWorker);
+      worker.rmContext.getDispatcher().getEventHandler().handle(
+          new WorkerEvent(worker.getWorkerId(), WorkerEventType.STARTED));
+    }
+  }
+
+  @Override
+  public void handle(WorkerEvent event) {
+    LOG.debug("Processing " + event.getWorkerId() + " of type " + event.getType());
+    try {
+      writeLock.lock();
+      WorkerState oldState = getState();
+      try {
+        stateMachine.doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state", e);
+        LOG.error("Invalid event " + event.getType() + " on Worker  " + getWorkerId());
+      }
+      if (oldState != getState()) {
+        LOG.info(getWorkerId() + " Node Transitioned from " + oldState + " to " + getState());
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEvent.java
new file mode 100644
index 0000000..389c3be
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEvent.java
@@ -0,0 +1,37 @@
+/**
+ * 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.master.rm;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+/**
+ * WorkerEvent describes all kinds of events which sent to {@link Worker}.
+ */
+public class WorkerEvent extends AbstractEvent<WorkerEventType> {
+  private final String workerId;
+
+  public WorkerEvent(String workerId, WorkerEventType workerEventType) {
+    super(workerEventType);
+    this.workerId = workerId;
+  }
+
+  public String getWorkerId() {
+    return workerId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEventType.java
new file mode 100644
index 0000000..0c97654
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEventType.java
@@ -0,0 +1,30 @@
+/**
+ * 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.master.rm;
+
+public enum WorkerEventType {
+
+  /** Source : {@link TajoResourceTracker}, Destination: {@link Worker} */
+  STARTED,
+  STATE_UPDATE,
+  RECONNECTED,
+
+  /** Source : {@link WorkerLivelinessMonitor}, Destination: {@link Worker} */
+  EXPIRE
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java
new file mode 100644
index 0000000..e3524d6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tajo.conf.TajoConf;
+
+/**
+ * It periodically checks the latest heartbeat time of {@link Worker}.
+ * If the latest heartbeat time is expired, it produces EXPIRE event to a corresponding {@link Worker}.
+ */
+public class WorkerLivelinessMonitor extends AbstractLivelinessMonitor<String> {
+
+  private EventHandler dispatcher;
+
+  public WorkerLivelinessMonitor(Dispatcher d) {
+    super(WorkerLivelinessMonitor.class.getSimpleName(), new SystemClock());
+    this.dispatcher = d.getEventHandler();
+  }
+
+  public void serviceInit(Configuration conf) throws Exception {
+    Preconditions.checkArgument(conf instanceof TajoConf);
+    TajoConf systemConf = (TajoConf) conf;
+    // milliseconds
+    int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.RESOURCE_TRACKER_HEARTBEAT_TIMEOUT);
+    setExpireInterval(expireIntvl);
+    setMonitorInterval(expireIntvl/3);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void expire(String id) {
+    dispatcher.handle(new WorkerEvent(id, WorkerEventType.EXPIRE));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerReconnectEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerReconnectEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerReconnectEvent.java
new file mode 100644
index 0000000..46f286d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerReconnectEvent.java
@@ -0,0 +1,35 @@
+/**
+ * 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.master.rm;
+
+/**
+ * {@link TajoResourceTracker} produces this event, and it's destination is {@link Worker}.
+ * This event occurs only when an inactive worker sends a ping again.
+ */
+public class WorkerReconnectEvent extends WorkerEvent {
+  private final Worker worker;
+  public WorkerReconnectEvent(String workerId, Worker worker) {
+    super(workerId, WorkerEventType.RECONNECTED);
+    this.worker = worker;
+  }
+
+  public Worker getWorker() {
+    return worker;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResource.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
new file mode 100644
index 0000000..bfe186c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
@@ -0,0 +1,235 @@
+/**
+ * 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.master.rm;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Describe current resources of a worker.
+ *
+ * It includes various resource capability of a worker as follows:
+ * <ul>
+ *   <li>used/total disk slots</li>
+ *   <li>used/total core slots</li>
+ *   <li>used/total memory</li>
+ *   <li>the number of running tasks</li>
+ * </ul>
+ */
+public class WorkerResource {
+  private static final Log LOG = LogFactory.getLog(WorkerResource.class);
+
+  private float diskSlots;
+  private int cpuCoreSlots;
+  private int memoryMB;
+
+  private float usedDiskSlots;
+  private int usedMemoryMB;
+  private int usedCpuCoreSlots;
+
+  private long maxHeap;
+  private long freeHeap;
+  private long totalHeap;
+
+  private int numRunningTasks;
+
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private final Lock rlock = lock.readLock();
+  private final Lock wlock = lock.writeLock();
+
+  private boolean queryMasterMode;
+
+  private boolean taskRunnerMode;
+
+  private AtomicInteger numQueryMasterTasks = new AtomicInteger(0);
+
+  public float getDiskSlots() {
+    return diskSlots;
+  }
+
+  public void setDiskSlots(float diskSlots) {
+    this.diskSlots = diskSlots;
+  }
+
+  public int getCpuCoreSlots() {
+    return cpuCoreSlots;
+  }
+
+  public void setCpuCoreSlots(int cpuCoreSlots) {
+    this.cpuCoreSlots = cpuCoreSlots;
+  }
+
+  public int getMemoryMB() {
+    try {
+      rlock.lock();
+      return memoryMB;
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  public void setMemoryMB(int memoryMB) {
+    try {
+      wlock.lock();
+      this.memoryMB = memoryMB;
+    } finally {
+      wlock.unlock();
+    }
+  }
+
+  public float getAvailableDiskSlots() {
+    return diskSlots - usedDiskSlots;
+  }
+
+  public int getAvailableMemoryMB() {
+    return memoryMB - usedMemoryMB;
+  }
+
+  public int getAvailableCpuCoreSlots() {
+    return cpuCoreSlots - usedCpuCoreSlots;
+  }
+
+  @Override
+  public String toString() {
+    return "slots=m:" + memoryMB + ",d:" + diskSlots +
+        ",c:" + cpuCoreSlots + ", used=m:" + usedMemoryMB + ",d:" + usedDiskSlots + ",c:" + usedCpuCoreSlots;
+  }
+
+  public int getUsedMemoryMB() {
+    try {
+      rlock.lock();
+      return usedMemoryMB;
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  public void setUsedMemoryMB(int usedMemoryMB) {
+    try {
+      wlock.lock();
+      this.usedMemoryMB = usedMemoryMB;
+    } finally {
+      wlock.unlock();
+    }
+  }
+
+  public int getUsedCpuCoreSlots() {
+    return usedCpuCoreSlots;
+  }
+
+  public void setUsedCpuCoreSlots(int usedCpuCoreSlots) {
+    this.usedCpuCoreSlots = usedCpuCoreSlots;
+  }
+
+  public float getUsedDiskSlots() {
+    return usedDiskSlots;
+  }
+
+  public boolean isQueryMasterMode() {
+    return queryMasterMode;
+  }
+
+  public void setQueryMasterMode(boolean queryMasterMode) {
+    this.queryMasterMode = queryMasterMode;
+  }
+
+  public boolean isTaskRunnerMode() {
+    return taskRunnerMode;
+  }
+
+  public void setTaskRunnerMode(boolean taskRunnerMode) {
+    this.taskRunnerMode = taskRunnerMode;
+  }
+
+  public void releaseResource(float diskSlots, int memoryMB) {
+    try {
+      wlock.lock();
+      usedMemoryMB = usedMemoryMB - memoryMB;
+      usedDiskSlots -= diskSlots;
+      if(usedMemoryMB < 0) {
+        LOG.warn("Used memory can't be a minus: " + usedMemoryMB);
+        usedMemoryMB = 0;
+      }
+      if(usedDiskSlots < 0) {
+        LOG.warn("Used disk slot can't be a minus: " + usedDiskSlots);
+        usedDiskSlots = 0;
+      }
+    } finally {
+      wlock.unlock();
+    }
+  }
+
+  public void allocateResource(float diskSlots, int memoryMB) {
+    try {
+      wlock.lock();
+      usedMemoryMB += memoryMB;
+      usedDiskSlots += diskSlots;
+
+      if(usedMemoryMB > this.memoryMB) {
+        usedMemoryMB = this.memoryMB;
+      }
+
+      if(usedDiskSlots > this.diskSlots) {
+        usedDiskSlots = this.diskSlots;
+      }
+    } finally {
+      wlock.unlock();
+    }
+  }
+
+  public long getMaxHeap() {
+    return maxHeap;
+  }
+
+  public void setMaxHeap(long maxHeap) {
+    this.maxHeap = maxHeap;
+  }
+
+  public long getFreeHeap() {
+    return freeHeap;
+  }
+
+  public void setFreeHeap(long freeHeap) {
+    this.freeHeap = freeHeap;
+  }
+
+  public long getTotalHeap() {
+    return totalHeap;
+  }
+
+  public void setTotalHeap(long totalHeap) {
+    this.totalHeap = totalHeap;
+  }
+
+  public int getNumRunningTasks() {
+    return numRunningTasks;
+  }
+
+  public void setNumRunningTasks(int numRunningTasks) {
+    this.numRunningTasks = numRunningTasks;
+  }
+
+  public int getNumQueryMasterTasks() {
+    return numQueryMasterTasks.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
new file mode 100644
index 0000000..54fe11c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
@@ -0,0 +1,110 @@
+/**
+ * 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.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+import static org.apache.tajo.ipc.TajoMasterProtocol.WorkerAllocatedResource;
+import static org.apache.tajo.ipc.TajoMasterProtocol.WorkerResourceAllocationResponse;
+
+/**
+ * An interface of WorkerResourceManager which allows TajoMaster to request allocation for containers
+ * and release the allocated containers.
+ */
+public interface WorkerResourceManager extends Service {
+
+  /**
+   * Request a resource container for a QueryMaster.
+   *
+   * @param queryInProgress QueryInProgress
+   * @return A allocated container resource
+   */
+  @Deprecated
+  public WorkerAllocatedResource allocateQueryMaster(QueryInProgress queryInProgress);
+
+  /**
+   * Request one or more resource containers. You can set the number of containers and resource capabilities, such as
+   * memory, CPU cores, and disk slots. This is an asynchronous call. You should use a callback to get allocated
+   * resource containers. Each container is identified {@link org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto}.
+   *
+   * @param request Request description
+   * @param rpcCallBack Callback function
+   */
+  public void allocateWorkerResources(TajoMasterProtocol.WorkerResourceAllocationRequest request,
+      RpcCallback<WorkerResourceAllocationResponse> rpcCallBack);
+
+  /**
+   * Release a container
+   *
+   * @param containerId ContainerIdProto to be released
+   */
+  public void releaseWorkerResource(ContainerIdProto containerId);
+
+  public String getSeedQueryId() throws IOException;
+
+  /**
+   * Check if a query master is stopped.
+   *
+   * @param queryId QueryId to be checked
+   * @return True if QueryMaster is stopped
+   */
+  public boolean isQueryMasterStopped(QueryId queryId);
+
+  /**
+   * Stop a query master
+   *
+   * @param queryId QueryId to be stopped
+   */
+  public void stopQueryMaster(QueryId queryId);
+
+  /**
+   *
+   * @return a Map instance containing active workers
+   */
+  public Map<String, Worker> getWorkers();
+
+  /**
+   *
+   * @return a Map instance containing inactive workers
+   */
+  public Map<String, Worker> getInactiveWorkers();
+
+  public void stop();
+
+  /**
+   *
+   * @return The overall summary of cluster resources
+   */
+  public TajoMasterProtocol.ClusterResourceSummary getClusterResourceSummary();
+
+  /**
+   *
+   * @return WorkerIds on which QueryMasters are running
+   */
+  Collection<String> getQueryMasters();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerState.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerState.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerState.java
new file mode 100644
index 0000000..a941008
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerState.java
@@ -0,0 +1,44 @@
+/**
+ * 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.master.rm;
+
+/**
+ * It presents the states of {@link Worker}.
+ */
+public enum WorkerState {
+  /** New worker */
+  NEW,
+
+  /** Running worker */
+  RUNNING,
+
+  /** Worker is unhealthy */
+  UNHEALTHY,
+
+  /** worker is out of service */
+  DECOMMISSIONED,
+
+  /** worker has not sent a heartbeat for some configured time threshold */
+  LOST;
+
+  @SuppressWarnings("unused")
+  public boolean isUnusable() {
+    return (this == UNHEALTHY || this == DECOMMISSIONED || this == LOST);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java
new file mode 100644
index 0000000..8c3d7c1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java
@@ -0,0 +1,54 @@
+/**
+ * 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.master.rm;
+
+/**
+ * {@link TajoResourceTracker} produces this event, and its destination is
+ * {@link org.apache.tajo.master.rm.Worker.StatusUpdateTransition} of {@link Worker}.
+ */
+public class WorkerStatusEvent extends WorkerEvent {
+  private final int runningTaskNum;
+  private final long maxHeap;
+  private final long freeHeap;
+  private final long totalHeap;
+
+  public WorkerStatusEvent(String workerId, int runningTaskNum, long maxHeap, long freeHeap, long totalHeap) {
+    super(workerId, WorkerEventType.STATE_UPDATE);
+    this.runningTaskNum = runningTaskNum;
+    this.maxHeap = maxHeap;
+    this.freeHeap = freeHeap;
+    this.totalHeap = totalHeap;
+  }
+
+  public int getRunningTaskNum() {
+    return runningTaskNum;
+  }
+
+  public long maxHeap() {
+    return maxHeap;
+  }
+
+  public long getFreeHeap() {
+    return freeHeap;
+  }
+
+  public long getTotalHeap() {
+    return totalHeap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
new file mode 100644
index 0000000..b9e132b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
@@ -0,0 +1,237 @@
+/**
+ * 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.master.rm;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.master.event.ContainerAllocationEvent;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
+import org.apache.tajo.master.querymaster.Query;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.SubQueryState;
+import org.apache.tajo.util.ApplicationIdUtils;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class YarnRMContainerAllocator extends AMRMClientImpl
+    implements EventHandler<ContainerAllocationEvent> {
+
+  /** Class Logger */
+  private static final Log LOG = LogFactory.getLog(YarnRMContainerAllocator.
+      class.getName());
+
+  private QueryMasterTask.QueryMasterTaskContext context;
+  private ApplicationAttemptId appAttemptId;
+  private final EventHandler eventHandler;
+
+  public YarnRMContainerAllocator(QueryMasterTask.QueryMasterTaskContext context) {
+    super();
+    this.context = context;
+    this.appAttemptId = ApplicationIdUtils.createApplicationAttemptId(context.getQueryId());
+    this.eventHandler = context.getDispatcher().getEventHandler();
+  }
+
+  public void init(Configuration conf) {
+    super.init(conf);
+  }
+
+  private static final int WAIT_INTERVAL_AVAILABLE_NODES = 500; // 0.5 second
+  public void start() {
+    super.start();
+
+    RegisterApplicationMasterResponse response;
+    try {
+      response = registerApplicationMaster("localhost", 10080, "http://localhost:1234");
+
+      // If the number of cluster nodes is ZERO, it waits for available nodes.
+      AllocateResponse allocateResponse = allocate(0.0f);
+      while(allocateResponse.getNumClusterNodes() < 1) {
+        try {
+          Thread.sleep(WAIT_INTERVAL_AVAILABLE_NODES);
+          LOG.info("Waiting for Available Cluster Nodes");
+          allocateResponse = allocate(0);
+        } catch (InterruptedException e) {
+          LOG.error(e);
+        }
+      }
+      context.getQueryMasterContext().getWorkerContext().setNumClusterNodes(allocateResponse.getNumClusterNodes());
+    } catch (IOException e) {
+      LOG.error(e);
+    } catch (YarnException e) {
+      LOG.error(e);
+    }
+
+    startAllocatorThread();
+  }
+
+  protected Thread allocatorThread;
+  private final AtomicBoolean stopped = new AtomicBoolean(false);
+  private int rmPollInterval = 100;//millis
+
+  protected void startAllocatorThread() {
+    allocatorThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
+          try {
+            try {
+              heartbeat();
+            } catch (YarnException e) {
+              LOG.error("Error communicating with RM: " + e.getMessage() , e);
+              return;
+            } catch (Exception e) {
+              LOG.error("ERROR IN CONTACTING RM. ", e);
+              // TODO: for other exceptions
+              if(stopped.get()) {
+                break;
+              }
+            }
+            Thread.sleep(rmPollInterval);
+          } catch (InterruptedException e) {
+            if (!stopped.get()) {
+              LOG.warn("Allocated thread interrupted. Returning.");
+            }
+            break;
+          }
+        }
+        LOG.info("Allocated thread stopped");
+      }
+    });
+    allocatorThread.setName("YarnRMContainerAllocator");
+    allocatorThread.start();
+  }
+
+  public void stop() {
+    if(stopped.get()) {
+      return;
+    }
+    LOG.info("un-registering ApplicationMaster(QueryMaster):" + appAttemptId);
+    stopped.set(true);
+
+    try {
+      FinalApplicationStatus status = FinalApplicationStatus.UNDEFINED;
+      Query query = context.getQuery();
+      if (query != null) {
+        TajoProtos.QueryState state = query.getState();
+        if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+          status = FinalApplicationStatus.SUCCEEDED;
+        } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
+          status = FinalApplicationStatus.FAILED;
+        } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
+          status = FinalApplicationStatus.FAILED;
+        }
+      }
+      unregisterApplicationMaster(status, "tajo query finished", null);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    allocatorThread.interrupt();
+    LOG.info("un-registered ApplicationMAster(QueryMaster) stopped:" + appAttemptId);
+
+    super.stop();
+  }
+
+  private final Map<Priority, ExecutionBlockId> subQueryMap =
+      new HashMap<Priority, ExecutionBlockId>();
+
+  private AtomicLong prevReportTime = new AtomicLong(0);
+  private int reportInterval = 5 * 1000; // second
+
+  public void heartbeat() throws Exception {
+    AllocateResponse allocateResponse = allocate(context.getProgress());
+
+    List<Container> allocatedContainers = allocateResponse.getAllocatedContainers();
+
+    long currentTime = System.currentTimeMillis();
+    if ((currentTime - prevReportTime.longValue()) >= reportInterval) {
+      LOG.debug("Available Cluster Nodes: " + allocateResponse.getNumClusterNodes());
+      LOG.debug("Num of Allocated Containers: " + allocatedContainers.size());
+      LOG.info("Available Resource: " + allocateResponse.getAvailableResources());
+      prevReportTime.set(currentTime);
+    }
+
+    if (allocatedContainers.size() > 0) {
+      LOG.info("================================================================");
+      for (Container container : allocateResponse.getAllocatedContainers()) {
+        LOG.info("> Container Id: " + container.getId());
+        LOG.info("> Node Id: " + container.getNodeId());
+        LOG.info("> Resource (Mem): " + container.getResource().getMemory());
+        LOG.info("> Priority: " + container.getPriority());
+      }
+      LOG.info("================================================================");
+
+      Map<ExecutionBlockId, List<Container>> allocated = new HashMap<ExecutionBlockId, List<Container>>();
+      for (Container container : allocatedContainers) {
+        ExecutionBlockId executionBlockId = subQueryMap.get(container.getPriority());
+        SubQueryState state = context.getSubQuery(executionBlockId).getState();
+        if (!(SubQuery.isRunningState(state))) {
+          releaseAssignedContainer(container.getId());
+        } else {
+          if (allocated.containsKey(executionBlockId)) {
+            allocated.get(executionBlockId).add(container);
+          } else {
+            allocated.put(executionBlockId, Lists.newArrayList(container));
+          }
+        }
+      }
+
+      for (Entry<ExecutionBlockId, List<Container>> entry : allocated.entrySet()) {
+        eventHandler.handle(new SubQueryContainerAllocationEvent(entry.getKey(), entry.getValue()));
+      }
+    }
+  }
+
+  @Override
+  public void handle(ContainerAllocationEvent event) {
+
+    if (event.getType() == ContainerAllocatorEventType.CONTAINER_REQ) {
+      LOG.info(event);
+      subQueryMap.put(event.getPriority(), event.getExecutionBlockId());
+      addContainerRequest(new ContainerRequest(event.getCapability(), null, null,
+          event.getPriority()));
+
+    } else if (event.getType() == ContainerAllocatorEventType.CONTAINER_DEALLOCATE) {
+      LOG.info(event);
+    } else {
+      LOG.info(event);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
new file mode 100644
index 0000000..6d5268c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
@@ -0,0 +1,349 @@
+/**
+ * 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.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.YarnContainerProxy;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.util.ApplicationIdUtils;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedAction;
+import java.util.*;
+
+import static org.apache.tajo.ipc.TajoMasterProtocol.WorkerAllocatedResource;
+import static org.apache.tajo.ipc.TajoMasterProtocol.WorkerResourceAllocationResponse;
+
+public class YarnTajoResourceManager extends AbstractService implements WorkerResourceManager {
+  private static final Log LOG = LogFactory.getLog(YarnTajoResourceManager.class);
+
+  private YarnClient yarnClient;
+  private ApplicationMasterProtocol rmClient;
+  private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+  private Configuration conf;
+  private TajoMaster.MasterContext masterContext;
+
+  public YarnTajoResourceManager() {
+    super(YarnTajoResourceManager.class.getSimpleName());
+  }
+
+  public YarnTajoResourceManager(TajoMaster.MasterContext masterContext) {
+    super(YarnTajoResourceManager.class.getSimpleName());
+    this.masterContext = masterContext;
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  @Override
+  public Map<String, Worker> getWorkers() {
+    return new HashMap<String, Worker>();
+  }
+
+  @Override
+  public Map<String, Worker> getInactiveWorkers() {
+    return new HashMap<String, Worker>();
+  }
+
+  public Collection<String> getQueryMasters() {
+    return new ArrayList<String>();
+  }
+
+  public TajoMasterProtocol.ClusterResourceSummary getClusterResourceSummary() {
+    return TajoMasterProtocol.ClusterResourceSummary.newBuilder()
+        .setNumWorkers(0)
+        .setTotalCpuCoreSlots(0)
+        .setTotalDiskSlots(0)
+        .setTotalMemoryMB(0)
+        .setTotalAvailableCpuCoreSlots(0)
+        .setTotalAvailableDiskSlots(0)
+        .setTotalAvailableMemoryMB(0)
+        .build();
+  }
+
+  @Override
+  public void releaseWorkerResource(YarnProtos.ContainerIdProto containerId) {
+    throw new UnimplementedException("releaseWorkerResource");
+  }
+
+  @Override
+  public WorkerAllocatedResource allocateQueryMaster(QueryInProgress queryInProgress) {
+    throw new UnimplementedException("allocateQueryMaster");
+  }
+
+  @Override
+  public void allocateWorkerResources(
+      TajoMasterProtocol.WorkerResourceAllocationRequest request,
+      RpcCallback<WorkerResourceAllocationResponse> rpcCallBack) {
+    throw new UnimplementedException("allocateWorkerResources");
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.conf = conf;
+    connectYarnClient();
+
+    final YarnConfiguration yarnConf = new YarnConfiguration(conf);
+    final YarnRPC rpc = YarnRPC.create(conf);
+    final InetSocketAddress rmAddress = conf.getSocketAddr(
+        YarnConfiguration.RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
+
+    UserGroupInformation currentUser;
+    try {
+      currentUser = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new YarnRuntimeException(e);
+    }
+
+    rmClient = currentUser.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
+      @Override
+      public ApplicationMasterProtocol run() {
+        return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rmAddress, yarnConf);
+      }
+    });
+  }
+
+  @Override
+  public String getSeedQueryId() throws IOException {
+    try {
+      YarnClientApplication app = yarnClient.createApplication();
+      return app.getApplicationSubmissionContext().getApplicationId().toString();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+
+      throw new IOException(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void stopQueryMaster(QueryId queryId) {
+    try {
+      FinalApplicationStatus appStatus = FinalApplicationStatus.UNDEFINED;
+      QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
+      if(queryInProgress == null) {
+        return;
+      }
+      TajoProtos.QueryState state = queryInProgress.getQueryInfo().getQueryState();
+      if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+        appStatus = FinalApplicationStatus.SUCCEEDED;
+      } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
+        appStatus = FinalApplicationStatus.FAILED;
+      } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
+        appStatus = FinalApplicationStatus.FAILED;
+      }
+      FinishApplicationMasterRequest request = recordFactory
+          .newRecordInstance(FinishApplicationMasterRequest.class);
+      request.setFinalApplicationStatus(appStatus);
+      request.setDiagnostics("QueryMaster shutdown by TajoMaster.");
+      rmClient.finishApplicationMaster(request);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  private void connectYarnClient() {
+    this.yarnClient = new YarnClientImpl();
+    this.yarnClient.init(conf);
+    this.yarnClient.start();
+  }
+
+  private ApplicationAttemptId allocateAndLaunchQueryMaster(QueryInProgress queryInProgress) throws IOException, YarnException {
+    QueryId queryId = queryInProgress.getQueryId();
+    ApplicationId appId = ApplicationIdUtils.queryIdToAppId(queryId);
+
+    LOG.info("Allocate and launch ApplicationMaster for QueryMaster: queryId=" +
+        queryId + ", appId=" + appId);
+
+    ApplicationSubmissionContext appContext = Records.newRecord(ApplicationSubmissionContext.class);
+
+    // set the application id
+    appContext.setApplicationId(appId);
+    // set the application name
+    appContext.setApplicationName("Tajo");
+
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(5);
+    appContext.setPriority(pri);
+
+    // Set the queue to which this application is to be submitted in the RM
+    appContext.setQueue("default");
+
+    ContainerLaunchContext commonContainerLaunchContext =
+        YarnContainerProxy.createCommonContainerLaunchContext(masterContext.getConf(), queryId.toString(), true);
+
+    // Setup environment by cloning from common env.
+    Map<String, String> env = commonContainerLaunchContext.getEnvironment();
+    Map<String, String> myEnv = new HashMap<String, String>(env.size());
+    myEnv.putAll(env);
+
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the local resources
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the necessary command to execute the application master
+    Vector<CharSequence> vargs = new Vector<CharSequence>(30);
+
+    // Set java executable command
+    //LOG.info("Setting up app master command");
+    vargs.add("${JAVA_HOME}" + "/bin/java");
+    // Set Xmx based on am memory size
+    String jvmOptions = masterContext.getConf().get("tajo.rm.yarn.querymaster.jvm.option", "-Xmx2000m");
+
+    for(String eachToken: jvmOptions.split((" "))) {
+      vargs.add(eachToken);
+    }
+    // Set Remote Debugging
+    //if (!context.getQuery().getSubQuery(event.getExecutionBlockId()).isLeafQuery()) {
+    //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
+    //}
+    // Set class name
+    vargs.add(TajoWorker.class.getCanonicalName());
+    vargs.add("qm");
+    vargs.add(queryId.toString()); // queryId
+    vargs.add(masterContext.getTajoMasterService().getBindAddress().getHostName() + ":" +
+        masterContext.getTajoMasterService().getBindAddress().getPort());
+
+    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+    // Get final commmand
+    StringBuilder command = new StringBuilder();
+    for (CharSequence str : vargs) {
+      command.append(str).append(" ");
+    }
+
+    LOG.info("Completed setting up QueryMasterRunner command " + command.toString());
+    List<String> commands = new ArrayList<String>();
+    commands.add(command.toString());
+
+    final Resource resource = Records.newRecord(Resource.class);
+    // TODO - get default value from conf
+    resource.setMemory(2000);
+    resource.setVirtualCores(1);
+
+    Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
+
+    ContainerLaunchContext masterContainerContext = BuilderUtils.newContainerLaunchContext(
+        commonContainerLaunchContext.getLocalResources(),
+        myEnv,
+        commands,
+        myServiceData,
+        null,
+        new HashMap<ApplicationAccessType, String>(2)
+    );
+
+    appContext.setAMContainerSpec(masterContainerContext);
+
+    LOG.info("Submitting QueryMaster to ResourceManager");
+    yarnClient.submitApplication(appContext);
+
+    ApplicationReport appReport = monitorApplication(appId, EnumSet.of(YarnApplicationState.ACCEPTED));
+    ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
+
+    LOG.info("Launching QueryMaster with appAttemptId: " + attemptId);
+
+    return attemptId;
+  }
+
+  private ApplicationReport monitorApplication(ApplicationId appId,
+                                               Set<YarnApplicationState> finalState) throws IOException, YarnException {
+
+    long sleepTime = 100;
+    int count = 1;
+    while (true) {
+      // Get application report for the appId we are interested in
+      ApplicationReport report = yarnClient.getApplicationReport(appId);
+
+      LOG.info("Got application report from ASM for" + ", appId="
+          + appId.getId() + ", appAttemptId="
+          + report.getCurrentApplicationAttemptId() + ", clientToken="
+          + report.getClientToAMToken() + ", appDiagnostics="
+          + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
+          + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
+          + report.getRpcPort() + ", appStartTime=" + report.getStartTime()
+          + ", yarnAppState=" + report.getYarnApplicationState().toString()
+          + ", distributedFinalState="
+          + report.getFinalApplicationStatus().toString() + ", appTrackingUrl="
+          + report.getTrackingUrl() + ", appUser=" + report.getUser());
+
+      YarnApplicationState state = report.getYarnApplicationState();
+      if (finalState.contains(state)) {
+        return report;
+      }
+      try {
+        Thread.sleep(sleepTime);
+        sleepTime = count * 100;
+        if(count < 10) {
+          count++;
+        }
+      } catch (InterruptedException e) {
+        //LOG.debug("Thread sleep in monitoring loop interrupted");
+      }
+    }
+  }
+
+  public boolean isQueryMasterStopped(QueryId queryId) {
+    ApplicationId appId = ApplicationIdUtils.queryIdToAppId(queryId);
+    try {
+      ApplicationReport report = yarnClient.getApplicationReport(appId);
+      YarnApplicationState state = report.getYarnApplicationState();
+      return EnumSet.of(
+          YarnApplicationState.FINISHED,
+          YarnApplicationState.KILLED,
+          YarnApplicationState.FAILED).contains(state);
+    } catch (YarnException e) {
+      LOG.error(e.getMessage(), e);
+      return false;
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java b/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java
new file mode 100644
index 0000000..3f48ca5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java
@@ -0,0 +1,25 @@
+/**
+ * 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.master.session;
+
+public class InvalidSessionException extends Exception {
+  public InvalidSessionException(String sessionId) {
+    super("Invalid session id \"" + sessionId + "\"");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java b/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java
new file mode 100644
index 0000000..686d860
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java
@@ -0,0 +1,25 @@
+/**
+ * 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.master.session;
+
+public class NoSuchSessionVariableException extends Exception {
+  public NoSuchSessionVariableException(String varname) {
+    super("No such session variable \"" + varname + "\"");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java b/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
new file mode 100644
index 0000000..4d244bf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
@@ -0,0 +1,124 @@
+/**
+ * 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.master.session;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.common.ProtoObject;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.SessionProto;
+
+public class Session implements SessionConstants, ProtoObject<SessionProto> {
+  private final String sessionId;
+  private final String userName;
+  private final Map<String, String> sessionVariables;
+
+  // transient status
+  private volatile long lastAccessTime;
+  private volatile String currentDatabase;
+
+  public Session(String sessionId, String userName, String databaseName) {
+    this.sessionId = sessionId;
+    this.userName = userName;
+    this.lastAccessTime = System.currentTimeMillis();
+    this.sessionVariables = new HashMap<String, String>();
+    selectDatabase(databaseName);
+  }
+
+  public Session(SessionProto proto) {
+    sessionId = proto.getSessionId();
+    userName = proto.getUsername();
+    currentDatabase = proto.getCurrentDatabase();
+    lastAccessTime = proto.getLastAccessTime();
+    Options options = new Options(proto.getVariables());
+    sessionVariables = options.getAllKeyValus();
+  }
+
+  public String getSessionId() {
+    return sessionId;
+  }
+
+  public String getUserName() {
+    return userName;
+  }
+
+  public void updateLastAccessTime() {
+    lastAccessTime = System.currentTimeMillis();
+  }
+
+  public long getLastAccessTime() {
+    return lastAccessTime;
+  }
+
+  public void setVariable(String name, String value) {
+    synchronized (sessionVariables) {
+      sessionVariables.put(name, value);
+    }
+  }
+
+  public String getVariable(String name) throws NoSuchSessionVariableException {
+    synchronized (sessionVariables) {
+      if (sessionVariables.containsKey(name)) {
+        return sessionVariables.get(name);
+      } else {
+        throw new NoSuchSessionVariableException(name);
+      }
+    }
+  }
+
+  public void removeVariable(String name) {
+    synchronized (sessionVariables) {
+      sessionVariables.remove(name);
+    }
+  }
+
+  public synchronized Map<String, String> getAllVariables() {
+    synchronized (sessionVariables) {
+      return ImmutableMap.copyOf(sessionVariables);
+    }
+  }
+
+  public void selectDatabase(String databaseName) {
+    this.currentDatabase = databaseName;
+  }
+
+  public String getCurrentDatabase() {
+    return this.currentDatabase;
+  }
+
+  @Override
+  public SessionProto getProto() {
+    SessionProto.Builder builder = SessionProto.newBuilder();
+    builder.setSessionId(sessionId);
+    builder.setUsername(userName);
+    builder.setCurrentDatabase(currentDatabase);
+    builder.setLastAccessTime(lastAccessTime);
+    Options variables = new Options();
+    variables.putAll(this.sessionVariables);
+    builder.setVariables(variables.getProto());
+    return builder.build();
+  }
+
+  public String toString() {
+    return "user=" + userName + ",id=" + sessionId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java
new file mode 100644
index 0000000..46f49a2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java
@@ -0,0 +1,23 @@
+/**
+ * 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.master.session;
+
+public interface SessionConstants {
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java
new file mode 100644
index 0000000..dce3ba6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java
@@ -0,0 +1,34 @@
+/**
+ * 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.master.session;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+public class SessionEvent extends AbstractEvent<SessionEventType> {
+  private final String sessionId;
+
+  public SessionEvent(String sessionId, SessionEventType sessionEventType) {
+    super(sessionEventType);
+    this.sessionId = sessionId;
+  }
+
+  public String getSessionId() {
+    return sessionId;
+  }
+}


[18/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
new file mode 100644
index 0000000..79f14a2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
@@ -0,0 +1,339 @@
+/**
+ * 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.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.ContainerProxy;
+import org.apache.tajo.master.TajoContainerProxy;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
+import org.apache.tajo.master.TaskRunnerLauncher;
+import org.apache.tajo.master.event.ContainerAllocationEvent;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.SubQueryState;
+import org.apache.tajo.master.rm.TajoWorkerContainer;
+import org.apache.tajo.master.rm.TajoWorkerContainerId;
+import org.apache.tajo.master.rm.Worker;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.util.ApplicationIdUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TajoResourceAllocator extends AbstractResourceAllocator {
+  private static final Log LOG = LogFactory.getLog(TajoResourceAllocator.class);
+
+  private TajoConf tajoConf;
+  private QueryMasterTask.QueryMasterTaskContext queryTaskContext;
+  private final ExecutorService executorService;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  public TajoResourceAllocator(QueryMasterTask.QueryMasterTaskContext queryTaskContext) {
+    this.queryTaskContext = queryTaskContext;
+    executorService = Executors.newFixedThreadPool(
+        queryTaskContext.getConf().getIntVar(TajoConf.ConfVars.YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
+  }
+
+  @Override
+  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerIdProto) {
+    TajoWorkerContainerId containerId = new TajoWorkerContainerId();
+    ApplicationAttemptId appAttemptId = new ApplicationAttemptIdPBImpl(containerIdProto.getAppAttemptId());
+    containerId.setApplicationAttemptId(appAttemptId);
+    containerId.setId(containerIdProto.getId());
+    return containerId;
+  }
+
+  @Override
+  public void allocateTaskWorker() {
+  }
+
+  @Override
+  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext,
+                                           int numTasks,
+                                           int memoryMBPerTask) {
+    //TODO consider disk slot
+
+    TajoMasterProtocol.ClusterResourceSummary clusterResource = workerContext.getClusterResource();
+    int clusterSlots = clusterResource == null ? 0 : clusterResource.getTotalMemoryMB() / memoryMBPerTask;
+    clusterSlots =  Math.max(1, clusterSlots - 1); // reserve query master slot
+    LOG.info("CalculateNumberRequestContainer - Number of Tasks=" + numTasks +
+        ", Number of Cluster Slots=" + clusterSlots);
+    return  Math.min(numTasks, clusterSlots);
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    tajoConf = (TajoConf)conf;
+
+    queryTaskContext.getDispatcher().register(TaskRunnerGroupEvent.EventType.class, new TajoTaskRunnerLauncher());
+
+    queryTaskContext.getDispatcher().register(ContainerAllocatorEventType.class, new TajoWorkerAllocationHandler());
+
+    super.init(conf);
+  }
+
+  @Override
+  public synchronized void stop() {
+    if (stopped.getAndSet(true)) {
+      return;
+    }
+
+    executorService.shutdownNow();
+
+    Map<ContainerId, ContainerProxy> containers = queryTaskContext.getResourceAllocator().getContainers();
+    List<ContainerProxy> list = new ArrayList<ContainerProxy>(containers.values());
+    for(ContainerProxy eachProxy: list) {
+      try {
+        eachProxy.stopContainer();
+      } catch (Exception e) {
+        LOG.warn(e.getMessage());
+      }
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  class TajoTaskRunnerLauncher implements TaskRunnerLauncher {
+    @Override
+    public void handle(TaskRunnerGroupEvent event) {
+      if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_LAUNCH) {
+        launchTaskRunners(event.getExecutionBlockId(), event.getContainers());
+      } else if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_CLEANUP) {
+        stopContainers(event.getContainers());
+      }
+    }
+  }
+
+  private void launchTaskRunners(ExecutionBlockId executionBlockId, Collection<Container> containers) {
+    // Query in standby mode doesn't need launch Worker.
+    // But, Assign ExecutionBlock to assigned tajo worker
+    for(Container eachContainer: containers) {
+      TajoContainerProxy containerProxy = new TajoContainerProxy(queryTaskContext, tajoConf,
+          eachContainer, executionBlockId);
+      executorService.submit(new LaunchRunner(eachContainer.getId(), containerProxy));
+    }
+  }
+
+  protected static class LaunchRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public LaunchRunner(ContainerId id, ContainerProxy proxy) {
+      this.proxy = proxy;
+      this.id = id;
+    }
+    @Override
+    public void run() {
+      proxy.launch(null);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("ContainerProxy started:" + id);
+      }
+    }
+  }
+
+  private void stopContainers(Collection<Container> containers) {
+    for (Container container : containers) {
+      final ContainerProxy proxy = queryTaskContext.getResourceAllocator().getContainer(container.getId());
+      executorService.submit(new StopContainerRunner(container.getId(), proxy));
+    }
+  }
+
+  private static class StopContainerRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public StopContainerRunner(ContainerId id, ContainerProxy proxy) {
+      this.id = id;
+      this.proxy = proxy;
+    }
+
+    @Override
+    public void run() {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("ContainerProxy stopped:" + id + "," + proxy.getId());
+      }
+      proxy.stopContainer();
+    }
+  }
+
+  class TajoWorkerAllocationHandler implements EventHandler<ContainerAllocationEvent> {
+    @Override
+    public void handle(ContainerAllocationEvent event) {
+      executorService.submit(new TajoWorkerAllocationThread(event));
+    }
+  }
+
+  class TajoWorkerAllocationThread extends Thread {
+    ContainerAllocationEvent event;
+    TajoWorkerAllocationThread(ContainerAllocationEvent event) {
+      this.event = event;
+    }
+
+    @Override
+    public void run() {
+      LOG.info("Start TajoWorkerAllocationThread");
+      CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack =
+          new CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse>();
+
+      //TODO consider task's resource usage pattern
+      int requiredMemoryMB = tajoConf.getIntVar(TajoConf.ConfVars.TASK_DEFAULT_MEMORY);
+      float requiredDiskSlots = tajoConf.getFloatVar(TajoConf.ConfVars.TASK_DEFAULT_DISK);
+
+      TajoMasterProtocol.WorkerResourceAllocationRequest request =
+          TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
+              .setMinMemoryMBPerContainer(requiredMemoryMB)
+              .setMaxMemoryMBPerContainer(requiredMemoryMB)
+              .setNumContainers(event.getRequiredNum())
+              .setResourceRequestPriority(!event.isLeafQuery() ? TajoMasterProtocol.ResourceRequestPriority.MEMORY
+                  : TajoMasterProtocol.ResourceRequestPriority.DISK)
+              .setMinDiskSlotPerContainer(requiredDiskSlots)
+              .setMaxDiskSlotPerContainer(requiredDiskSlots)
+              .setQueryId(event.getExecutionBlockId().getQueryId().getProto())
+              .build();
+
+      RpcConnectionPool connPool = RpcConnectionPool.getPool(queryTaskContext.getConf());
+      NettyClientBase tmClient = null;
+      try {
+        tmClient = connPool.getConnection(
+            queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
+            TajoMasterProtocol.class, true);
+        TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+        masterClientService.allocateWorkerResources(null, request, callBack);
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+      } finally {
+        connPool.releaseConnection(tmClient);
+      }
+
+      TajoMasterProtocol.WorkerResourceAllocationResponse response = null;
+      while(!stopped.get()) {
+        try {
+          response = callBack.get(3, TimeUnit.SECONDS);
+          break;
+        } catch (InterruptedException e) {
+          if(stopped.get()) {
+            return;
+          }
+        } catch (TimeoutException e) {
+          LOG.info("No available worker resource for " + event.getExecutionBlockId());
+          continue;
+        }
+      }
+      int numAllocatedContainers = 0;
+
+      if(response != null) {
+        List<TajoMasterProtocol.WorkerAllocatedResource> allocatedResources = response.getWorkerAllocatedResourceList();
+        ExecutionBlockId executionBlockId = event.getExecutionBlockId();
+
+        List<Container> containers = new ArrayList<Container>();
+        for(TajoMasterProtocol.WorkerAllocatedResource eachAllocatedResource: allocatedResources) {
+          TajoWorkerContainer container = new TajoWorkerContainer();
+          NodeId nodeId = NodeId.newInstance(eachAllocatedResource.getWorkerHost(),
+              eachAllocatedResource.getPeerRpcPort());
+
+          TajoWorkerContainerId containerId = new TajoWorkerContainerId();
+
+          containerId.setApplicationAttemptId(
+              ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId(),
+                  eachAllocatedResource.getContainerId().getAppAttemptId().getAttemptId()));
+          containerId.setId(eachAllocatedResource.getContainerId().getId());
+
+          container.setId(containerId);
+          container.setNodeId(nodeId);
+
+
+          WorkerResource workerResource = new WorkerResource();
+          workerResource.setMemoryMB(eachAllocatedResource.getAllocatedMemoryMB());
+          workerResource.setDiskSlots(eachAllocatedResource.getAllocatedDiskSlots());
+
+          Worker worker = new Worker(null, workerResource);
+          worker.setHostName(nodeId.getHost());
+          worker.setPeerRpcPort(nodeId.getPort());
+          worker.setQueryMasterPort(eachAllocatedResource.getQueryMasterPort());
+          worker.setPullServerPort(eachAllocatedResource.getWorkerPullServerPort());
+
+          container.setWorkerResource(worker);
+
+          containers.add(container);
+        }
+
+        SubQueryState state = queryTaskContext.getSubQuery(executionBlockId).getState();
+        if (!SubQuery.isRunningState(state)) {
+          try {
+            List<ContainerId> containerIds = new ArrayList<ContainerId>();
+            for(Container eachContainer: containers) {
+              containerIds.add(eachContainer.getId());
+            }
+            TajoContainerProxy.releaseWorkerResource(queryTaskContext, executionBlockId, containerIds);
+          } catch (Exception e) {
+            LOG.error(e.getMessage(), e);
+          }
+          return;
+        }
+
+        if (allocatedResources.size() > 0) {
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("SubQueryContainerAllocationEvent fire:" + executionBlockId);
+          }
+          queryTaskContext.getEventHandler().handle(new SubQueryContainerAllocationEvent(executionBlockId, containers));
+        }
+        numAllocatedContainers += allocatedResources.size();
+
+      }
+      if(event.getRequiredNum() > numAllocatedContainers) {
+        ContainerAllocationEvent shortRequestEvent = new ContainerAllocationEvent(
+            event.getType(), event.getExecutionBlockId(), event.getPriority(),
+            event.getResource(),
+            event.getRequiredNum() - numAllocatedContainers,
+            event.isLeafQuery(), event.getProgress()
+        );
+        queryTaskContext.getEventHandler().handle(shortRequestEvent);
+
+      }
+      LOG.info("Stop TajoWorkerAllocationThread");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
new file mode 100644
index 0000000..0b8d6c2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -0,0 +1,583 @@
+/**
+ * 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.worker;
+
+import com.codahale.metrics.Gauge;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathData;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.catalog.CatalogClient;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryMaster;
+import org.apache.tajo.master.querymaster.QueryMasterManagerService;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.pullserver.TajoPullServerService;
+import org.apache.tajo.rpc.RpcChannelFactory;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.util.metrics.TajoSystemMetrics;
+import org.apache.tajo.webapp.StaticHttpServer;
+
+import java.io.*;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+
+public class TajoWorker extends CompositeService {
+  public static final PrimitiveProtos.BoolProto TRUE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
+  public static final PrimitiveProtos.BoolProto FALSE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
+  public static final String WORKER_MODE_YARN_TASKRUNNER = "tr";
+  public static final String WORKER_MODE_YARN_QUERYMASTER = "qm";
+  public static final String WORKER_MODE_STANDBY = "standby";
+  public static final String WORKER_MODE_QUERY_MASTER = "standby-qm";
+  public static final String WORKER_MODE_TASKRUNNER = "standby-tr";
+
+  private static final Log LOG = LogFactory.getLog(TajoWorker.class);
+
+  private TajoConf systemConf;
+
+  private StaticHttpServer webServer;
+
+  private TajoWorkerClientService tajoWorkerClientService;
+
+  private QueryMasterManagerService queryMasterManagerService;
+
+  private TajoWorkerManagerService tajoWorkerManagerService;
+
+  private InetSocketAddress tajoMasterAddress;
+
+  private InetSocketAddress workerResourceTrackerAddr;
+
+  private CatalogClient catalogClient;
+
+  private WorkerContext workerContext;
+
+  private TaskRunnerManager taskRunnerManager;
+
+  private TajoPullServerService pullService;
+
+  private boolean yarnContainerMode;
+
+  private boolean queryMasterMode;
+
+  private boolean taskRunnerMode;
+
+  private WorkerHeartbeatService workerHeartbeatThread;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private AtomicInteger numClusterNodes = new AtomicInteger();
+
+  private TajoMasterProtocol.ClusterResourceSummary clusterResource;
+
+  private int httpPort;
+
+  private ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+
+  private RpcConnectionPool connPool;
+
+  private String[] cmdArgs;
+
+  private DeletionService deletionService;
+
+  private TajoSystemMetrics workerSystemMetrics;
+
+  public TajoWorker() throws Exception {
+    super(TajoWorker.class.getName());
+  }
+
+  public void startWorker(TajoConf systemConf, String[] args) {
+    this.systemConf = systemConf;
+    this.cmdArgs = args;
+    setWorkerMode(args);
+    init(systemConf);
+    start();
+  }
+
+  private void setWorkerMode(String[] args) {
+    if(args.length < 1) {
+      queryMasterMode = systemConf.getBoolean("tajo.worker.mode.querymaster", true);
+      taskRunnerMode = systemConf.getBoolean("tajo.worker.mode.taskrunner", true);
+    } else {
+      if(WORKER_MODE_STANDBY.equals(args[0])) {
+        queryMasterMode = true;
+        taskRunnerMode = true;
+      } else if(WORKER_MODE_YARN_TASKRUNNER.equals(args[0])) {
+        yarnContainerMode = true;
+        queryMasterMode = true;
+      } else if(WORKER_MODE_YARN_QUERYMASTER.equals(args[0])) {
+        yarnContainerMode = true;
+        taskRunnerMode = true;
+      } else if(WORKER_MODE_QUERY_MASTER.equals(args[0])) {
+        yarnContainerMode = false;
+        queryMasterMode = true;
+      } else {
+        yarnContainerMode = false;
+        taskRunnerMode = true;
+      }
+    }
+    if(!queryMasterMode && !taskRunnerMode) {
+      LOG.fatal("Worker daemon exit cause no worker mode(querymaster/taskrunner) property");
+      System.exit(0);
+    }
+  }
+  
+  @Override
+  public void init(Configuration conf) {
+    Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
+
+    this.systemConf = (TajoConf)conf;
+    RackResolver.init(systemConf);
+
+    this.connPool = RpcConnectionPool.getPool(systemConf);
+    this.workerContext = new WorkerContext();
+
+    String resourceManagerClassName = systemConf.getVar(ConfVars.RESOURCE_MANAGER_CLASS);
+
+    boolean randomPort = true;
+    if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+      randomPort = false;
+    }
+    int clientPort = systemConf.getSocketAddrVar(ConfVars.WORKER_CLIENT_RPC_ADDRESS).getPort();
+    int peerRpcPort = systemConf.getSocketAddrVar(ConfVars.WORKER_PEER_RPC_ADDRESS).getPort();
+    int qmManagerPort = systemConf.getSocketAddrVar(ConfVars.WORKER_QM_RPC_ADDRESS).getPort();
+
+    if(randomPort) {
+      clientPort = 0;
+      peerRpcPort = 0;
+      qmManagerPort = 0;
+      systemConf.setIntVar(ConfVars.PULLSERVER_PORT, 0);
+    }
+
+    // querymaster worker
+    tajoWorkerClientService = new TajoWorkerClientService(workerContext, clientPort);
+    addService(tajoWorkerClientService);
+
+    queryMasterManagerService = new QueryMasterManagerService(workerContext, qmManagerPort);
+    addService(queryMasterManagerService);
+
+    // taskrunner worker
+    taskRunnerManager = new TaskRunnerManager(workerContext);
+    addService(taskRunnerManager);
+
+    tajoWorkerManagerService = new TajoWorkerManagerService(workerContext, peerRpcPort);
+    addService(tajoWorkerManagerService);
+
+    if(!yarnContainerMode) {
+      if(taskRunnerMode) {
+        pullService = new TajoPullServerService();
+        addService(pullService);
+      }
+
+      if (!systemConf.get(CommonTestingUtil.TAJO_TEST, "FALSE").equalsIgnoreCase("TRUE")) {
+        try {
+          httpPort = systemConf.getSocketAddrVar(ConfVars.WORKER_INFO_ADDRESS).getPort();
+          if(queryMasterMode && !taskRunnerMode) {
+            //If QueryMaster and TaskRunner run on single host, http port conflicts
+            httpPort = systemConf.getSocketAddrVar(ConfVars.WORKER_QM_INFO_ADDRESS).getPort();
+          }
+          webServer = StaticHttpServer.getInstance(this ,"worker", null, httpPort ,
+              true, null, systemConf, null);
+          webServer.start();
+          httpPort = webServer.getPort();
+          LOG.info("Worker info server started:" + httpPort);
+
+          deletionService = new DeletionService(getMountPath().size(), 0);
+          if(systemConf.getBoolVar(ConfVars.WORKER_TEMPORAL_DIR_CLEANUP)){
+            getWorkerContext().cleanupTemporalDirectories();
+          }
+        } catch (IOException e) {
+          LOG.error(e.getMessage(), e);
+        }
+      }
+    }
+
+    LOG.info("Tajo Worker started: queryMaster=" + queryMasterMode + " taskRunner=" + taskRunnerMode +
+        ", qmRpcPort=" + qmManagerPort +
+        ",yarnContainer=" + yarnContainerMode + ", clientPort=" + clientPort +
+        ", peerRpcPort=" + peerRpcPort + ":" + qmManagerPort + ",httpPort" + httpPort);
+
+    super.init(conf);
+
+    if(yarnContainerMode && queryMasterMode) {
+      tajoMasterAddress = NetUtils.createSocketAddr(cmdArgs[2]);
+      connectToCatalog();
+
+      QueryId queryId = TajoIdUtils.parseQueryId(cmdArgs[1]);
+      queryMasterManagerService.getQueryMaster().reportQueryStatusToQueryMaster(
+          queryId, TajoProtos.QueryState.QUERY_MASTER_LAUNCHED);
+    } else if(yarnContainerMode && taskRunnerMode) { //TaskRunner mode
+      taskRunnerManager.startTask(cmdArgs);
+    } else {
+      tajoMasterAddress = NetUtils.createSocketAddr(systemConf.getVar(ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS));
+      workerResourceTrackerAddr = NetUtils.createSocketAddr(systemConf.getVar(ConfVars.RESOURCE_TRACKER_RPC_ADDRESS));
+      connectToCatalog();
+    }
+
+    workerHeartbeatThread = new WorkerHeartbeatService(workerContext);
+    workerHeartbeatThread.init(conf);
+    addIfService(workerHeartbeatThread);
+  }
+
+  private void initWorkerMetrics() {
+    workerSystemMetrics = new TajoSystemMetrics(systemConf, "worker", workerContext.getWorkerName());
+    workerSystemMetrics.start();
+
+    workerSystemMetrics.register("querymaster", "runningQueries", new Gauge<Integer>() {
+      @Override
+      public Integer getValue() {
+        if(queryMasterManagerService != null) {
+          return queryMasterManagerService.getQueryMaster().getQueryMasterTasks().size();
+        } else {
+          return 0;
+        }
+      }
+    });
+
+    workerSystemMetrics.register("task", "runningTasks", new Gauge<Integer>() {
+      @Override
+      public Integer getValue() {
+        if(taskRunnerManager != null) {
+          return taskRunnerManager.getNumTasks();
+        } else {
+          return 0;
+        }
+      }
+    });
+  }
+
+  public WorkerContext getWorkerContext() {
+    return workerContext;
+  }
+
+  @Override
+  public void start() {
+    super.start();
+    initWorkerMetrics();
+  }
+
+  @Override
+  public void stop() {
+    if(stopped.getAndSet(true)) {
+      return;
+    }
+
+    if(webServer != null) {
+      try {
+        webServer.stop();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+
+    if (catalogClient != null) {
+      catalogClient.close();
+    }
+
+    if(connPool != null) {
+      connPool.shutdown();
+      RpcChannelFactory.shutdown();
+    }
+
+    if(webServer != null && webServer.isAlive()) {
+      try {
+        webServer.stop();
+      } catch (Exception e) {
+      }
+    }
+
+    if(workerSystemMetrics != null) {
+      workerSystemMetrics.stop();
+    }
+
+    if(deletionService != null) deletionService.stop();
+    super.stop();
+    LOG.info("TajoWorker main thread exiting");
+  }
+
+  public class WorkerContext {
+    public QueryMaster getQueryMaster() {
+      if(queryMasterManagerService == null) {
+        return null;
+      }
+      return queryMasterManagerService.getQueryMaster();
+    }
+
+    public TajoWorkerManagerService getTajoWorkerManagerService() {
+      return tajoWorkerManagerService;
+    }
+
+    public QueryMasterManagerService getQueryMasterManagerService() {
+      return queryMasterManagerService;
+    }
+
+    public TajoWorkerClientService getTajoWorkerClientService() {
+      return tajoWorkerClientService;
+    }
+
+    public TaskRunnerManager getTaskRunnerManager() {
+      return taskRunnerManager;
+    }
+
+    public CatalogService getCatalog() {
+      return catalogClient;
+    }
+
+    public TajoPullServerService getPullService() {
+      return pullService;
+    }
+
+    public int getHttpPort() {
+      return httpPort;
+    }
+
+    public String getWorkerName() {
+      if(queryMasterMode) {
+        return getQueryMasterManagerService().getHostAndPort();
+      } else {
+        return getTajoWorkerManagerService().getHostAndPort();
+      }
+    }
+    public void stopWorker(boolean force) {
+      stop();
+      if(force) {
+        System.exit(0);
+      }
+    }
+
+    protected void cleanup(String strPath) {
+      if(deletionService == null) return;
+
+      LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
+
+      try {
+        Iterable<Path> iter = lDirAllocator.getAllLocalPathsToRead(strPath, systemConf);
+        FileSystem localFS = FileSystem.getLocal(systemConf);
+        for (Path path : iter){
+          deletionService.delete(localFS.makeQualified(path));
+        }
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+
+    protected void cleanupTemporalDirectories() {
+      if(deletionService == null) return;
+
+      LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
+
+      try {
+        Iterable<Path> iter = lDirAllocator.getAllLocalPathsToRead(".", systemConf);
+        FileSystem localFS = FileSystem.getLocal(systemConf);
+        for (Path path : iter){
+          PathData[] items = PathData.expandAsGlob(localFS.makeQualified(new Path(path, "*")).toString(), systemConf);
+
+          ArrayList<Path> paths = new ArrayList<Path>();
+          for (PathData pd : items){
+            paths.add(pd.path);
+          }
+          if(paths.size() == 0) continue;
+
+          deletionService.delete(null, paths.toArray(new Path[paths.size()]));
+        }
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+
+    public boolean isYarnContainerMode() {
+      return yarnContainerMode;
+    }
+
+    public void setNumClusterNodes(int numClusterNodes) {
+      TajoWorker.this.numClusterNodes.set(numClusterNodes);
+    }
+
+    public int getNumClusterNodes() {
+      return TajoWorker.this.numClusterNodes.get();
+    }
+
+    public void setClusterResource(TajoMasterProtocol.ClusterResourceSummary clusterResource) {
+      synchronized(numClusterNodes) {
+        TajoWorker.this.clusterResource = clusterResource;
+      }
+    }
+
+    public TajoMasterProtocol.ClusterResourceSummary getClusterResource() {
+      synchronized(numClusterNodes) {
+        return TajoWorker.this.clusterResource;
+      }
+    }
+
+    public InetSocketAddress getTajoMasterAddress() {
+      return tajoMasterAddress;
+    }
+
+    public InetSocketAddress getResourceTrackerAddress() {
+      return workerResourceTrackerAddr;
+    }
+
+    public int getPeerRpcPort() {
+      return getTajoWorkerManagerService() == null ? 0 : getTajoWorkerManagerService().getBindAddr().getPort();
+    }
+
+    public boolean isQueryMasterMode() {
+      return queryMasterMode;
+    }
+
+    public boolean isTaskRunnerMode() {
+      return taskRunnerMode;
+    }
+
+    public TajoSystemMetrics getWorkerSystemMetrics() {
+      return workerSystemMetrics;
+    }
+  }
+
+  public void stopWorkerForce() {
+    stop();
+  }
+
+  private void connectToCatalog() {
+    try {
+      catalogClient = new CatalogClient(systemConf);
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  private class ShutdownHook implements Runnable {
+    @Override
+    public void run() {
+      if(!stopped.get()) {
+        LOG.info("============================================");
+        LOG.info("TajoWorker received SIGINT Signal");
+        LOG.info("============================================");
+        stop();
+      }
+    }
+  }
+
+  String getThreadTaskName(long id, String name) {
+    if (name == null) {
+      return Long.toString(id);
+    }
+    return id + " (" + name + ")";
+  }
+
+  public void dumpThread(Writer writer) {
+    PrintWriter stream = new PrintWriter(writer);
+    int STACK_DEPTH = 20;
+    boolean contention = threadBean.isThreadContentionMonitoringEnabled();
+    long[] threadIds = threadBean.getAllThreadIds();
+    stream.println("Process Thread Dump: Tajo Worker");
+    stream.println(threadIds.length + " active threads");
+    for (long tid : threadIds) {
+      ThreadInfo info = threadBean.getThreadInfo(tid, STACK_DEPTH);
+      if (info == null) {
+        stream.println("  Inactive");
+        continue;
+      }
+      stream.println("Thread " + getThreadTaskName(info.getThreadId(), info.getThreadName()) + ":");
+      Thread.State state = info.getThreadState();
+      stream.println("  State: " + state + ",  Blocked count: " + info.getBlockedCount() +
+          ",  Waited count: " + info.getWaitedCount());
+      if (contention) {
+        stream.println("  Blocked time: " + info.getBlockedTime() + ",  Waited time: " + info.getWaitedTime());
+      }
+      if (state == Thread.State.WAITING) {
+        stream.println("  Waiting on " + info.getLockName());
+      } else if (state == Thread.State.BLOCKED) {
+        stream.println("  Blocked on " + info.getLockName() +
+            ", Blocked by " + getThreadTaskName(info.getLockOwnerId(), info.getLockOwnerName()));
+      }
+      stream.println("  Stack:");
+      for (StackTraceElement frame : info.getStackTrace()) {
+        stream.println("    " + frame.toString());
+      }
+      stream.println("");
+    }
+  }
+
+  public static List<File> getMountPath() throws IOException {
+    BufferedReader mountOutput = null;
+    try {
+      Process mountProcess = Runtime.getRuntime ().exec("mount");
+      mountOutput = new BufferedReader(new InputStreamReader(mountProcess.getInputStream()));
+      List<File> mountPaths = new ArrayList<File>();
+      while (true) {
+        String line = mountOutput.readLine();
+        if (line == null) {
+          break;
+        }
+
+        int indexStart = line.indexOf(" on /");
+        int indexEnd = line.indexOf(" ", indexStart + 4);
+
+        mountPaths.add(new File(line.substring (indexStart + 4, indexEnd)));
+      }
+      return mountPaths;
+    } catch (IOException e) {
+      e.printStackTrace();
+      throw e;
+    } finally {
+      if(mountOutput != null) {
+        mountOutput.close();
+      }
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    StringUtils.startupShutdownMessage(TajoWorker.class, args, LOG);
+
+    TajoConf tajoConf = new TajoConf();
+    tajoConf.addResource(new Path(TajoConstants.SYSTEM_CONF_FILENAME));
+
+    try {
+      TajoWorker tajoWorker = new TajoWorker();
+      tajoWorker.startWorker(tajoConf, args);
+    } catch (Throwable t) {
+      LOG.fatal("Error starting TajoWorker", t);
+      System.exit(-1);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
new file mode 100644
index 0000000..937d886
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
@@ -0,0 +1,221 @@
+/**
+ * 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.worker;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.QueryMasterClientProtocol;
+import org.apache.tajo.master.querymaster.Query;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.rpc.BlockingRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public class TajoWorkerClientService extends AbstractService {
+  private static final Log LOG = LogFactory.getLog(TajoWorkerClientService.class);
+  private final PrimitiveProtos.BoolProto BOOL_TRUE =
+          PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
+  private final PrimitiveProtos.BoolProto BOOL_FALSE =
+          PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
+  private BlockingRpcServer rpcServer;
+  private InetSocketAddress bindAddr;
+  private String addr;
+  private int port;
+  private TajoConf conf;
+  private TajoWorker.WorkerContext workerContext;
+  private TajoWorkerClientProtocolServiceHandler serviceHandler;
+
+  public TajoWorkerClientService(TajoWorker.WorkerContext workerContext, int port) {
+    super(TajoWorkerClientService.class.getName());
+
+    this.port = port;
+    this.workerContext = workerContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    Preconditions.checkArgument(conf instanceof TajoConf);
+    this.conf = (TajoConf) conf;
+    this.serviceHandler = new TajoWorkerClientProtocolServiceHandler();
+
+    // init RPC Server in constructor cause Heartbeat Thread use bindAddr
+    // Setup RPC server
+    try {
+      // TODO initial port num is value of config and find unused port with sequence
+      InetSocketAddress initIsa = new InetSocketAddress("0.0.0.0", port);
+      if (initIsa.getAddress() == null) {
+        throw new IllegalArgumentException("Failed resolve of " + initIsa);
+      }
+
+      // TODO blocking/non-blocking??
+      int workerNum = this.conf.getIntVar(TajoConf.ConfVars.WORKER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM);
+      this.rpcServer = new BlockingRpcServer(QueryMasterClientProtocol.class, serviceHandler, initIsa, workerNum);
+      this.rpcServer.start();
+
+      this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+      this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+      this.port = bindAddr.getPort();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    // Get the master address
+    LOG.info(TajoWorkerClientService.class.getSimpleName() + " is bind to " + addr);
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    LOG.info("TajoWorkerClientService stopping");
+    if(rpcServer != null) {
+      rpcServer.shutdown();
+    }
+    LOG.info("TajoWorkerClientService stopped");
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddr() {
+    return bindAddr;
+  }
+
+  public class TajoWorkerClientProtocolServiceHandler
+          implements QueryMasterClientProtocol.QueryMasterClientProtocolService.BlockingInterface {
+    @Override
+    public PrimitiveProtos.BoolProto updateSessionVariables(
+            RpcController controller,
+            ClientProtos.UpdateSessionVariableRequest request) throws ServiceException {
+      return null;
+    }
+
+    @Override
+    public ClientProtos.GetQueryResultResponse getQueryResult(
+            RpcController controller,
+            ClientProtos.GetQueryResultRequest request) throws ServiceException {
+      QueryId queryId = new QueryId(request.getQueryId());
+      Query query = workerContext.getQueryMaster().getQueryMasterTask(queryId, true).getQuery();
+
+      ClientProtos.GetQueryResultResponse.Builder builder = ClientProtos.GetQueryResultResponse.newBuilder();
+      try {
+        builder.setTajoUserName(UserGroupInformation.getCurrentUser().getUserName());
+      } catch (IOException e) {
+        LOG.warn("Can't get current user name");
+      }
+
+      if(query == null) {
+        builder.setErrorMessage("No Query for " + queryId);
+      } else {
+        switch (query.getState()) {
+          case QUERY_SUCCEEDED:
+            builder.setTableDesc(query.getResultDesc().getProto());
+            break;
+          case QUERY_FAILED:
+          case QUERY_ERROR:
+            builder.setErrorMessage("Query " + queryId + " is failed");
+          default:
+            builder.setErrorMessage("Query " + queryId + " is still running");
+        }
+      }
+      return builder.build();
+    }
+
+    @Override
+    public ClientProtos.GetQueryStatusResponse getQueryStatus(
+            RpcController controller,
+            ClientProtos.GetQueryStatusRequest request) throws ServiceException {
+      ClientProtos.GetQueryStatusResponse.Builder builder
+              = ClientProtos.GetQueryStatusResponse.newBuilder();
+      QueryId queryId = new QueryId(request.getQueryId());
+
+      builder.setQueryId(request.getQueryId());
+
+      if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
+        builder.setResultCode(ClientProtos.ResultCode.OK);
+        builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
+      } else {
+        QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId);
+
+        builder.setResultCode(ClientProtos.ResultCode.OK);
+        builder.setQueryMasterHost(bindAddr.getHostName());
+        builder.setQueryMasterPort(bindAddr.getPort());
+
+        if (queryMasterTask == null) {
+          queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId, true);
+        }
+        if (queryMasterTask == null) {
+          builder.setState(TajoProtos.QueryState.QUERY_NOT_ASSIGNED);
+          return builder.build();
+        }
+
+        builder.setHasResult(
+            !(queryMasterTask.getQueryTaskContext().getQueryContext().isCreateTable() ||
+                queryMasterTask.getQueryTaskContext().getQueryContext().isInsert())
+        );
+
+        queryMasterTask.touchSessionTime();
+        Query query = queryMasterTask.getQuery();
+
+        if (query != null) {
+          builder.setState(query.getState());
+          builder.setProgress(query.getProgress());
+          builder.setSubmitTime(query.getAppSubmitTime());
+          if (query.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+            builder.setFinishTime(query.getFinishTime());
+          } else {
+            builder.setFinishTime(System.currentTimeMillis());
+          }
+        } else {
+          builder.setState(queryMasterTask.getState());
+          builder.setErrorMessage(queryMasterTask.getErrorMessage());
+        }
+      }
+      return builder.build();
+    }
+
+    @Override
+    public PrimitiveProtos.BoolProto closeQuery (
+            RpcController controller,
+            TajoIdProtos.QueryIdProto request) throws ServiceException {
+      final QueryId queryId = new QueryId(request);
+      LOG.info("Stop Query:" + queryId);
+      return BOOL_TRUE;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
new file mode 100644
index 0000000..392a7cf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
@@ -0,0 +1,152 @@
+/**
+ * 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.worker;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.rpc.AsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.InetSocketAddress;
+
+public class TajoWorkerManagerService extends CompositeService
+    implements TajoWorkerProtocol.TajoWorkerProtocolService.Interface {
+  private static final Log LOG = LogFactory.getLog(TajoWorkerManagerService.class.getName());
+
+  private AsyncRpcServer rpcServer;
+  private InetSocketAddress bindAddr;
+  private String addr;
+  private int port;
+
+  private TajoWorker.WorkerContext workerContext;
+
+  public TajoWorkerManagerService(TajoWorker.WorkerContext workerContext, int port) {
+    super(TajoWorkerManagerService.class.getName());
+    this.workerContext = workerContext;
+    this.port = port;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    Preconditions.checkArgument(conf instanceof TajoConf);
+    TajoConf tajoConf = (TajoConf) conf;
+    try {
+      // Setup RPC server
+      InetSocketAddress initIsa =
+          new InetSocketAddress("0.0.0.0", port);
+      if (initIsa.getAddress() == null) {
+        throw new IllegalArgumentException("Failed resolve of " + initIsa);
+      }
+
+      int workerNum = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_RPC_SERVER_WORKER_THREAD_NUM);
+      this.rpcServer = new AsyncRpcServer(TajoWorkerProtocol.class, this, initIsa, workerNum);
+      this.rpcServer.start();
+
+      this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+      this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+      this.port = bindAddr.getPort();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    // Get the master address
+    LOG.info("TajoWorkerManagerService is bind to " + addr);
+    tajoConf.setVar(TajoConf.ConfVars.WORKER_PEER_RPC_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddr));
+    super.init(tajoConf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(rpcServer != null) {
+      rpcServer.shutdown();
+    }
+    LOG.info("TajoWorkerManagerService stopped");
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddr() {
+    return bindAddr;
+  }
+
+  public String getHostAndPort() {
+    return bindAddr.getHostName() + ":" + bindAddr.getPort();
+  }
+
+  @Override
+  public void ping(RpcController controller,
+                   TajoIdProtos.QueryUnitAttemptIdProto attemptId,
+                   RpcCallback<PrimitiveProtos.BoolProto> done) {
+    done.run(TajoWorker.TRUE_PROTO);
+  }
+
+  @Override
+  public void executeExecutionBlock(RpcController controller,
+                                    TajoWorkerProtocol.RunExecutionBlockRequestProto request,
+                                    RpcCallback<PrimitiveProtos.BoolProto> done) {
+    workerContext.getWorkerSystemMetrics().counter("query", "executedExecutionBlocksNum").inc();
+    try {
+      String[] params = new String[7];
+      params[0] = "standby";  //mode(never used)
+      params[1] = request.getExecutionBlockId();
+      // NodeId has a form of hostname:port.
+      params[2] = request.getNodeId();
+      params[3] = request.getContainerId();
+
+      // QueryMaster's address
+      params[4] = request.getQueryMasterHost();
+      params[5] = String.valueOf(request.getQueryMasterPort());
+      params[6] = request.getQueryOutputPath();
+      workerContext.getTaskRunnerManager().startTask(params);
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void killTaskAttempt(RpcController controller, TajoIdProtos.QueryUnitAttemptIdProto request,
+                              RpcCallback<PrimitiveProtos.BoolProto> done) {
+    workerContext.getTaskRunnerManager().findTaskByQueryUnitAttemptId(new QueryUnitAttemptId(request)).kill();
+    done.run(TajoWorker.TRUE_PROTO);
+  }
+
+  @Override
+  public void cleanup(RpcController controller, TajoIdProtos.QueryIdProto request,
+                      RpcCallback<PrimitiveProtos.BoolProto> done) {
+    workerContext.cleanup(new QueryId(request).toString());
+    done.run(TajoWorker.TRUE_PROTO);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
new file mode 100644
index 0000000..30f56ee
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
@@ -0,0 +1,762 @@
+/**
+ * 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.worker;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoProtos.TaskAttemptState;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.engine.planner.logical.SortNode;
+import org.apache.tajo.engine.planner.physical.PhysicalExec;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.engine.query.QueryUnitRequest;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
+import org.apache.tajo.ipc.TajoWorkerProtocol.*;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.RpcChannelFactory;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.ApplicationIdUtils;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.text.NumberFormat;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public class Task {
+  private static final Log LOG = LogFactory.getLog(Task.class);
+  private static final float FETCHER_PROGRESS = 0.5f;
+
+  private final TajoConf systemConf;
+  private final QueryContext queryContext;
+  private final FileSystem localFS;
+  private TaskRunner.TaskRunnerContext taskRunnerContext;
+  private final QueryMasterProtocolService.Interface masterProxy;
+  private final LocalDirAllocator lDirAllocator;
+  private final QueryUnitAttemptId taskId;
+
+  private final Path taskDir;
+  private final QueryUnitRequest request;
+  private TaskAttemptContext context;
+  private List<Fetcher> fetcherRunners;
+  private LogicalNode plan;
+  private final Map<String, TableDesc> descs = Maps.newHashMap();
+  private PhysicalExec executor;
+  private boolean interQuery;
+  private boolean killed = false;
+  private boolean aborted = false;
+  private boolean stopped = false;
+  private final Reporter reporter;
+  private Path inputTableBaseDir;
+
+  private static int completedTasksNum = 0;
+  private static int succeededTasksNum = 0;
+  private static int killedTasksNum = 0;
+  private static int failedTasksNum = 0;
+
+  private long startTime;
+  private long finishTime;
+
+  private final TableStats inputStats;
+
+  // TODO - to be refactored
+  private ShuffleType shuffleType = null;
+  private Schema finalSchema = null;
+  private TupleComparator sortComp = null;
+  private ClientSocketChannelFactory channelFactory = null;
+
+  static final String OUTPUT_FILE_PREFIX="part-";
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SUBQUERY =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(2);
+          return fmt;
+        }
+      };
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_TASK =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(6);
+          return fmt;
+        }
+      };
+
+  public Task(QueryUnitAttemptId taskId,
+              final TaskRunner.TaskRunnerContext worker,
+              final QueryMasterProtocolService.Interface masterProxy,
+              final QueryUnitRequest request) throws IOException {
+    this.request = request;
+    this.taskId = taskId;
+
+    this.systemConf = worker.getConf();
+    this.queryContext = request.getQueryContext();
+    this.taskRunnerContext = worker;
+    this.masterProxy = masterProxy;
+    this.localFS = worker.getLocalFS();
+    this.lDirAllocator = worker.getLocalDirAllocator();
+    this.taskDir = StorageUtil.concatPath(taskRunnerContext.getBaseDir(),
+        taskId.getQueryUnitId().getId() + "_" + taskId.getId());
+
+    this.context = new TaskAttemptContext(systemConf, taskId,
+        request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir);
+    this.context.setDataChannel(request.getDataChannel());
+    this.context.setEnforcer(request.getEnforcer());
+    this.inputStats = new TableStats();
+
+    this.reporter = new Reporter(taskId, masterProxy);
+    this.reporter.startCommunicationThread();
+
+    plan = CoreGsonHelper.fromJson(request.getSerializedData(), LogicalNode.class);
+    LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN);
+    for (LogicalNode node : scanNode) {
+      ScanNode scan = (ScanNode)node;
+      descs.put(scan.getCanonicalName(), scan.getTableDesc());
+    }
+
+    interQuery = request.getProto().getInterQuery();
+    if (interQuery) {
+      context.setInterQuery();
+      this.shuffleType = context.getDataChannel().getShuffleType();
+
+      if (shuffleType == ShuffleType.RANGE_SHUFFLE) {
+        SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT);
+        this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys());
+        this.sortComp = new TupleComparator(finalSchema, sortNode.getSortKeys());
+      }
+    } else {
+      // The final result of a task will be written in a file named part-ss-nnnnnnn,
+      // where ss is the subquery id associated with this task, and nnnnnn is the task id.
+      Path outFilePath = StorageUtil.concatPath(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME,
+          OUTPUT_FILE_PREFIX +
+          OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskId.getQueryUnitId().getExecutionBlockId().getId()) + "-" +
+          OUTPUT_FILE_FORMAT_TASK.get().format(taskId.getQueryUnitId().getId()));
+      LOG.info("Output File Path: " + outFilePath);
+      context.setOutputPath(outFilePath);
+    }
+
+    context.setState(TaskAttemptState.TA_PENDING);
+    LOG.info("==================================");
+    LOG.info("* Subquery " + request.getId() + " is initialized");
+    LOG.info("* InterQuery: " + interQuery
+        + (interQuery ? ", Use " + this.shuffleType + " shuffle":""));
+
+    LOG.info("* Fragments (num: " + request.getFragments().size() + ")");
+    LOG.info("* Fetches (total:" + request.getFetches().size() + ") :");
+    for (Fetch f : request.getFetches()) {
+      LOG.info("Table Id: " + f.getName() + ", url: " + f.getUrls());
+    }
+    LOG.info("* Local task dir: " + taskDir);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("* plan:\n");
+      LOG.debug(plan.toString());
+    }
+    LOG.info("==================================");
+  }
+
+  public void init() throws IOException {
+    // initialize a task temporal dir
+    localFS.mkdirs(taskDir);
+
+    if (request.getFetches().size() > 0) {
+      inputTableBaseDir = localFS.makeQualified(
+          lDirAllocator.getLocalPathForWrite(
+              getTaskAttemptDir(context.getTaskId()).toString(), systemConf));
+      localFS.mkdirs(inputTableBaseDir);
+      Path tableDir;
+      for (String inputTable : context.getInputTables()) {
+        tableDir = new Path(inputTableBaseDir, inputTable);
+        if (!localFS.exists(tableDir)) {
+          LOG.info("the directory is created  " + tableDir.toUri());
+          localFS.mkdirs(tableDir);
+        }
+      }
+    }
+    // for localizing the intermediate data
+    localize(request);
+  }
+
+  public QueryUnitAttemptId getTaskId() {
+    return taskId;
+  }
+
+  public static Log getLog() {
+    return LOG;
+  }
+
+  public void localize(QueryUnitRequest request) throws IOException {
+    fetcherRunners = getFetchRunners(context, request.getFetches());
+  }
+
+  public QueryUnitAttemptId getId() {
+    return context.getTaskId();
+  }
+
+  public TaskAttemptState getStatus() {
+    return context.getState();
+  }
+
+  public String toString() {
+    return "queryId: " + this.getId() + " status: " + this.getStatus();
+  }
+
+  public void setState(TaskAttemptState status) {
+    context.setState(status);
+  }
+
+  public TaskAttemptContext getContext() {
+    return context;
+  }
+
+  public boolean hasFetchPhase() {
+    return fetcherRunners.size() > 0;
+  }
+
+  public void fetch() {
+    for (Fetcher f : fetcherRunners) {
+      taskRunnerContext.getFetchLauncher().submit(new FetchRunner(context, f));
+    }
+  }
+
+  public void kill() {
+    killed = true;
+    context.stop();
+    context.setState(TaskAttemptState.TA_KILLED);
+    releaseChannelFactory();
+  }
+
+  public void abort() {
+    aborted = true;
+    context.stop();
+    releaseChannelFactory();
+  }
+
+  public void cleanUp() {
+    // remove itself from worker
+    if (context.getState() == TaskAttemptState.TA_SUCCEEDED) {
+      try {
+        localFS.delete(context.getWorkDir(), true);
+        synchronized (taskRunnerContext.getTasks()) {
+          taskRunnerContext.getTasks().remove(this.getId());
+        }
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    } else {
+      LOG.error("QueryUnitAttemptId: " + context.getTaskId() + " status: " + context.getState());
+    }
+  }
+
+  public TaskStatusProto getReport() {
+    TaskStatusProto.Builder builder = TaskStatusProto.newBuilder();
+    builder.setWorkerName(taskRunnerContext.getNodeId());
+    builder.setId(context.getTaskId().getProto())
+        .setProgress(context.getProgress())
+        .setState(context.getState());
+
+    builder.setInputStats(reloadInputStats());
+
+    if (context.getResultStats() != null) {
+      builder.setResultStats(context.getResultStats().getProto());
+    }
+    return builder.build();
+  }
+
+  private CatalogProtos.TableStatsProto reloadInputStats() {
+    synchronized(inputStats) {
+      if (this.executor == null) {
+        return inputStats.getProto();
+      }
+
+      TableStats executorInputStats = this.executor.getInputStats();
+
+      if (executorInputStats != null) {
+        inputStats.setValues(executorInputStats);
+      }
+      return inputStats.getProto();
+    }
+  }
+
+  private TaskCompletionReport getTaskCompletionReport() {
+    TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder();
+    builder.setId(context.getTaskId().getProto());
+
+    builder.setInputStats(reloadInputStats());
+
+    if (context.hasResultStats()) {
+      builder.setResultStats(context.getResultStats().getProto());
+    } else {
+      builder.setResultStats(new TableStats().getProto());
+    }
+
+    Iterator<Entry<Integer,String>> it = context.getShuffleFileOutputs();
+    if (it.hasNext()) {
+      do {
+        Entry<Integer,String> entry = it.next();
+        ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder();
+        part.setPartId(entry.getKey());
+        builder.addShuffleFileOutputs(part.build());
+      } while (it.hasNext());
+    }
+
+    return builder.build();
+  }
+
+  private void waitForFetch() throws InterruptedException, IOException {
+    context.getFetchLatch().await();
+    LOG.info(context.getTaskId() + " All fetches are done!");
+    Collection<String> inputs = Lists.newArrayList(context.getInputTables());
+    for (String inputTable: inputs) {
+      File tableDir = new File(context.getFetchIn(), inputTable);
+      FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta());
+      context.updateAssignedFragments(inputTable, frags);
+    }
+    releaseChannelFactory();
+  }
+
+  public void run() {
+    startTime = System.currentTimeMillis();
+    String errorMessage = null;
+    try {
+      context.setState(TaskAttemptState.TA_RUNNING);
+
+      if (context.hasFetchPhase()) {
+        // If the fetch is still in progress, the query unit must wait for
+        // complete.
+        waitForFetch();
+        context.setFetcherProgress(FETCHER_PROGRESS);
+        context.setProgress(FETCHER_PROGRESS);
+      }
+
+      if (context.getFragmentSize() > 0) {
+        this.executor = taskRunnerContext.getTQueryEngine().
+            createPlan(context, plan);
+        this.executor.init();
+        while(!killed && executor.next() != null) {
+        }
+        this.executor.close();
+        reloadInputStats();
+        this.executor = null;
+      }
+    } catch (Exception e) {
+      // errorMessage will be sent to master.
+      errorMessage = ExceptionUtils.getStackTrace(e);
+      LOG.error(errorMessage);
+      aborted = true;
+    } finally {
+      context.setProgress(1.0f);
+      stopped = true;
+      completedTasksNum++;
+
+      if (killed || aborted) {
+        context.setExecutorProgress(0.0f);
+        context.setProgress(0.0f);
+        if(killed) {
+          context.setState(TaskAttemptState.TA_KILLED);
+          masterProxy.statusUpdate(null, getReport(), NullCallback.get());
+          killedTasksNum++;
+        } else {
+          context.setState(TaskAttemptState.TA_FAILED);
+          TaskFatalErrorReport.Builder errorBuilder =
+              TaskFatalErrorReport.newBuilder()
+                  .setId(getId().getProto());
+          if (errorMessage != null) {
+            errorBuilder.setErrorMessage(errorMessage);
+          }
+
+          masterProxy.fatalError(null, errorBuilder.build(), NullCallback.get());
+          failedTasksNum++;
+        }
+
+        // stopping the status report
+        try {
+          reporter.stopCommunicationThread();
+        } catch (InterruptedException e) {
+          LOG.warn(e);
+        }
+
+      } else {
+        // if successful
+        context.setProgress(1.0f);
+        context.setState(TaskAttemptState.TA_SUCCEEDED);
+
+        // stopping the status report
+        try {
+          reporter.stopCommunicationThread();
+        } catch (InterruptedException e) {
+          LOG.warn(e);
+        }
+
+        TaskCompletionReport report = getTaskCompletionReport();
+        masterProxy.done(null, report, NullCallback.get());
+        succeededTasksNum++;
+      }
+
+      finishTime = System.currentTimeMillis();
+
+      cleanupTask();
+      LOG.info("Task Counter - total:" + completedTasksNum + ", succeeded: " + succeededTasksNum
+          + ", killed: " + killedTasksNum + ", failed: " + failedTasksNum);
+    }
+  }
+
+  public void cleanupTask() {
+    taskRunnerContext.addTaskHistory(getId(), getTaskHistory());
+    taskRunnerContext.getTasks().remove(getId());
+    taskRunnerContext = null;
+
+    fetcherRunners.clear();
+    executor = null;
+    plan = null;
+    context = null;
+    releaseChannelFactory();
+  }
+
+  public TaskHistory getTaskHistory() {
+    TaskHistory taskHistory = new TaskHistory();
+    taskHistory.setStartTime(startTime);
+    taskHistory.setFinishTime(finishTime);
+    if (context.getOutputPath() != null) {
+      taskHistory.setOutputPath(context.getOutputPath().toString());
+    }
+
+    if (context.getWorkDir() != null) {
+      taskHistory.setWorkingPath(context.getWorkDir().toString());
+    }
+
+    try {
+      taskHistory.setStatus(getStatus().toString());
+      taskHistory.setProgress(context.getProgress());
+
+      taskHistory.setInputStats(new TableStats(reloadInputStats()));
+      if (context.getResultStats() != null) {
+        taskHistory.setOutputStats((TableStats)context.getResultStats().clone());
+      }
+
+      if (hasFetchPhase()) {
+        Map<URI, TaskHistory.FetcherHistory> fetcherHistories = new HashMap<URI, TaskHistory.FetcherHistory>();
+
+        for(Fetcher eachFetcher: fetcherRunners) {
+          TaskHistory.FetcherHistory fetcherHistory = new TaskHistory.FetcherHistory();
+          fetcherHistory.setStartTime(eachFetcher.getStartTime());
+          fetcherHistory.setFinishTime(eachFetcher.getFinishTime());
+          fetcherHistory.setStatus(eachFetcher.getStatus());
+          fetcherHistory.setUri(eachFetcher.getURI().toString());
+          fetcherHistory.setFileLen(eachFetcher.getFileLen());
+          fetcherHistory.setMessageReceiveCount(eachFetcher.getMessageReceiveCount());
+
+          fetcherHistories.put(eachFetcher.getURI(), fetcherHistory);
+        }
+
+        taskHistory.setFetchers(fetcherHistories);
+      }
+    } catch (Exception e) {
+      taskHistory.setStatus(StringUtils.stringifyException(e));
+      e.printStackTrace();
+    }
+
+    return taskHistory;
+  }
+
+  public int hashCode() {
+    return context.hashCode();
+  }
+
+  public boolean equals(Object obj) {
+    if (obj instanceof Task) {
+      Task other = (Task) obj;
+      return this.context.equals(other.context);
+    }
+    return false;
+  }
+
+  private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta)
+      throws IOException {
+    Configuration c = new Configuration(systemConf);
+    c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///");
+    FileSystem fs = FileSystem.get(c);
+    Path tablePath = new Path(file.getAbsolutePath());
+
+    List<FileFragment> listTablets = new ArrayList<FileFragment>();
+    FileFragment tablet;
+
+    FileStatus[] fileLists = fs.listStatus(tablePath);
+    for (FileStatus f : fileLists) {
+      if (f.getLen() == 0) {
+        continue;
+      }
+      tablet = new FileFragment(name, f.getPath(), 0l, f.getLen());
+      listTablets.add(tablet);
+    }
+
+    FileFragment[] tablets = new FileFragment[listTablets.size()];
+    listTablets.toArray(tablets);
+
+    return tablets;
+  }
+
+  private class FetchRunner implements Runnable {
+    private final TaskAttemptContext ctx;
+    private final Fetcher fetcher;
+
+    public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) {
+      this.ctx = ctx;
+      this.fetcher = fetcher;
+    }
+
+    @Override
+    public void run() {
+      int retryNum = 0;
+      int maxRetryNum = 5;
+      int retryWaitTime = 1000;
+
+      try { // for releasing fetch latch
+        while(retryNum < maxRetryNum) {
+          if (retryNum > 0) {
+            try {
+              Thread.sleep(retryWaitTime);
+            } catch (InterruptedException e) {
+              LOG.error(e);
+            }
+            LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")");
+          }
+          try {
+            File fetched = fetcher.get();
+            if (fetched != null) {
+              break;
+            }
+          } catch (IOException e) {
+            LOG.error("Fetch failed: " + fetcher.getURI(), e);
+          }
+          retryNum++;
+        }
+      } finally {
+        fetcherFinished(ctx);
+      }
+
+      if (retryNum == maxRetryNum) {
+        LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")");
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public static float adjustFetchProcess(int totalFetcher, int remainFetcher) {
+    return ((float)(totalFetcher - remainFetcher)) / (float)totalFetcher * FETCHER_PROGRESS;
+  }
+
+  private synchronized void fetcherFinished(TaskAttemptContext ctx) {
+    int fetcherSize = fetcherRunners.size();
+    if(fetcherSize == 0) {
+      return;
+    }
+    try {
+      int numRunningFetcher = (int)(ctx.getFetchLatch().getCount()) - 1;
+
+      if (numRunningFetcher == 0) {
+        context.setProgress(FETCHER_PROGRESS);
+      } else {
+        context.setProgress(adjustFetchProcess(fetcherSize, numRunningFetcher));
+      }
+    } finally {
+      ctx.getFetchLatch().countDown();
+    }
+  }
+
+  private void releaseChannelFactory(){
+    if(channelFactory != null) {
+      channelFactory.shutdown();
+      channelFactory.releaseExternalResources();
+      channelFactory = null;
+    }
+  }
+
+  private List<Fetcher> getFetchRunners(TaskAttemptContext ctx,
+                                        List<Fetch> fetches) throws IOException {
+
+    if (fetches.size() > 0) {
+
+      releaseChannelFactory();
+
+
+      int workerNum = ctx.getConf().getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM);
+      channelFactory = RpcChannelFactory.createClientChannelFactory("Fetcher", workerNum);
+      Path inputDir = lDirAllocator.
+          getLocalPathToRead(
+              getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf);
+      File storeDir;
+
+      int i = 0;
+      File storeFile;
+      List<Fetcher> runnerList = Lists.newArrayList();
+      for (Fetch f : fetches) {
+        storeDir = new File(inputDir.toString(), f.getName());
+        if (!storeDir.exists()) {
+          storeDir.mkdirs();
+        }
+        storeFile = new File(storeDir, "in_" + i);
+        Fetcher fetcher = new Fetcher(URI.create(f.getUrls()), storeFile, channelFactory);
+        runnerList.add(fetcher);
+        i++;
+      }
+      ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString()));
+      return runnerList;
+    } else {
+      return Lists.newArrayList();
+    }
+  }
+
+  protected class Reporter {
+    private QueryMasterProtocolService.Interface masterStub;
+    private Thread pingThread;
+    private AtomicBoolean stop = new AtomicBoolean(false);
+    private static final int PROGRESS_INTERVAL = 3000;
+    private QueryUnitAttemptId taskId;
+
+    public Reporter(QueryUnitAttemptId taskId, QueryMasterProtocolService.Interface masterStub) {
+      this.taskId = taskId;
+      this.masterStub = masterStub;
+    }
+
+    Runnable createReporterThread() {
+
+      return new Runnable() {
+        final int MAX_RETRIES = 3;
+        int remainingRetries = MAX_RETRIES;
+        @Override
+        public void run() {
+          while (!stop.get() && !stopped) {
+            try {
+              if(executor != null && context.getProgress() < 1.0f) {
+                float progress = executor.getProgress();
+                context.setExecutorProgress(progress);
+              }
+            } catch (Throwable t) {
+              LOG.error("Get progress error: " + t.getMessage(), t);
+            }
+
+            try {
+              if (context.isPorgressChanged()) {
+                masterStub.statusUpdate(null, getReport(), NullCallback.get());
+              } else {
+                masterStub.ping(null, taskId.getProto(), NullCallback.get());
+              }
+            } catch (Throwable t) {
+              LOG.error(t.getMessage(), t);
+              remainingRetries -=1;
+              if (remainingRetries == 0) {
+                ReflectionUtils.logThreadInfo(LOG, "Communication exception", 0);
+                LOG.warn("Last retry, exiting ");
+                throw new RuntimeException(t);
+              }
+            } finally {
+              if (remainingRetries > 0) {
+                synchronized (pingThread) {
+                  try {
+                    pingThread.wait(PROGRESS_INTERVAL);
+                  } catch (InterruptedException e) {
+                  }
+                }
+              }
+            }
+          }
+        }
+      };
+    }
+
+    public void startCommunicationThread() {
+      if (pingThread == null) {
+        pingThread = new Thread(createReporterThread());
+        pingThread.setName("communication thread");
+        pingThread.start();
+      }
+    }
+
+    public void stopCommunicationThread() throws InterruptedException {
+      if(stop.getAndSet(true)){
+        return;
+      }
+
+      if (pingThread != null) {
+        // Intent of the lock is to not send an interupt in the middle of an
+        // umbilical.ping or umbilical.statusUpdate
+        synchronized(pingThread) {
+          //Interrupt if sleeping. Otherwise wait for the RPC call to return.
+          pingThread.notifyAll();
+        }
+      }
+    }
+  }
+
+  public static final String FILECACHE = "filecache";
+  public static final String APPCACHE = "appcache";
+  public static final String USERCACHE = "usercache";
+
+  String fileCache;
+  public String getFileCacheDir() {
+    fileCache = USERCACHE + "/" + "hyunsik" + "/" + APPCACHE + "/" +
+        ConverterUtils.toString(ApplicationIdUtils.queryIdToAppId(taskId.getQueryUnitId().getExecutionBlockId().getQueryId())) +
+        "/" + "output";
+    return fileCache;
+  }
+
+  public static Path getTaskAttemptDir(QueryUnitAttemptId quid) {
+    Path workDir =
+        StorageUtil.concatPath(
+            quid.getQueryUnitId().getExecutionBlockId().getQueryId().toString(),
+            "in",
+            quid.getQueryUnitId().getExecutionBlockId().toString(),
+            String.valueOf(quid.getQueryUnitId().getId()),
+            String.valueOf(quid.getId()));
+    return workDir;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
new file mode 100644
index 0000000..6f3281c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
@@ -0,0 +1,270 @@
+/**
+ * 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.worker;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoProtos.TaskAttemptState;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+
+import java.io.File;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+
+/**
+ * Contains the information about executing subquery.
+ */
+public class TaskAttemptContext {
+  private static final Log LOG = LogFactory.getLog(TaskAttemptContext.class);
+  private final TajoConf conf;
+  private final Map<String, List<FragmentProto>> fragmentMap = Maps.newHashMap();
+
+  private TaskAttemptState state;
+  private TableStats resultStats;
+  private QueryUnitAttemptId queryId;
+  private final Path workDir;
+  private boolean needFetch = false;
+  private CountDownLatch doneFetchPhaseSignal;
+  private float progress = 0.0f;
+  private float fetcherProgress = 0.0f;
+  private AtomicBoolean progressChanged = new AtomicBoolean(false);
+
+  /** a map of shuffled file outputs */
+  private Map<Integer, String> shuffleFileOutputs;
+  private File fetchIn;
+  private boolean stopped = false;
+  private boolean interQuery = false;
+  private Path outputPath;
+  private DataChannel dataChannel;
+  private Enforcer enforcer;
+
+  public TaskAttemptContext(TajoConf conf, final QueryUnitAttemptId queryId,
+                            final FragmentProto[] fragments,
+                            final Path workDir) {
+    this.conf = conf;
+    this.queryId = queryId;
+    
+    for(FragmentProto t : fragments) {
+      if (fragmentMap.containsKey(t.getId())) {
+        fragmentMap.get(t.getId()).add(t);
+      } else {
+        List<FragmentProto> frags = new ArrayList<FragmentProto>();
+        frags.add(t);
+        fragmentMap.put(t.getId(), frags);
+      }
+    }
+
+    this.workDir = workDir;
+    this.shuffleFileOutputs = Maps.newHashMap();
+
+    state = TaskAttemptState.TA_PENDING;
+  }
+
+  @VisibleForTesting
+  public TaskAttemptContext(TajoConf conf, final QueryUnitAttemptId queryId,
+                            final Fragment [] fragments,  final Path workDir) {
+    this(conf, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir);
+  }
+
+  public TajoConf getConf() {
+    return this.conf;
+  }
+  
+  public TaskAttemptState getState() {
+    return this.state;
+  }
+  
+  public void setState(TaskAttemptState state) {
+    this.state = state;
+    LOG.info("Query status of " + getTaskId() + " is changed to " + state);
+  }
+
+  public void setDataChannel(DataChannel dataChannel) {
+    this.dataChannel = dataChannel;
+  }
+
+  public DataChannel getDataChannel() {
+    return dataChannel;
+  }
+
+  public void setEnforcer(Enforcer enforcer) {
+    this.enforcer = enforcer;
+  }
+
+  public Enforcer getEnforcer() {
+    return this.enforcer;
+  }
+
+  public boolean hasResultStats() {
+    return resultStats != null;
+  }
+
+  public void setResultStats(TableStats stats) {
+    this.resultStats = stats;
+  }
+
+  public TableStats getResultStats() {
+    return this.resultStats;
+  }
+
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+  public void setInterQuery() {
+    this.interQuery = true;
+  }
+
+  public void setOutputPath(Path outputPath) {
+    this.outputPath = outputPath;
+  }
+
+  public Path getOutputPath() {
+    return this.outputPath;
+  }
+
+  public boolean isInterQuery() {
+    return this.interQuery;
+  }
+  
+  public void stop() {
+    this.stopped = true;
+  }
+  
+  public void addFetchPhase(int count, File fetchIn) {
+    this.needFetch = true;
+    this.doneFetchPhaseSignal = new CountDownLatch(count);
+    this.fetchIn = fetchIn;
+  }
+  
+  public File getFetchIn() {
+    return this.fetchIn;
+  }
+  
+  public boolean hasFetchPhase() {
+    return this.needFetch;
+  }
+  
+  public CountDownLatch getFetchLatch() {
+    return doneFetchPhaseSignal;
+  }
+  
+  public void addShuffleFileOutput(int partId, String fileName) {
+    shuffleFileOutputs.put(partId, fileName);
+  }
+  
+  public Iterator<Entry<Integer,String>> getShuffleFileOutputs() {
+    return shuffleFileOutputs.entrySet().iterator();
+  }
+  
+  public void updateAssignedFragments(String tableId, Fragment[] fragments) {
+    fragmentMap.remove(tableId);
+    for(Fragment t : fragments) {
+      if (fragmentMap.containsKey(t.getTableName())) {
+        fragmentMap.get(t.getTableName()).add(t.getProto());
+      } else {
+        List<FragmentProto> frags = new ArrayList<FragmentProto>();
+        frags.add(t.getProto());
+        fragmentMap.put(t.getTableName(), frags);
+      }
+    }
+  }
+  
+  public Path getWorkDir() {
+    return this.workDir;
+  }
+  
+  public QueryUnitAttemptId getTaskId() {
+    return this.queryId;
+  }
+  
+  public float getProgress() {
+    return this.progress;
+  }
+  
+  public void setProgress(float progress) {
+    float previousProgress = this.progress;
+    this.progress = progress;
+    progressChanged.set(previousProgress != progress);
+  }
+
+  public boolean isPorgressChanged() {
+    return progressChanged.get();
+  }
+  public void setExecutorProgress(float executorProgress) {
+    float adjustProgress = executorProgress * (1 - fetcherProgress);
+    setProgress(fetcherProgress + adjustProgress);
+  }
+
+  public void setFetcherProgress(float fetcherProgress) {
+    this.fetcherProgress = fetcherProgress;
+  }
+
+  public FragmentProto getTable(String id) {
+    if (fragmentMap.get(id) == null) {
+      //for empty table
+      return null;
+    }
+    return fragmentMap.get(id).get(0);
+  }
+
+  public int getFragmentSize() {
+    return fragmentMap.size();
+  }
+
+  public Collection<String> getInputTables() {
+    return fragmentMap.keySet();
+  }
+  
+  public FragmentProto [] getTables(String id) {
+    if (fragmentMap.get(id) == null) {
+      //for empty table
+      return null;
+    }
+    return fragmentMap.get(id).toArray(new FragmentProto[fragmentMap.get(id).size()]);
+  }
+  
+  public int hashCode() {
+    return Objects.hashCode(queryId);
+  }
+  
+  public boolean equals(Object obj) {
+    if (obj instanceof TaskAttemptContext) {
+      TaskAttemptContext other = (TaskAttemptContext) obj;
+      return queryId.equals(other.getTaskId());
+    } else {
+      return false;
+    }
+  }
+}
\ No newline at end of file


[38/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java
new file mode 100644
index 0000000..17b5d0a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java
@@ -0,0 +1,91 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.engine.planner.logical.*;
+
+import java.util.Stack;
+
+public interface LogicalPlanVisitor<CONTEXT, RESULT> {
+  RESULT visitRoot(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, LogicalRootNode node,
+                   Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitProjection(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, ProjectionNode node,
+                         Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitLimit(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, LimitNode node,
+                    Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitSort(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, SortNode node,
+                   Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitHaving(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, HavingNode node,
+                      Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitGroupBy(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, GroupbyNode node,
+                      Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitFilter(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, SelectionNode node,
+                     Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitJoin(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode node,
+                   Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitUnion(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, UnionNode node,
+                    Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitExcept(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, ExceptNode node,
+                     Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitIntersect(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, IntersectNode node,
+                        Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitTableSubQuery(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, TableSubQueryNode node,
+                            Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitScan(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode node,
+                   Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitPartitionedTableScan(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                   PartitionedTableScanNode node, Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitStoreTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, StoreTableNode node,
+                         Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitInsert(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, InsertNode node,
+                     Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitCreateDatabase(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, CreateDatabaseNode node,
+                          Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitDropDatabase(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, DropDatabaseNode node,
+                             Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitCreateTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, CreateTableNode node,
+                          Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitDropTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, DropTableNode node,
+                        Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitAlterTablespace(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, AlterTablespaceNode node,
+                          Stack<LogicalNode> stack) throws PlanningException;
+
+  RESULT visitAlterTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, AlterTableNode node,
+                         Stack<LogicalNode> stack) throws PlanningException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
new file mode 100644
index 0000000..f2ddf13
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
@@ -0,0 +1,1579 @@
+/**
+ * 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.planner;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.exception.VerifyException;
+import org.apache.tajo.engine.planner.LogicalPlan.QueryBlock;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.planner.rewrite.ProjectionPushDownRule;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+import static org.apache.tajo.algebra.CreateTable.PartitionType;
+import static org.apache.tajo.engine.planner.ExprNormalizer.ExprNormalizedResult;
+import static org.apache.tajo.engine.planner.LogicalPlan.BlockType;
+import static org.apache.tajo.engine.planner.LogicalPlanPreprocessor.PreprocessContext;
+
+/**
+ * This class creates a logical plan from a nested tajo algebra expression ({@link org.apache.tajo.algebra})
+ */
+public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContext, LogicalNode> {
+  private static Log LOG = LogFactory.getLog(LogicalPlanner.class);
+  private final CatalogService catalog;
+  private final LogicalPlanPreprocessor preprocessor;
+  private final ExprAnnotator exprAnnotator;
+  private final ExprNormalizer normalizer;
+
+  public LogicalPlanner(CatalogService catalog) {
+    this.catalog = catalog;
+    this.exprAnnotator = new ExprAnnotator(catalog);
+    this.preprocessor = new LogicalPlanPreprocessor(catalog, exprAnnotator);
+    this.normalizer = new ExprNormalizer();
+  }
+
+  public static class PlanContext {
+    Session session;
+    LogicalPlan plan;
+
+    // transient data for each query block
+    QueryBlock queryBlock;
+
+    boolean debugOrUnitTests;
+
+    public PlanContext(Session session, LogicalPlan plan, QueryBlock block, boolean debugOrUnitTests) {
+      this.session = session;
+      this.plan = plan;
+      this.queryBlock = block;
+      this.debugOrUnitTests = debugOrUnitTests;
+    }
+
+    public PlanContext(PlanContext context, QueryBlock block) {
+      this.session = context.session;
+      this.plan = context.plan;
+      this.queryBlock = block;
+      this.debugOrUnitTests = context.debugOrUnitTests;
+    }
+
+    public String toString() {
+      return "block=" + queryBlock.getName() + ", relNum=" + queryBlock.getRelations().size() + ", "+
+          queryBlock.namedExprsMgr.toString();
+    }
+  }
+
+  /**
+   * This generates a logical plan.
+   *
+   * @param expr A relational algebraic expression for a query.
+   * @return A logical plan
+   */
+  public LogicalPlan createPlan(Session session, Expr expr) throws PlanningException {
+    return createPlan(session, expr, false);
+  }
+
+  @VisibleForTesting
+  public LogicalPlan createPlan(Session session, Expr expr, boolean debug) throws PlanningException {
+
+    LogicalPlan plan = new LogicalPlan(session.getCurrentDatabase(), this);
+
+    QueryBlock rootBlock = plan.newAndGetBlock(LogicalPlan.ROOT_BLOCK);
+    PreprocessContext preProcessorCtx = new PreprocessContext(session, plan, rootBlock);
+    preprocessor.visit(preProcessorCtx, new Stack<Expr>(), expr);
+
+    PlanContext context = new PlanContext(session, plan, plan.getRootBlock(), debug);
+    LogicalNode topMostNode = this.visit(context, new Stack<Expr>(), expr);
+
+    // Add Root Node
+    LogicalRootNode root = plan.createNode(LogicalRootNode.class);
+    root.setInSchema(topMostNode.getOutSchema());
+    root.setChild(topMostNode);
+    root.setOutSchema(topMostNode.getOutSchema());
+    plan.getRootBlock().setRoot(root);
+
+    return plan;
+  }
+
+  public ExprAnnotator getExprAnnotator() {
+    return this.exprAnnotator;
+  }
+
+  public void preHook(PlanContext context, Stack<Expr> stack, Expr expr) throws PlanningException {
+    context.queryBlock.updateCurrentNode(expr);
+  }
+
+  public LogicalNode postHook(PlanContext context, Stack<Expr> stack, Expr expr, LogicalNode current)
+      throws PlanningException {
+
+
+    // Some generated logical nodes (e.g., implicit aggregation) without exprs will pass NULL as a expr parameter.
+    // We should skip them.
+    if (expr != null) {
+      // A relation list including a single ScanNode will return a ScanNode instance that already passed postHook.
+      // So, it skips the already-visited ScanNode instance.
+      if (expr.getType() == OpType.RelationList && current.getType() == NodeType.SCAN) {
+        return current;
+      }
+    }
+
+    QueryBlock queryBlock = context.queryBlock;
+    queryBlock.updateLatestNode(current);
+
+    // if this node is the topmost
+    if (stack.size() == 0) {
+      queryBlock.setRoot(current);
+    }
+
+    if (!stack.empty()) {
+      queryBlock.updateCurrentNode(stack.peek());
+    }
+    return current;
+  }
+
+  public LogicalNode visitExplain(PlanContext ctx, Stack<Expr> stack, Explain expr) throws PlanningException {
+    ctx.plan.setExplain();
+    return visit(ctx, stack, expr.getChild());
+  }
+
+  /*===============================================================================================
+    Data Manupulation Language (DML) SECTION
+   ===============================================================================================*/
+
+
+  /*===============================================================================================
+    PROJECTION SECTION
+   ===============================================================================================*/
+  @Override
+  public LogicalNode visitProjection(PlanContext context, Stack<Expr> stack, Projection projection)
+      throws PlanningException {
+
+
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    // If a non-from statement is given
+    if (!projection.hasChild()) {
+      return buildPlanForNoneFromStatement(context, stack, projection);
+    }
+
+    String [] referenceNames;
+    // in prephase, insert all target list into NamedExprManagers.
+    // Then it gets reference names, each of which points an expression in target list.
+    referenceNames = doProjectionPrephase(context, projection);
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Child Plan
+    ////////////////////////////////////////////////////////
+    stack.push(projection);
+    LogicalNode child = visit(context, stack, projection.getChild());
+
+    // check if it is implicit aggregation. If so, it inserts group-by node to its child.
+    if (block.isAggregationRequired()) {
+      child = insertGroupbyNode(context, child, stack);
+    }
+    stack.pop();
+    ////////////////////////////////////////////////////////
+
+    ProjectionNode projectionNode;
+    Target [] targets;
+    targets = buildTargets(plan, block, referenceNames);
+
+    // Set ProjectionNode
+    projectionNode = context.queryBlock.getNodeFromExpr(projection);
+    projectionNode.setInSchema(child.getOutSchema());
+    projectionNode.setTargets(targets);
+    projectionNode.setChild(child);
+
+    if (projection.isDistinct() && block.hasNode(NodeType.GROUP_BY)) {
+      throw new VerifyException("Cannot support grouping and distinct at the same time yet");
+    } else {
+      if (projection.isDistinct()) {
+        insertDistinctOperator(context, projectionNode, child, stack);
+      }
+    }
+
+    // It's for debugging and unit tests purpose.
+    // It sets raw targets, all of them are raw expressions instead of references.
+    if (context.debugOrUnitTests) {
+      setRawTargets(context, targets, referenceNames, projection);
+    }
+
+    verifyProjectedFields(block, projectionNode);
+    return projectionNode;
+  }
+
+  private void setRawTargets(PlanContext context, Target[] targets, String[] referenceNames,
+                             Projection projection) throws PlanningException {
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    // It's for debugging or unit tests.
+    Target [] rawTargets = new Target[projection.getNamedExprs().length];
+    for (int i = 0; i < projection.getNamedExprs().length; i++) {
+      NamedExpr namedExpr = projection.getNamedExprs()[i];
+      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
+      rawTargets[i] = new Target(evalNode, referenceNames[i]);
+    }
+    // it's for debugging or unit testing
+    block.setRawTargets(rawTargets);
+  }
+
+  private void insertDistinctOperator(PlanContext context, ProjectionNode projectionNode, LogicalNode child,
+                                      Stack<Expr> stack) throws PlanningException {
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    Schema outSchema = projectionNode.getOutSchema();
+    GroupbyNode dupRemoval = context.plan.createNode(GroupbyNode.class);
+    dupRemoval.setChild(child);
+    dupRemoval.setInSchema(projectionNode.getInSchema());
+    dupRemoval.setTargets(PlannerUtil.schemaToTargets(outSchema));
+    dupRemoval.setGroupingColumns(outSchema.toArray());
+
+    block.registerNode(dupRemoval);
+    postHook(context, stack, null, dupRemoval);
+
+    projectionNode.setChild(dupRemoval);
+    projectionNode.setInSchema(dupRemoval.getOutSchema());
+  }
+
+  private String [] doProjectionPrephase(PlanContext context, Projection projection) throws PlanningException {
+    QueryBlock block = context.queryBlock;
+
+    int finalTargetNum = projection.size();
+    String [] referenceNames = new String[finalTargetNum];
+    ExprNormalizedResult [] normalizedExprList = new ExprNormalizedResult[finalTargetNum];
+    NamedExpr namedExpr;
+    for (int i = 0; i < finalTargetNum; i++) {
+      namedExpr = projection.getNamedExprs()[i];
+
+      if (PlannerUtil.existsAggregationFunction(namedExpr)) {
+        block.setAggregationRequire();
+      }
+      // dissect an expression into multiple parts (at most dissected into three parts)
+      normalizedExprList[i] = normalizer.normalize(context, namedExpr.getExpr());
+    }
+
+    // Note: Why separate normalization and add(Named)Expr?
+    //
+    // ExprNormalizer internally makes use of the named exprs in NamedExprsManager.
+    // If we don't separate normalization work and addExprWithName, addExprWithName will find named exprs evaluated
+    // the same logical node. It will cause impossible evaluation in physical executors.
+    for (int i = 0; i < finalTargetNum; i++) {
+      namedExpr = projection.getNamedExprs()[i];
+      // Get all projecting references
+      if (namedExpr.hasAlias()) {
+        NamedExpr aliasedExpr = new NamedExpr(normalizedExprList[i].baseExpr, namedExpr.getAlias());
+        referenceNames[i] = block.namedExprsMgr.addNamedExpr(aliasedExpr);
+      } else {
+        referenceNames[i] = block.namedExprsMgr.addExpr(normalizedExprList[i].baseExpr);
+      }
+
+      // Add sub-expressions (i.e., aggregation part and scalar part) from dissected parts.
+      block.namedExprsMgr.addNamedExprArray(normalizedExprList[i].aggExprs);
+      block.namedExprsMgr.addNamedExprArray(normalizedExprList[i].scalarExprs);
+    }
+
+    return referenceNames;
+  }
+
+  /**
+   * It builds non-from statement (only expressions) like '<code>SELECT 1+3 as plus</code>'.
+   */
+  private EvalExprNode buildPlanForNoneFromStatement(PlanContext context, Stack<Expr> stack, Projection projection)
+      throws PlanningException {
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    int finalTargetNum = projection.getNamedExprs().length;
+    Target [] targets = new Target[finalTargetNum];
+
+    for (int i = 0; i < targets.length; i++) {
+      NamedExpr namedExpr = projection.getNamedExprs()[i];
+      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
+      if (namedExpr.hasAlias()) {
+        targets[i] = new Target(evalNode, namedExpr.getAlias());
+      } else {
+        targets[i] = new Target(evalNode, context.plan.generateUniqueColumnName(namedExpr.getExpr()));
+      }
+    }
+    EvalExprNode evalExprNode = context.queryBlock.getNodeFromExpr(projection);
+    evalExprNode.setTargets(targets);
+    evalExprNode.setOutSchema(PlannerUtil.targetToSchema(targets));
+    // it's for debugging or unit testing
+    block.setRawTargets(targets);
+    return evalExprNode;
+  }
+
+  private Target [] buildTargets(LogicalPlan plan, QueryBlock block, String[] referenceNames)
+      throws PlanningException {
+    Target [] targets = new Target[referenceNames.length];
+
+    for (int i = 0; i < referenceNames.length; i++) {
+      if (block.namedExprsMgr.isEvaluated(referenceNames[i])) {
+        targets[i] = block.namedExprsMgr.getTarget(referenceNames[i]);
+      } else {
+        NamedExpr namedExpr = block.namedExprsMgr.getNamedExpr(referenceNames[i]);
+        EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
+        block.namedExprsMgr.markAsEvaluated(referenceNames[i], evalNode);
+        targets[i] = new Target(evalNode, referenceNames[i]);
+      }
+    }
+    return targets;
+  }
+
+  public static void verifyProjectedFields(QueryBlock block, Projectable projectable) throws PlanningException {
+    if (projectable instanceof ProjectionNode && block.hasNode(NodeType.GROUP_BY)) {
+      for (Target target : projectable.getTargets()) {
+        Set<Column> columns = EvalTreeUtil.findUniqueColumns(target.getEvalTree());
+        for (Column c : columns) {
+          if (!projectable.getInSchema().contains(c)) {
+            throw new PlanningException(c.getQualifiedName()
+                + " must appear in the GROUP BY clause or be used in an aggregate function at node ("
+                + projectable.getPID() + ")" );
+          }
+        }
+      }
+    } else  if (projectable instanceof GroupbyNode) {
+      GroupbyNode groupbyNode = (GroupbyNode) projectable;
+      // It checks if all column references within each target can be evaluated with the input schema.
+      int groupingColumnNum = groupbyNode.getGroupingColumns().length;
+      for (int i = 0; i < groupingColumnNum; i++) {
+        Set<Column> columns = EvalTreeUtil.findUniqueColumns(groupbyNode.getTargets()[i].getEvalTree());
+        if (!projectable.getInSchema().containsAll(columns)) {
+          throw new PlanningException(String.format("Cannot get the field(s) \"%s\" at node (%d)",
+              TUtil.collectionToString(columns), projectable.getPID()));
+        }
+      }
+      if (groupbyNode.hasAggFunctions()) {
+        for (AggregationFunctionCallEval f : groupbyNode.getAggFunctions()) {
+          Set<Column> columns = EvalTreeUtil.findUniqueColumns(f);
+          for (Column c : columns) {
+            if (!projectable.getInSchema().contains(c)) {
+              throw new PlanningException(String.format("Cannot get the field \"%s\" at node (%d)",
+                  c, projectable.getPID()));
+            }
+          }
+        }
+      }
+    } else if (projectable instanceof RelationNode) {
+      RelationNode relationNode = (RelationNode) projectable;
+      for (Target target : projectable.getTargets()) {
+        Set<Column> columns = EvalTreeUtil.findUniqueColumns(target.getEvalTree());
+        for (Column c : columns) {
+          if (!relationNode.getTableSchema().contains(c)) {
+            throw new PlanningException(String.format("Cannot get the field \"%s\" at node (%d)",
+                c, projectable.getPID()));
+          }
+        }
+      }
+    } else {
+      for (Target target : projectable.getTargets()) {
+        Set<Column> columns = EvalTreeUtil.findUniqueColumns(target.getEvalTree());
+        for (Column c : columns) {
+          if (!projectable.getInSchema().contains(c)) {
+            throw new PlanningException(String.format("Cannot get the field \"%s\" at node (%d)",
+                c, projectable.getPID()));
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Insert a group-by operator before a sort or a projection operator.
+   * It is used only when a group-by clause is not given.
+   */
+  private LogicalNode insertGroupbyNode(PlanContext context, LogicalNode child, Stack<Expr> stack)
+      throws PlanningException {
+
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+    GroupbyNode groupbyNode = context.plan.createNode(GroupbyNode.class);
+    groupbyNode.setChild(child);
+    groupbyNode.setInSchema(child.getOutSchema());
+
+    groupbyNode.setGroupingColumns(new Column[] {});
+
+    Set<String> aggEvalNames = new LinkedHashSet<String>();
+    Set<AggregationFunctionCallEval> aggEvals = new LinkedHashSet<AggregationFunctionCallEval>();
+    boolean includeDistinctFunction = false;
+    for (Iterator<NamedExpr> it = block.namedExprsMgr.getIteratorForUnevaluatedExprs(); it.hasNext();) {
+      NamedExpr rawTarget = it.next();
+      try {
+        includeDistinctFunction = PlannerUtil.existsDistinctAggregationFunction(rawTarget.getExpr());
+        EvalNode evalNode = exprAnnotator.createEvalNode(context.plan, context.queryBlock, rawTarget.getExpr());
+        if (evalNode.getType() == EvalType.AGG_FUNCTION) {
+          aggEvalNames.add(rawTarget.getAlias());
+          aggEvals.add((AggregationFunctionCallEval) evalNode);
+          block.namedExprsMgr.markAsEvaluated(rawTarget.getAlias(), evalNode);
+        }
+      } catch (VerifyException ve) {
+      }
+    }
+
+    groupbyNode.setDistinct(includeDistinctFunction);
+    groupbyNode.setAggFunctions(aggEvals.toArray(new AggregationFunctionCallEval[aggEvals.size()]));
+    Target [] targets = ProjectionPushDownRule.buildGroupByTarget(groupbyNode, null,
+        aggEvalNames.toArray(new String[aggEvalNames.size()]));
+    groupbyNode.setTargets(targets);
+
+    // this inserted group-by node doesn't pass through preprocessor. So manually added.
+    block.registerNode(groupbyNode);
+    postHook(context, stack, null, groupbyNode);
+    return groupbyNode;
+  }
+
+  /*===============================================================================================
+    SORT SECTION
+  ===============================================================================================*/
+  @Override
+  public LimitNode visitLimit(PlanContext context, Stack<Expr> stack, Limit limit) throws PlanningException {
+    QueryBlock block = context.queryBlock;
+
+    EvalNode firstFetNum;
+    LogicalNode child;
+    if (limit.getFetchFirstNum().getType() == OpType.Literal) {
+      firstFetNum = exprAnnotator.createEvalNode(context.plan, block, limit.getFetchFirstNum());
+
+      ////////////////////////////////////////////////////////
+      // Visit and Build Child Plan
+      ////////////////////////////////////////////////////////
+      stack.push(limit);
+      child = visit(context, stack, limit.getChild());
+      stack.pop();
+      ////////////////////////////////////////////////////////
+    } else {
+      ExprNormalizedResult normalizedResult = normalizer.normalize(context, limit.getFetchFirstNum());
+      String referName = block.namedExprsMgr.addExpr(normalizedResult.baseExpr);
+      block.namedExprsMgr.addNamedExprArray(normalizedResult.aggExprs);
+      block.namedExprsMgr.addNamedExprArray(normalizedResult.scalarExprs);
+
+      ////////////////////////////////////////////////////////
+      // Visit and Build Child Plan
+      ////////////////////////////////////////////////////////
+      stack.push(limit);
+      child = visit(context, stack, limit.getChild());
+      stack.pop();
+      ////////////////////////////////////////////////////////
+
+      if (block.namedExprsMgr.isEvaluated(referName)) {
+        firstFetNum = block.namedExprsMgr.getTarget(referName).getEvalTree();
+      } else {
+        NamedExpr namedExpr = block.namedExprsMgr.getNamedExpr(referName);
+        firstFetNum = exprAnnotator.createEvalNode(context.plan, block, namedExpr.getExpr());
+        block.namedExprsMgr.markAsEvaluated(referName, firstFetNum);
+      }
+    }
+    LimitNode limitNode = block.getNodeFromExpr(limit);
+    limitNode.setChild(child);
+    limitNode.setInSchema(child.getOutSchema());
+    limitNode.setOutSchema(child.getOutSchema());
+
+    limitNode.setFetchFirst(firstFetNum.eval(null, null).asInt8());
+
+    return limitNode;
+  }
+
+  @Override
+  public SortNode visitSort(PlanContext context, Stack<Expr> stack, Sort sort) throws PlanningException {
+    QueryBlock block = context.queryBlock;
+
+    int sortKeyNum = sort.getSortSpecs().length;
+    Sort.SortSpec[] sortSpecs = sort.getSortSpecs();
+    String [] referNames = new String[sortKeyNum];
+
+    ExprNormalizedResult [] normalizedExprList = new ExprNormalizedResult[sortKeyNum];
+    for (int i = 0; i < sortKeyNum; i++) {
+      normalizedExprList[i] = normalizer.normalize(context, sortSpecs[i].getKey());
+    }
+    for (int i = 0; i < sortKeyNum; i++) {
+      referNames[i] = block.namedExprsMgr.addExpr(normalizedExprList[i].baseExpr);
+      block.namedExprsMgr.addNamedExprArray(normalizedExprList[i].aggExprs);
+      block.namedExprsMgr.addNamedExprArray(normalizedExprList[i].scalarExprs);
+    }
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Child Plan
+    ////////////////////////////////////////////////////////
+    stack.push(sort);
+    LogicalNode child = visit(context, stack, sort.getChild());
+    if (block.isAggregationRequired()) {
+      child = insertGroupbyNode(context, child, stack);
+    }
+    stack.pop();
+    ////////////////////////////////////////////////////////
+
+    SortNode sortNode = block.getNodeFromExpr(sort);
+    sortNode.setChild(child);
+    sortNode.setInSchema(child.getOutSchema());
+    sortNode.setOutSchema(child.getOutSchema());
+
+
+    // Building sort keys
+    Column column;
+    SortSpec [] annotatedSortSpecs = new SortSpec[sortKeyNum];
+    for (int i = 0; i < sortKeyNum; i++) {
+      if (block.namedExprsMgr.isEvaluated(referNames[i])) {
+        column = block.namedExprsMgr.getTarget(referNames[i]).getNamedColumn();
+      } else {
+        throw new IllegalStateException("Unexpected State: " + TUtil.arrayToString(sortSpecs));
+      }
+      annotatedSortSpecs[i] = new SortSpec(column, sortSpecs[i].isAscending(), sortSpecs[i].isNullFirst());
+    }
+
+    sortNode.setSortSpecs(annotatedSortSpecs);
+    return sortNode;
+  }
+
+  /*===============================================================================================
+    GROUP BY SECTION
+   ===============================================================================================*/
+
+  @Override
+  public LogicalNode visitHaving(PlanContext context, Stack<Expr> stack, Having expr) throws PlanningException {
+    QueryBlock block = context.queryBlock;
+
+    ExprNormalizedResult normalizedResult = normalizer.normalize(context, expr.getQual());
+    String referName = block.namedExprsMgr.addExpr(normalizedResult.baseExpr);
+    block.namedExprsMgr.addNamedExprArray(normalizedResult.aggExprs);
+    block.namedExprsMgr.addNamedExprArray(normalizedResult.scalarExprs);
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Child Plan
+    ////////////////////////////////////////////////////////
+    stack.push(expr);
+    LogicalNode child = visit(context, stack, expr.getChild());
+    stack.pop();
+    ////////////////////////////////////////////////////////
+
+    HavingNode having = new HavingNode(context.plan.newPID());
+    having.setChild(child);
+    having.setInSchema(child.getOutSchema());
+    having.setOutSchema(child.getOutSchema());
+
+    EvalNode havingCondition;
+    if (block.namedExprsMgr.isEvaluated(referName)) {
+      havingCondition = block.namedExprsMgr.getTarget(referName).getEvalTree();
+    } else {
+      NamedExpr namedExpr = block.namedExprsMgr.getNamedExpr(referName);
+      havingCondition = exprAnnotator.createEvalNode(context.plan, block, namedExpr.getExpr());
+      block.namedExprsMgr.markAsEvaluated(referName, havingCondition);
+    }
+
+    // set having condition
+    having.setQual(havingCondition);
+
+    return having;
+  }
+
+  @Override
+  public LogicalNode visitGroupBy(PlanContext context, Stack<Expr> stack, Aggregation aggregation)
+      throws PlanningException {
+
+    // Initialization Phase:
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    // Normalize grouping keys and add normalized grouping keys to NamedExprManager
+    int groupingKeyNum = aggregation.getGroupSet()[0].getGroupingSets().length;
+    ExprNormalizedResult [] normalizedResults = new ExprNormalizedResult[groupingKeyNum];
+    for (int i = 0; i < groupingKeyNum; i++) {
+      Expr groupingKey = aggregation.getGroupSet()[0].getGroupingSets()[i];
+      normalizedResults[i] = normalizer.normalize(context, groupingKey);
+    }
+
+    String [] groupingKeyRefNames = new String[groupingKeyNum];
+    for (int i = 0; i < groupingKeyNum; i++) {
+      groupingKeyRefNames[i] = block.namedExprsMgr.addExpr(normalizedResults[i].baseExpr);
+      block.namedExprsMgr.addNamedExprArray(normalizedResults[i].aggExprs);
+      block.namedExprsMgr.addNamedExprArray(normalizedResults[i].scalarExprs);
+    }
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Child Plan
+    ////////////////////////////////////////////////////////
+    stack.push(aggregation);
+    LogicalNode child = visit(context, stack, aggregation.getChild());
+    stack.pop();
+    ////////////////////////////////////////////////////////
+    GroupbyNode groupingNode = context.queryBlock.getNodeFromExpr(aggregation);
+    groupingNode.setChild(child);
+    groupingNode.setInSchema(child.getOutSchema());
+
+    // Set grouping sets
+    Column [] groupingColumns = new Column[aggregation.getGroupSet()[0].getGroupingSets().length];
+    for (int i = 0; i < groupingColumns.length; i++) {
+      if (block.namedExprsMgr.isEvaluated(groupingKeyRefNames[i])) {
+        groupingColumns[i] = block.namedExprsMgr.getTarget(groupingKeyRefNames[i]).getNamedColumn();
+      } else {
+        throw new PlanningException("Each grouping column expression must be a scalar expression.");
+      }
+    }
+    groupingNode.setGroupingColumns(groupingColumns);
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Child Plan
+    ////////////////////////////////////////////////////////
+
+    // create EvalNodes and check if each EvalNode can be evaluated here.
+    List<String> aggEvalNames = TUtil.newList();
+    List<AggregationFunctionCallEval> aggEvalNodes = TUtil.newList();
+    boolean includeDistinctFunction = false;
+    for (Iterator<NamedExpr> iterator = block.namedExprsMgr.getIteratorForUnevaluatedExprs(); iterator.hasNext();) {
+      NamedExpr namedExpr = iterator.next();
+      try {
+        includeDistinctFunction |= PlannerUtil.existsDistinctAggregationFunction(namedExpr.getExpr());
+        EvalNode evalNode = exprAnnotator.createEvalNode(context.plan, context.queryBlock, namedExpr.getExpr());
+        if (evalNode.getType() == EvalType.AGG_FUNCTION) {
+          block.namedExprsMgr.markAsEvaluated(namedExpr.getAlias(), evalNode);
+          aggEvalNames.add(namedExpr.getAlias());
+          aggEvalNodes.add((AggregationFunctionCallEval) evalNode);
+        }
+      } catch (VerifyException ve) {
+      }
+    }
+    // if there is at least one distinct aggregation function
+    groupingNode.setDistinct(includeDistinctFunction);
+    groupingNode.setAggFunctions(aggEvalNodes.toArray(new AggregationFunctionCallEval[aggEvalNodes.size()]));
+
+    Target [] targets = new Target[groupingKeyNum + aggEvalNames.size()];
+
+    // In target, grouping columns will be followed by aggregation evals.
+    //
+    // col1, col2, col3,   sum(..),  agv(..)
+    // ^^^^^^^^^^^^^^^    ^^^^^^^^^^^^^^^^^^
+    //  grouping keys      aggregation evals
+
+    // Build grouping keys
+    for (int i = 0; i < groupingKeyNum; i++) {
+      Target target = block.namedExprsMgr.getTarget(groupingNode.getGroupingColumns()[i].getQualifiedName());
+      targets[i] = target;
+    }
+
+    for (int i = 0, targetIdx = groupingKeyNum; i < aggEvalNodes.size(); i++, targetIdx++) {
+      targets[targetIdx] = block.namedExprsMgr.getTarget(aggEvalNames.get(i));
+    }
+
+    groupingNode.setTargets(targets);
+    block.unsetAggregationRequire();
+
+    verifyProjectedFields(block, groupingNode);
+    return groupingNode;
+  }
+
+  private static final Column[] ALL= Lists.newArrayList().toArray(new Column[0]);
+
+  public static List<Column[]> generateCuboids(Column[] columns) {
+    int numCuboids = (int) Math.pow(2, columns.length);
+    int maxBits = columns.length;
+
+    List<Column[]> cube = Lists.newArrayList();
+    List<Column> cuboidCols;
+
+    cube.add(ALL);
+    for (int cuboidId = 1; cuboidId < numCuboids; cuboidId++) {
+      cuboidCols = Lists.newArrayList();
+      for (int j = 0; j < maxBits; j++) {
+        int bit = 1 << j;
+        if ((cuboidId & bit) == bit) {
+          cuboidCols.add(columns[j]);
+        }
+      }
+      cube.add(cuboidCols.toArray(new Column[cuboidCols.size()]));
+    }
+    return cube;
+  }
+
+  @Override
+  public SelectionNode visitFilter(PlanContext context, Stack<Expr> stack, Selection selection)
+      throws PlanningException {
+    QueryBlock block = context.queryBlock;
+
+    ExprNormalizedResult normalizedResult = normalizer.normalize(context, selection.getQual());
+    block.namedExprsMgr.addExpr(normalizedResult.baseExpr);
+    if (normalizedResult.aggExprs.size() > 0 || normalizedResult.scalarExprs.size() > 0) {
+      throw new VerifyException("Filter condition cannot include aggregation function");
+    }
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Child Plan
+    ////////////////////////////////////////////////////////
+    stack.push(selection);
+    LogicalNode child = visit(context, stack, selection.getChild());
+    stack.pop();
+    ////////////////////////////////////////////////////////
+
+    SelectionNode selectionNode = context.queryBlock.getNodeFromExpr(selection);
+    selectionNode.setChild(child);
+    selectionNode.setInSchema(child.getOutSchema());
+    selectionNode.setOutSchema(child.getOutSchema());
+
+    // Create EvalNode for a search condition.
+    EvalNode searchCondition = exprAnnotator.createEvalNode(context.plan, block, selection.getQual());
+    EvalNode simplified = AlgebraicUtil.eliminateConstantExprs(searchCondition);
+    // set selection condition
+    selectionNode.setQual(simplified);
+
+    return selectionNode;
+  }
+
+  /*===============================================================================================
+    JOIN SECTION
+   ===============================================================================================*/
+
+  @Override
+  public LogicalNode visitJoin(PlanContext context, Stack<Expr> stack, Join join)
+      throws PlanningException {
+    // Phase 1: Init
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    if (join.hasQual()) {
+      ExprNormalizedResult normalizedResult = normalizer.normalize(context, join.getQual());
+      block.namedExprsMgr.addExpr(normalizedResult.baseExpr);
+      if (normalizedResult.aggExprs.size() > 0 || normalizedResult.scalarExprs.size() > 0) {
+        throw new VerifyException("Filter condition cannot include aggregation function");
+      }
+    }
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Child Plan
+    ////////////////////////////////////////////////////////
+    stack.push(join);
+    LogicalNode left = visit(context, stack, join.getLeft());
+    LogicalNode right = visit(context, stack, join.getRight());
+    stack.pop();
+    ////////////////////////////////////////////////////////
+
+    JoinNode joinNode = context.queryBlock.getNodeFromExpr(join);
+    joinNode.setJoinType(join.getJoinType());
+    joinNode.setLeftChild(left);
+    joinNode.setRightChild(right);
+
+    // Set A merged input schema
+    Schema merged;
+    if (join.isNatural()) {
+      merged = getNaturalJoinSchema(left, right);
+    } else {
+      merged = SchemaUtil.merge(left.getOutSchema(), right.getOutSchema());
+    }
+    joinNode.setInSchema(merged);
+
+    // Create EvalNode for a search condition.
+    EvalNode joinCondition = null;
+    if (join.hasQual()) {
+      EvalNode evalNode = exprAnnotator.createEvalNode(context.plan, block, join.getQual());
+      joinCondition = AlgebraicUtil.eliminateConstantExprs(evalNode);
+    }
+
+    List<String> newlyEvaluatedExprs = getNewlyEvaluatedExprsForJoin(plan, block, joinNode, stack);
+    List<Target> targets = TUtil.newList(PlannerUtil.schemaToTargets(merged));
+
+    for (String newAddedExpr : newlyEvaluatedExprs) {
+      targets.add(block.namedExprsMgr.getTarget(newAddedExpr, true));
+    }
+    joinNode.setTargets(targets.toArray(new Target[targets.size()]));
+
+    // Determine join conditions
+    if (join.isNatural()) { // if natural join, it should have the equi-join conditions by common column names
+      EvalNode njCond = getNaturalJoinCondition(joinNode);
+      joinNode.setJoinQual(njCond);
+    } else if (join.hasQual()) { // otherwise, the given join conditions are set
+      joinNode.setJoinQual(joinCondition);
+    }
+
+    return joinNode;
+  }
+
+  private List<String> getNewlyEvaluatedExprsForJoin(LogicalPlan plan, QueryBlock block, JoinNode joinNode,
+                                                   Stack<Expr> stack) {
+    EvalNode evalNode;
+    List<String> newlyEvaluatedExprs = TUtil.newList();
+    for (Iterator<NamedExpr> it = block.namedExprsMgr.getIteratorForUnevaluatedExprs(); it.hasNext();) {
+      NamedExpr namedExpr = it.next();
+      try {
+        evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
+        if (LogicalPlanner.checkIfBeEvaluatedAtJoin(block, evalNode, joinNode, stack.peek().getType() != OpType.Join)) {
+          block.namedExprsMgr.markAsEvaluated(namedExpr.getAlias(), evalNode);
+          newlyEvaluatedExprs.add(namedExpr.getAlias());
+        }
+      } catch (VerifyException ve) {} catch (PlanningException e) {
+        e.printStackTrace();
+      }
+    }
+    return newlyEvaluatedExprs;
+  }
+
+  private static Schema getNaturalJoinSchema(LogicalNode left, LogicalNode right) {
+    Schema joinSchema = new Schema();
+    Schema commons = SchemaUtil.getNaturalJoinColumns(left.getOutSchema(), right.getOutSchema());
+    joinSchema.addColumns(commons);
+    for (Column c : left.getOutSchema().getColumns()) {
+      if (!joinSchema.contains(c.getQualifiedName())) {
+        joinSchema.addColumn(c);
+      }
+    }
+
+    for (Column c : right.getOutSchema().getColumns()) {
+      if (!joinSchema.contains(c.getQualifiedName())) {
+        joinSchema.addColumn(c);
+      }
+    }
+    return joinSchema;
+  }
+
+  private static EvalNode getNaturalJoinCondition(JoinNode joinNode) {
+    Schema leftSchema = joinNode.getLeftChild().getInSchema();
+    Schema rightSchema = joinNode.getRightChild().getInSchema();
+    Schema commons = SchemaUtil.getNaturalJoinColumns(leftSchema, rightSchema);
+
+    EvalNode njQual = null;
+    EvalNode equiQual;
+    Column leftJoinKey;
+    Column rightJoinKey;
+
+    for (Column common : commons.getColumns()) {
+      leftJoinKey = leftSchema.getColumn(common.getQualifiedName());
+      rightJoinKey = rightSchema.getColumn(common.getQualifiedName());
+      equiQual = new BinaryEval(EvalType.EQUAL,
+          new FieldEval(leftJoinKey), new FieldEval(rightJoinKey));
+      if (njQual == null) {
+        njQual = equiQual;
+      } else {
+        njQual = new BinaryEval(EvalType.AND, njQual, equiQual);
+      }
+    }
+
+    return njQual;
+  }
+
+  private LogicalNode createCartesianProduct(PlanContext context, LogicalNode left, LogicalNode right)
+      throws PlanningException {
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    Schema merged = SchemaUtil.merge(left.getOutSchema(), right.getOutSchema());
+    JoinNode join = plan.createNode(JoinNode.class);
+    join.init(JoinType.CROSS, left, right);
+    join.setInSchema(merged);
+
+    EvalNode evalNode;
+    List<String> newlyEvaluatedExprs = TUtil.newList();
+    for (Iterator<NamedExpr> it = block.namedExprsMgr.getIteratorForUnevaluatedExprs(); it.hasNext();) {
+      NamedExpr namedExpr = it.next();
+      try {
+        evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
+        if (EvalTreeUtil.findDistinctAggFunction(evalNode).size() == 0) {
+          block.namedExprsMgr.markAsEvaluated(namedExpr.getAlias(), evalNode);
+          newlyEvaluatedExprs.add(namedExpr.getAlias());
+        }
+      } catch (VerifyException ve) {}
+    }
+
+    List<Target> targets = TUtil.newList(PlannerUtil.schemaToTargets(merged));
+    for (String newAddedExpr : newlyEvaluatedExprs) {
+      targets.add(block.namedExprsMgr.getTarget(newAddedExpr, true));
+    }
+    join.setTargets(targets.toArray(new Target[targets.size()]));
+    return join;
+  }
+
+  @Override
+  public LogicalNode visitRelationList(PlanContext context, Stack<Expr> stack, RelationList relations)
+      throws PlanningException {
+
+    LogicalNode current = visit(context, stack, relations.getRelations()[0]);
+
+    LogicalNode left;
+    LogicalNode right;
+    if (relations.size() > 1) {
+
+      for (int i = 1; i < relations.size(); i++) {
+        left = current;
+        right = visit(context, stack, relations.getRelations()[i]);
+        current = createCartesianProduct(context, left, right);
+      }
+    }
+    context.queryBlock.registerNode(current);
+
+    return current;
+  }
+
+  @Override
+  public ScanNode visitRelation(PlanContext context, Stack<Expr> stack, Relation expr)
+      throws PlanningException {
+    QueryBlock block = context.queryBlock;
+
+    ScanNode scanNode = block.getNodeFromExpr(expr);
+    updatePhysicalInfo(scanNode.getTableDesc());
+
+    // Find expression which can be evaluated at this relation node.
+    // Except for column references, additional expressions used in select list, where clause, order-by clauses
+    // can be evaluated here. Their reference names are kept in newlyEvaluatedExprsRef.
+    Set<String> newlyEvaluatedExprsReferences = new LinkedHashSet<String>();
+    for (Iterator<NamedExpr> iterator = block.namedExprsMgr.getIteratorForUnevaluatedExprs(); iterator.hasNext();) {
+      NamedExpr rawTarget = iterator.next();
+      try {
+        EvalNode evalNode = exprAnnotator.createEvalNode(context.plan, context.queryBlock, rawTarget.getExpr());
+        if (checkIfBeEvaluatedAtRelation(block, evalNode, scanNode)) {
+          block.namedExprsMgr.markAsEvaluated(rawTarget.getAlias(), evalNode);
+          newlyEvaluatedExprsReferences.add(rawTarget.getAlias()); // newly added exr
+        }
+      } catch (VerifyException ve) {
+      }
+    }
+
+    // Assume that each unique expr is evaluated once.
+    LinkedHashSet<Target> targets = createFieldTargetsFromRelation(block, scanNode, newlyEvaluatedExprsReferences);
+
+    // The fact the some expr is included in newlyEvaluatedExprsReferences means that it is already evaluated.
+    // So, we get a raw expression and then creates a target.
+    for (String reference : newlyEvaluatedExprsReferences) {
+      NamedExpr refrrer = block.namedExprsMgr.getNamedExpr(reference);
+      EvalNode evalNode = exprAnnotator.createEvalNode(context.plan, block, refrrer.getExpr());
+      targets.add(new Target(evalNode, reference));
+    }
+
+    scanNode.setTargets(targets.toArray(new Target[targets.size()]));
+
+    verifyProjectedFields(block, scanNode);
+    return scanNode;
+  }
+
+  private static LinkedHashSet<Target> createFieldTargetsFromRelation(QueryBlock block, RelationNode relationNode,
+                                                      Set<String> newlyEvaluatedRefNames) {
+    LinkedHashSet<Target> targets = Sets.newLinkedHashSet();
+    for (Column column : relationNode.getTableSchema().getColumns()) {
+      String aliasName = block.namedExprsMgr.checkAndGetIfAliasedColumn(column.getQualifiedName());
+      if (aliasName != null) {
+        targets.add(new Target(new FieldEval(column), aliasName));
+        newlyEvaluatedRefNames.remove(aliasName);
+      } else {
+        targets.add(new Target(new FieldEval(column)));
+      }
+    }
+    return targets;
+  }
+
+  private void updatePhysicalInfo(TableDesc desc) {
+    if (desc.getPath() != null) {
+      try {
+        FileSystem fs = desc.getPath().getFileSystem(new Configuration());
+        FileStatus status = fs.getFileStatus(desc.getPath());
+        if (desc.getStats() != null && (status.isDirectory() || status.isFile())) {
+          ContentSummary summary = fs.getContentSummary(desc.getPath());
+          if (summary != null) {
+            long volume = summary.getLength();
+            desc.getStats().setNumBytes(volume);
+          }
+        }
+      } catch (Throwable t) {
+        LOG.warn(t);
+      }
+    }
+  }
+
+  public TableSubQueryNode visitTableSubQuery(PlanContext context, Stack<Expr> stack, TablePrimarySubQuery expr)
+      throws PlanningException {
+    QueryBlock block = context.queryBlock;
+
+    QueryBlock childBlock = context.plan.getBlock(context.plan.getBlockNameByExpr(expr.getSubQuery()));
+    PlanContext newContext = new PlanContext(context, childBlock);
+    LogicalNode child = visit(newContext, new Stack<Expr>(), expr.getSubQuery());
+    TableSubQueryNode subQueryNode = context.queryBlock.getNodeFromExpr(expr);
+    context.plan.connectBlocks(childBlock, context.queryBlock, BlockType.TableSubQuery);
+    subQueryNode.setSubQuery(child);
+
+    // Add additional expressions required in upper nodes.
+    Set<String> newlyEvaluatedExprs = TUtil.newHashSet();
+    for (NamedExpr rawTarget : block.namedExprsMgr.getAllNamedExprs()) {
+      try {
+        EvalNode evalNode = exprAnnotator.createEvalNode(context.plan, context.queryBlock, rawTarget.getExpr());
+        if (checkIfBeEvaluatedAtRelation(block, evalNode, subQueryNode)) {
+          block.namedExprsMgr.markAsEvaluated(rawTarget.getAlias(), evalNode);
+          newlyEvaluatedExprs.add(rawTarget.getAlias()); // newly added exr
+        }
+      } catch (VerifyException ve) {
+      }
+    }
+
+    // Assume that each unique expr is evaluated once.
+    LinkedHashSet<Target> targets = createFieldTargetsFromRelation(block, subQueryNode, newlyEvaluatedExprs);
+
+    for (String newAddedExpr : newlyEvaluatedExprs) {
+      targets.add(block.namedExprsMgr.getTarget(newAddedExpr, true));
+    }
+
+    subQueryNode.setTargets(targets.toArray(new Target[targets.size()]));
+
+    return subQueryNode;
+  }
+
+    /*===============================================================================================
+    SET OPERATION SECTION
+   ===============================================================================================*/
+
+  @Override
+  public LogicalNode visitUnion(PlanContext context, Stack<Expr> stack, SetOperation setOperation)
+      throws PlanningException {
+    return buildSetPlan(context, stack, setOperation);
+  }
+
+  @Override
+  public LogicalNode visitExcept(PlanContext context, Stack<Expr> stack, SetOperation setOperation)
+      throws PlanningException {
+    return buildSetPlan(context, stack, setOperation);
+  }
+
+  @Override
+  public LogicalNode visitIntersect(PlanContext context, Stack<Expr> stack, SetOperation setOperation)
+      throws PlanningException {
+    return buildSetPlan(context, stack, setOperation);
+  }
+
+  private LogicalNode buildSetPlan(PlanContext context, Stack<Expr> stack, SetOperation setOperation)
+      throws PlanningException {
+
+    // 1. Init Phase
+    LogicalPlan plan = context.plan;
+    QueryBlock block = context.queryBlock;
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Left Child Plan
+    ////////////////////////////////////////////////////////
+    QueryBlock leftBlock = context.plan.getBlockByExpr(setOperation.getLeft());
+    PlanContext leftContext = new PlanContext(context, leftBlock);
+    stack.push(setOperation);
+    LogicalNode leftChild = visit(leftContext, new Stack<Expr>(), setOperation.getLeft());
+    stack.pop();
+    // Connect left child and current blocks
+    context.plan.connectBlocks(leftContext.queryBlock, context.queryBlock, BlockType.TableSubQuery);
+
+    ////////////////////////////////////////////////////////
+    // Visit and Build Right Child Plan
+    ////////////////////////////////////////////////////////
+    QueryBlock rightBlock = context.plan.getBlockByExpr(setOperation.getRight());
+    PlanContext rightContext = new PlanContext(context, rightBlock);
+    stack.push(setOperation);
+    LogicalNode rightChild = visit(rightContext, new Stack<Expr>(), setOperation.getRight());
+    stack.pop();
+    // Connect right child and current blocks
+    context.plan.connectBlocks(rightContext.queryBlock, context.queryBlock, BlockType.TableSubQuery);
+
+    BinaryNode setOp;
+    if (setOperation.getType() == OpType.Union) {
+      setOp = block.getNodeFromExpr(setOperation);
+    } else if (setOperation.getType() == OpType.Except) {
+      setOp = block.getNodeFromExpr(setOperation);
+    } else if (setOperation.getType() == OpType.Intersect) {
+      setOp = block.getNodeFromExpr(setOperation);
+    } else {
+      throw new VerifyException("Invalid Type: " + setOperation.getType());
+    }
+    setOp.setLeftChild(leftChild);
+    setOp.setRightChild(rightChild);
+
+    // An union statement can be derived from two query blocks.
+    // For one union statement between both relations, we can ensure that each corresponding data domain of both
+    // relations are the same. However, if necessary, the schema of left query block will be used as a base schema.
+    Target [] leftStrippedTargets = PlannerUtil.stripTarget(
+        PlannerUtil.schemaToTargets(leftBlock.getRoot().getOutSchema()));
+
+    setOp.setInSchema(leftChild.getOutSchema());
+    Schema outSchema = PlannerUtil.targetToSchema(leftStrippedTargets);
+    setOp.setOutSchema(outSchema);
+
+    return setOp;
+  }
+
+  /*===============================================================================================
+    INSERT SECTION
+   ===============================================================================================*/
+
+  public LogicalNode visitInsert(PlanContext context, Stack<Expr> stack, Insert expr) throws PlanningException {
+    stack.push(expr);
+    LogicalNode subQuery = super.visitInsert(context, stack, expr);
+    stack.pop();
+
+    InsertNode insertNode = context.queryBlock.getNodeFromExpr(expr);
+    insertNode.setOverwrite(expr.isOverwrite());
+    insertNode.setSubQuery(subQuery);
+
+    if (expr.hasTableName()) { // INSERT (OVERWRITE) INTO TABLE ...
+      return buildInsertIntoTablePlan(context, insertNode, expr);
+    } else if (expr.hasLocation()) { // INSERT (OVERWRITE) INTO LOCATION ...
+      return buildInsertIntoLocationPlan(context, insertNode, expr);
+    } else {
+      throw new IllegalStateException("Invalid Query");
+    }
+  }
+
+  /**
+   * Builds a InsertNode with a target table.
+   *
+   * ex) INSERT OVERWRITE INTO TABLE ...
+   * <br />
+   *
+   * We use the following terms, such target table, target column
+   * <pre>
+   * INSERT INTO    TB_NAME        (col1, col2)          SELECT    c1,   c2        FROM ...
+   *                ^^^^^^^        ^^^^^^^^^^^^                  ^^^^^^^^^^^^
+   *             target table   target columns (or schema)     projected columns (or schema)
+   * </pre>
+   */
+  private InsertNode buildInsertIntoTablePlan(PlanContext context, InsertNode insertNode, Insert expr)
+      throws PlanningException {
+    // Get and set a target table
+    TableDesc desc = catalog.getTableDesc(context.session.getCurrentDatabase(), expr.getTableName());
+    insertNode.setTargetTable(desc);
+
+    //
+    // When we use 'INSERT (OVERWIRTE) INTO TABLE statements, there are two cases.
+    //
+    // First, when a user specified target columns
+    // INSERT (OVERWRITE)? INTO table_name (col1 type, col2 type) SELECT ...
+    //
+    // Second, when a user do not specified target columns
+    // INSERT (OVERWRITE)? INTO table_name SELECT ...
+    //
+    // In the former case is, target columns' schema and corresponding projected columns' schema
+    // must be equivalent or be available to cast implicitly.
+    //
+    // In the later case, the target table's schema and projected column's
+    // schema of select clause can be different to each other. In this case,
+    // we use only a sequence of preceding columns of target table's schema
+    // as target columns.
+    //
+    // For example, consider a target table and an 'insert into' query are give as follows:
+    //
+    // CREATE TABLE TB1                  (col1 int,  col2 int, col3 long);
+    //                                      ||          ||
+    // INSERT OVERWRITE INTO TB1 SELECT  order_key,  part_num               FROM ...
+    //
+    // In this example, only col1 and col2 are used as target columns.
+
+    if (expr.hasTargetColumns()) { // when a user specified target columns
+
+      if (expr.getTargetColumns().length > insertNode.getChild().getOutSchema().size()) {
+        throw new PlanningException("Target columns and projected columns are mismatched to each other");
+      }
+
+      // See PreLogicalPlanVerifier.visitInsert.
+      // It guarantees that the equivalence between the numbers of target and projected columns.
+      ScanNode scanNode = context.plan.createNode(ScanNode.class);
+      scanNode.init(desc);
+      context.queryBlock.addRelation(scanNode);
+      String [] targets = expr.getTargetColumns();
+      Schema targetColumns = new Schema();
+      for (int i = 0; i < targets.length; i++) {
+        Column targetColumn = context.plan.resolveColumn(context.queryBlock, new ColumnReferenceExpr(targets[i]));
+        targetColumns.addColumn(targetColumn);
+      }
+      insertNode.setTargetSchema(targetColumns);
+      insertNode.setOutSchema(targetColumns);
+      buildProjectedInsert(insertNode);
+
+    } else { // when a user do not specified target columns
+
+      // The output schema of select clause determines the target columns.
+      Schema tableSchema = desc.getLogicalSchema();
+      Schema projectedSchema = insertNode.getChild().getOutSchema();
+
+      Schema targetColumns = new Schema();
+      for (int i = 0; i < projectedSchema.size(); i++) {
+        targetColumns.addColumn(tableSchema.getColumn(i));
+      }
+      insertNode.setTargetSchema(targetColumns);
+      buildProjectedInsert(insertNode);
+    }
+
+    if (desc.hasPartition()) {
+      insertNode.setPartitionMethod(desc.getPartitionMethod());
+    }
+    return insertNode;
+  }
+
+  private void buildProjectedInsert(InsertNode insertNode) {
+    Schema tableSchema = insertNode.getTableSchema();
+    Schema targetColumns = insertNode.getTargetSchema();
+
+    ProjectionNode projectionNode = insertNode.getChild();
+
+    // Modifying projected columns by adding NULL constants
+    // It is because that table appender does not support target columns to be written.
+    List<Target> targets = TUtil.newList();
+    for (int i = 0, j = 0; i < tableSchema.size(); i++) {
+      Column column = tableSchema.getColumn(i);
+
+      if(targetColumns.contains(column) && j < projectionNode.getTargets().length) {
+        targets.add(projectionNode.getTargets()[j++]);
+      } else {
+        targets.add(new Target(new ConstEval(NullDatum.get()), column.getSimpleName()));
+      }
+    }
+    projectionNode.setTargets(targets.toArray(new Target[targets.size()]));
+
+    insertNode.setInSchema(projectionNode.getOutSchema());
+    insertNode.setOutSchema(projectionNode.getOutSchema());
+    insertNode.setProjectedSchema(PlannerUtil.targetToSchema(targets));
+  }
+
+  /**
+   * Build a InsertNode with a location.
+   *
+   * ex) INSERT OVERWRITE INTO LOCATION 'hdfs://....' ..
+   */
+  private InsertNode buildInsertIntoLocationPlan(PlanContext context, InsertNode insertNode, Insert expr) {
+    // INSERT (OVERWRITE)? INTO LOCATION path (USING file_type (param_clause)?)? query_expression
+
+    Schema childSchema = insertNode.getChild().getOutSchema();
+    insertNode.setInSchema(childSchema);
+    insertNode.setOutSchema(childSchema);
+    insertNode.setTableSchema(childSchema);
+    insertNode.setTargetLocation(new Path(expr.getLocation()));
+
+    if (expr.hasStorageType()) {
+      insertNode.setStorageType(CatalogUtil.getStoreType(expr.getStorageType()));
+    }
+    if (expr.hasParams()) {
+      Options options = new Options();
+      options.putAll(expr.getParams());
+      insertNode.setOptions(options);
+    }
+    return insertNode;
+  }
+
+  /*===============================================================================================
+    Data Definition Language (DDL) SECTION
+   ===============================================================================================*/
+
+  @Override
+  public LogicalNode visitCreateDatabase(PlanContext context, Stack<Expr> stack, CreateDatabase expr)
+      throws PlanningException {
+    CreateDatabaseNode createDatabaseNode = context.queryBlock.getNodeFromExpr(expr);
+    createDatabaseNode.init(expr.getDatabaseName(), expr.isIfNotExists());
+    return createDatabaseNode;
+  }
+
+  @Override
+  public LogicalNode visitDropDatabase(PlanContext context, Stack<Expr> stack, DropDatabase expr)
+      throws PlanningException {
+    DropDatabaseNode dropDatabaseNode = context.plan.createNode(DropDatabaseNode.class);
+    dropDatabaseNode.init(expr.getDatabaseName(), expr.isIfExists());
+    return dropDatabaseNode;
+  }
+
+  @Override
+  public LogicalNode visitCreateTable(PlanContext context, Stack<Expr> stack, CreateTable expr)
+      throws PlanningException {
+
+    CreateTableNode createTableNode = context.queryBlock.getNodeFromExpr(expr);
+    createTableNode.setIfNotExists(expr.isIfNotExists());
+
+    // Set a table name to be created.
+    if (CatalogUtil.isFQTableName(expr.getTableName())) {
+      createTableNode.setTableName(expr.getTableName());
+    } else {
+      createTableNode.setTableName(
+          CatalogUtil.buildFQName(context.session.getCurrentDatabase(), expr.getTableName()));
+    }
+
+
+    if (expr.hasStorageType()) { // If storage type (using clause) is specified
+      createTableNode.setStorageType(CatalogUtil.getStoreType(expr.getStorageType()));
+    } else { // otherwise, default type
+      createTableNode.setStorageType(CatalogProtos.StoreType.CSV);
+    }
+
+    // Set default storage properties to be created.
+    Options options = StorageUtil.newPhysicalProperties(createTableNode.getStorageType());
+    if (expr.hasParams()) {
+      options.putAll(expr.getParams());
+    }
+
+    createTableNode.setOptions(options);
+
+    if (expr.hasPartition()) {
+      if (expr.getPartitionMethod().getPartitionType().equals(PartitionType.COLUMN)) {
+        createTableNode.setPartitionMethod(getPartitionMethod(context, expr.getTableName(), expr.getPartitionMethod()));
+      } else {
+        throw new PlanningException(String.format("Not supported PartitonType: %s",
+            expr.getPartitionMethod().getPartitionType()));
+      }
+    }
+
+    if (expr.hasSubQuery()) { // CREATE TABLE .. AS SELECT
+      stack.add(expr);
+      LogicalNode subQuery = visit(context, stack, expr.getSubQuery());
+      stack.pop();
+      createTableNode.setChild(subQuery);
+      createTableNode.setInSchema(subQuery.getOutSchema());
+
+      // If the table schema is defined
+      // ex) CREATE TABLE tbl(col1 type, col2 type) AS SELECT ...
+      if (expr.hasTableElements()) {
+        createTableNode.setOutSchema(convertTableElementsSchema(expr.getTableElements()));
+        createTableNode.setTableSchema(convertTableElementsSchema(expr.getTableElements()));
+      } else {
+        // if no table definition, the select clause's output schema will be used.
+        // ex) CREATE TABLE tbl AS SELECT ...
+
+        if (expr.hasPartition()) {
+          PartitionMethodDesc partitionMethod = createTableNode.getPartitionMethod();
+
+          Schema queryOutputSchema = subQuery.getOutSchema();
+          Schema partitionExpressionSchema = partitionMethod.getExpressionSchema();
+          if (partitionMethod.getPartitionType() == CatalogProtos.PartitionType.COLUMN &&
+              queryOutputSchema.size() < partitionExpressionSchema.size()) {
+            throw new VerifyException("Partition columns cannot be more than table columns.");
+          }
+          Schema tableSchema = new Schema();
+          for (int i = 0; i < queryOutputSchema.size() - partitionExpressionSchema.size(); i++) {
+            tableSchema.addColumn(queryOutputSchema.getColumn(i));
+          }
+          createTableNode.setOutSchema(tableSchema);
+          createTableNode.setTableSchema(tableSchema);
+        } else {
+          createTableNode.setOutSchema(subQuery.getOutSchema());
+          createTableNode.setTableSchema(subQuery.getOutSchema());
+        }
+      }
+
+      return createTableNode;
+
+    } else { // if CREATE AN EMPTY TABLE
+      Schema tableSchema = convertColumnsToSchema(expr.getTableElements());
+      createTableNode.setTableSchema(tableSchema);
+
+      if (expr.isExternal()) {
+        createTableNode.setExternal(true);
+      }
+
+      if (expr.hasLocation()) {
+        createTableNode.setPath(new Path(expr.getLocation()));
+      }
+
+      return createTableNode;
+    }
+  }
+
+  private PartitionMethodDesc getPartitionMethod(PlanContext context,
+                                                 String tableName,
+                                                 CreateTable.PartitionMethodDescExpr expr) throws PlanningException {
+    PartitionMethodDesc partitionMethodDesc;
+
+    if(expr.getPartitionType() == PartitionType.COLUMN) {
+      CreateTable.ColumnPartition partition = (CreateTable.ColumnPartition) expr;
+      String partitionExpression = Joiner.on(',').join(partition.getColumns());
+
+      partitionMethodDesc = new PartitionMethodDesc(context.session.getCurrentDatabase(), tableName,
+          CatalogProtos.PartitionType.COLUMN, partitionExpression, convertColumnsToSchema(partition.getColumns()));
+    } else {
+      throw new PlanningException(String.format("Not supported PartitonType: %s", expr.getPartitionType()));
+    }
+    return partitionMethodDesc;
+  }
+
+  /**
+   * It transforms table definition elements to schema.
+   *
+   * @param elements to be transformed
+   * @return schema transformed from table definition elements
+   */
+  private Schema convertColumnsToSchema(ColumnDefinition[] elements) {
+    Schema schema = new Schema();
+
+    for (ColumnDefinition columnDefinition: elements) {
+      schema.addColumn(convertColumn(columnDefinition));
+    }
+
+    return schema;
+  }
+
+  /**
+   * It transforms table definition elements to schema.
+   *
+   * @param elements to be transformed
+   * @return schema transformed from table definition elements
+   */
+  private Schema convertTableElementsSchema(ColumnDefinition[] elements) {
+    Schema schema = new Schema();
+
+    for (ColumnDefinition columnDefinition: elements) {
+      schema.addColumn(convertColumn(columnDefinition));
+    }
+
+    return schema;
+  }
+
+  private Column convertColumn(ColumnDefinition columnDefinition) {
+    return new Column(columnDefinition.getColumnName(), convertDataType(columnDefinition));
+  }
+
+  static TajoDataTypes.DataType convertDataType(DataTypeExpr dataType) {
+    TajoDataTypes.Type type = TajoDataTypes.Type.valueOf(dataType.getTypeName());
+
+    TajoDataTypes.DataType.Builder builder = TajoDataTypes.DataType.newBuilder();
+    builder.setType(type);
+    if (dataType.hasLengthOrPrecision()) {
+      builder.setLength(dataType.getLengthOrPrecision());
+    }
+    return builder.build();
+  }
+
+
+  @Override
+  public LogicalNode visitDropTable(PlanContext context, Stack<Expr> stack, DropTable dropTable) {
+    DropTableNode dropTableNode = context.queryBlock.getNodeFromExpr(dropTable);
+    String qualified;
+    if (CatalogUtil.isFQTableName(dropTable.getTableName())) {
+      qualified = dropTable.getTableName();
+    } else {
+      qualified = CatalogUtil.buildFQName(context.session.getCurrentDatabase(), dropTable.getTableName());
+    }
+    dropTableNode.init(qualified, dropTable.isIfExists(), dropTable.isPurge());
+    return dropTableNode;
+  }
+
+  public LogicalNode visitAlterTablespace(PlanContext context, Stack<Expr> stack, AlterTablespace alterTablespace) {
+    AlterTablespaceNode alter = context.queryBlock.getNodeFromExpr(alterTablespace);
+    alter.setTablespaceName(alterTablespace.getTablespaceName());
+    alter.setLocation(alterTablespace.getLocation());
+    return alter;
+  }
+
+  @Override
+  public LogicalNode visitAlterTable(PlanContext context, Stack<Expr> stack, AlterTable alterTable) {
+    AlterTableNode alterTableNode = context.queryBlock.getNodeFromExpr(alterTable);
+    alterTableNode.setTableName(alterTable.getTableName());
+    alterTableNode.setNewTableName(alterTable.getNewTableName());
+    alterTableNode.setColumnName(alterTable.getColumnName());
+    alterTableNode.setNewColumnName(alterTable.getNewColumnName());
+
+    if (null != alterTable.getAddNewColumn()) {
+      alterTableNode.setAddNewColumn(convertColumn(alterTable.getAddNewColumn()));
+    }
+    alterTableNode.setAlterTableOpType(alterTable.getAlterTableOpType());
+    return alterTableNode;
+  }
+
+  /*===============================================================================================
+    Util SECTION
+  ===============================================================================================*/
+
+  public static boolean checkIfBeEvaluatedAtGroupBy(EvalNode evalNode, GroupbyNode node) {
+    Set<Column> columnRefs = EvalTreeUtil.findUniqueColumns(evalNode);
+
+    if (columnRefs.size() > 0 && !node.getInSchema().containsAll(columnRefs)) {
+      return false;
+    }
+
+    return true;
+  }
+
+  public static boolean checkIfBeEvaluatedAtJoin(QueryBlock block, EvalNode evalNode, JoinNode node,
+                                                 boolean isTopMostJoin) {
+    Set<Column> columnRefs = EvalTreeUtil.findUniqueColumns(evalNode);
+
+    if (EvalTreeUtil.findDistinctAggFunction(evalNode).size() > 0) {
+      return false;
+    }
+
+    if (columnRefs.size() > 0 && !node.getInSchema().containsAll(columnRefs)) {
+      return false;
+    }
+
+    // When a 'case-when' is used with outer join, the case-when expression must be evaluated
+    // at the topmost join operator.
+    // TODO - It's also valid that case-when is evalauted at the topmost outer operator.
+    //        But, how can we know there is no further outer join operator after this node?
+    if (!checkIfCaseWhenWithOuterJoinBeEvaluated(block, evalNode, isTopMostJoin)) {
+      return false;
+    }
+
+    return true;
+  }
+
+  private static boolean checkIfCaseWhenWithOuterJoinBeEvaluated(QueryBlock block, EvalNode evalNode,
+                                                                 boolean isTopMostJoin) {
+    if (block.containsJoinType(JoinType.LEFT_OUTER) || block.containsJoinType(JoinType.RIGHT_OUTER)) {
+      Collection<CaseWhenEval> caseWhenEvals = EvalTreeUtil.findEvalsByType(evalNode, EvalType.CASE);
+      if (caseWhenEvals.size() > 0 && !isTopMostJoin) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * It checks if evalNode can be evaluated at this @{link RelationNode}.
+   */
+  public static boolean checkIfBeEvaluatedAtRelation(QueryBlock block, EvalNode evalNode, RelationNode node) {
+    Set<Column> columnRefs = EvalTreeUtil.findUniqueColumns(evalNode);
+
+    // aggregation functions cannot be evaluated in scan node
+    if (EvalTreeUtil.findDistinctAggFunction(evalNode).size() > 0) {
+      return false;
+    }
+
+    if (columnRefs.size() > 0 && !node.getTableSchema().containsAll(columnRefs)) {
+      return false;
+    }
+
+    // Why? - When a {case when} is used with outer join, case when must be evaluated at topmost outer join.
+    if (block.containsJoinType(JoinType.LEFT_OUTER) || block.containsJoinType(JoinType.RIGHT_OUTER)) {
+      Collection<CaseWhenEval> found = EvalTreeUtil.findEvalsByType(evalNode, EvalType.CASE);
+      if (found.size() > 0) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  public static boolean checkIfBeEvaluatedAtThis(EvalNode evalNode, LogicalNode node) {
+    Set<Column> columnRefs = EvalTreeUtil.findUniqueColumns(evalNode);
+    if (columnRefs.size() > 0 && !node.getInSchema().containsAll(columnRefs)) {
+      return false;
+    }
+
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java
new file mode 100644
index 0000000..2fcf3bc
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java
@@ -0,0 +1,369 @@
+/**
+ * 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.planner;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.tajo.algebra.ColumnReferenceExpr;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.algebra.NamedExpr;
+import org.apache.tajo.algebra.OpType;
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.eval.EvalType;
+import org.apache.tajo.engine.eval.FieldEval;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+/**
+ * NamedExprsManager manages an expressions used in a query block. All expressions used in a query block must be
+ * added to NamedExprsManager. When an expression is added to NamedExprsManager, NamedExprsManager gives a reference
+ * to the expression. If the expression already has an alias name, it gives the alias name as the reference
+ * to the expression. If the expression does not have any alias, it gives a generated name as the reference to the
+ * expression. Usually, predicates in WHERE clause, expressions in GROUP-BY, ORDER-BY, LIMIT clauses are not given
+ * any alias name. Those reference names are used to identify an individual expression.
+ *
+ * NamedExprsManager only keeps unique expressions. Since expressions in a query block can be duplicated,
+ * one or more reference names can point one expressions. Due to this process, it naturally removes duplicated
+ * expression.
+ *
+ * As we mentioned above, one or more reference names can indicate one expression. Primary names are used for
+ * representing expressions. A primary name of an expression indicates the reference obtained when
+ * the expression is added firstly. All output schemas uses only primary names of expressions.
+ *
+ * Each expression that NamedExprsManager keeps has an boolean state to indicate whether the expression is evaluated
+ * or not. The <code>evaluated</code> state means that upper logical operators can access this expression like a column
+ * reference. For it, the reference name is used to access this expression like a column reference,
+ * The evaluated state is set with an EvalNode which is an annotated expression.
+ * {@link #getTarget(String)} returns EvalNodes by a reference name.
+ */
+public class NamedExprsManager {
+  /** a sequence id */
+  private int sequenceId = 0;
+
+  /** Map: Name -> ID. Two or more different names can indicates the same id. */
+  private LinkedHashMap<String, Integer> nameToIdMap = Maps.newLinkedHashMap();
+
+  /** Map; ID <-> EvalNode */
+  private BiMap<Integer, EvalNode> idToEvalMap = HashBiMap.create();
+
+  /** Map: ID -> Names */
+  private LinkedHashMap<Integer, List<String>> idToNamesMap = Maps.newLinkedHashMap();
+
+  /** Map: ID -> Expr */
+  private BiMap<Integer, Expr> idToExprBiMap = HashBiMap.create();
+
+  /** Map; Name -> Boolean (if it is resolved or not) */
+  private LinkedHashMap<Integer, Boolean> evaluationStateMap = Maps.newLinkedHashMap();
+
+  /** Map: Alias Name <-> Original Name */
+  private BiMap<String, String> aliasedColumnMap = HashBiMap.create();
+
+  private LogicalPlan plan;
+
+  public NamedExprsManager(LogicalPlan plan) {
+    this.plan = plan;
+  }
+
+  private int getNextId() {
+    return sequenceId++;
+  }
+
+  /**
+   * Check whether the expression corresponding to a given name was evaluated.
+   *
+   * @param name The name of a certain expression to be checked
+   * @return true if resolved. Otherwise, false.
+   */
+  public boolean isEvaluated(String name) {
+    if (nameToIdMap.containsKey(name)) {
+      int refId = nameToIdMap.get(name);
+      return evaluationStateMap.containsKey(refId) && evaluationStateMap.get(refId);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean contains(String name) {
+    return nameToIdMap.containsKey(name);
+  }
+
+  public boolean contains(Expr expr) {
+    return idToExprBiMap.inverse().containsKey(expr);
+  }
+
+  private Expr getExpr(String name) {
+    return idToExprBiMap.get(nameToIdMap.get(name));
+  }
+
+  public NamedExpr getNamedExpr(String name) {
+    String normalized = name;
+    return new NamedExpr(getExpr(name), normalized);
+  }
+
+  public boolean isAliased(String name) {
+    return aliasedColumnMap.containsKey(name);
+  }
+
+  public String getAlias(String originalName) {
+    return aliasedColumnMap.get(originalName);
+  }
+
+  public boolean isAliasedName(String aliasName) {
+    return aliasedColumnMap.inverse().containsKey(aliasName);
+  }
+
+  public String getOriginalName(String aliasName) {
+    return aliasedColumnMap.inverse().get(aliasName);
+  }
+
+  /**
+   * Adds an expression and returns a reference name.
+   */
+  public String addExpr(Expr expr) throws PlanningException {
+    if (idToExprBiMap.inverse().containsKey(expr)) {
+      int refId = idToExprBiMap.inverse().get(expr);
+      return idToNamesMap.get(refId).get(0);
+    }
+
+    String generatedName = plan.generateUniqueColumnName(expr);
+    return addExpr(expr, generatedName);
+  }
+
+  /**
+   * Adds an expression with an alias name and returns a reference name.
+   * It specifies the alias as an reference name.
+   */
+  public String addExpr(Expr expr, String alias) throws PlanningException {
+
+    // if this name already exists, just returns the name.
+    if (nameToIdMap.containsKey(alias)) {
+      return alias;
+    }
+
+    // if the name is first
+    int refId;
+    if (idToExprBiMap.inverse().containsKey(expr)) {
+      refId = idToExprBiMap.inverse().get(expr);
+    } else {
+      refId = getNextId();
+      idToExprBiMap.put(refId, expr);
+    }
+
+    nameToIdMap.put(alias, refId);
+    evaluationStateMap.put(refId, false);
+
+    // add the entry to idToNames map
+    TUtil.putToNestedList(idToNamesMap, refId, alias);
+
+    return alias;
+  }
+
+  /**
+   * Adds an expression and returns a reference name.
+   * If an alias is given, it specifies the alias as an reference name.
+   */
+  public String addNamedExpr(NamedExpr namedExpr) throws PlanningException {
+    if (namedExpr.hasAlias()) {
+      return addExpr(namedExpr.getExpr(), namedExpr.getAlias());
+    } else {
+      return addExpr(namedExpr.getExpr());
+    }
+  }
+
+  /**
+   * Adds a list of expressions and returns a list of reference names.
+   * If some NamedExpr has an alias, NamedExprsManager specifies the alias for the NamedExpr.
+   */
+  public String [] addNamedExprArray(@Nullable Collection<NamedExpr> namedExprs) throws PlanningException {
+    if (namedExprs != null && namedExprs.size() > 0) {
+      String [] names = new String[namedExprs.size()];
+      int i = 0;
+      for (NamedExpr target : namedExprs) {
+        names[i++] = addNamedExpr(target);
+      }
+      return names;
+    } else {
+      return null;
+    }
+  }
+
+  public Collection<NamedExpr> getAllNamedExprs() {
+    List<NamedExpr> namedExprList = Lists.newArrayList();
+    for (Map.Entry<Integer, Expr> entry: idToExprBiMap.entrySet()) {
+      namedExprList.add(new NamedExpr(entry.getValue(), idToNamesMap.get(entry.getKey()).get(0)));
+    }
+    return namedExprList;
+  }
+
+  /**
+   * It marks the expression identified by the reference name as <code>evaluated</code>.
+   * In addition, it adds an EvanNode for the expression identified by the reference.
+   *
+   * @param referenceName The reference name to be marked as 'evaluated'.
+   * @param evalNode EvalNode to be added.
+   */
+  public void markAsEvaluated(String referenceName, EvalNode evalNode) throws PlanningException {
+    String normalized = referenceName;
+
+    int refId = nameToIdMap.get(normalized);
+    evaluationStateMap.put(refId, true);
+    idToEvalMap.put(refId, evalNode);
+
+    String originalName = checkAndGetIfAliasedColumn(normalized);
+    if (originalName != null) {
+      aliasedColumnMap.put(originalName, normalized);
+    }
+  }
+
+  /**
+   * It returns an original column name if it is aliased column reference.
+   * Otherwise, it will return NULL.
+   */
+  public String checkAndGetIfAliasedColumn(String name) {
+    Expr expr = getExpr(name);
+    if (expr != null && expr.getType() == OpType.Column) {
+      ColumnReferenceExpr column = (ColumnReferenceExpr) expr;
+      if (!column.getCanonicalName().equals(name)) {
+        return column.getCanonicalName();
+      }
+    }
+    return null;
+  }
+
+  public Target getTarget(String name) {
+    return getTarget(name, false);
+  }
+
+  /**
+   * It checks if a given name is the primary name.
+   *
+   * @See {@link NamedExprsManager}
+   * @see {@link NamedExprsManager#getPrimaryName}
+   *
+   * @param id The expression id
+   * @param name The name to be checked if it is primary name.
+   * @return The primary name
+   */
+  private boolean isPrimaryName(int id, String name) {
+    return idToNamesMap.get(id).get(0).equals(name);
+  }
+
+  /**
+   * One or more reference names can indicate one expression. Primary names are used for
+   * representing expressions. A primary name of an expression indicates the reference obtained when
+   * the expression is added firstly. All output schemas uses only primary names of expressions.
+   *
+   * @param id The expression id
+   * @return The primary name
+   */
+  private String getPrimaryName(int id) {
+    return idToNamesMap.get(id).get(0);
+  }
+
+  /**
+   * get a Target instance. A target consists of a reference name and an EvalNode corresponding to the reference name.
+   * According to evaluation state, it returns different EvalNodes.
+   * If the expression corresponding to the reference name is evaluated, it just returns {@link FieldEval}
+   * (i.e., a column reference). Otherwise, it returns the original EvalNode of the expression.
+   *
+   * @param referenceName The reference name to get EvalNode
+   * @param unevaluatedForm If TRUE, it always return the annotated EvalNode of the expression.
+   * @return
+   */
+  public Target getTarget(String referenceName, boolean unevaluatedForm) {
+    String normalized = referenceName;
+    int refId = nameToIdMap.get(normalized);
+
+    if (!unevaluatedForm && evaluationStateMap.containsKey(refId) && evaluationStateMap.get(refId)) {
+      EvalNode evalNode = idToEvalMap.get(refId);
+
+      // If the expression is already evaluated, it should use the FieldEval to access a field value.
+      // But, if this reference name is not primary name, it cannot use the reference name.
+      // It changes the given reference name to the primary name.
+      if (isEvaluated(normalized) && !isPrimaryName(refId, referenceName)) {
+        return new Target(new FieldEval(getPrimaryName(refId),evalNode.getValueType()), referenceName);
+      }
+
+      EvalNode referredEval;
+      if (evalNode.getType() == EvalType.CONST) {
+        referredEval = evalNode;
+      } else {
+        referredEval = new FieldEval(idToNamesMap.get(refId).get(0), evalNode.getValueType());
+      }
+      return new Target(referredEval, referenceName);
+
+    } else {
+      if (idToEvalMap.containsKey(refId)) {
+        return new Target(idToEvalMap.get(refId), referenceName);
+      } else {
+        return null;
+      }
+    }
+  }
+
+  public String toString() {
+    return "unevaluated=" + nameToIdMap.size() + ", evaluated=" + idToEvalMap.size()
+        + ", renamed=" + aliasedColumnMap.size();
+  }
+
+  /**
+   * It returns an iterator for unevaluated NamedExprs.
+   */
+  public Iterator<NamedExpr> getIteratorForUnevaluatedExprs() {
+    return new UnevaluatedIterator();
+  }
+
+  public class UnevaluatedIterator implements Iterator<NamedExpr> {
+    private final Iterator<NamedExpr> iterator;
+
+    public UnevaluatedIterator() {
+      List<NamedExpr> unEvaluatedList = TUtil.newList();
+      for (Integer refId: idToNamesMap.keySet()) {
+        String name = idToNamesMap.get(refId).get(0);
+        if (!isEvaluated(name)) {
+          Expr expr = idToExprBiMap.get(refId);
+          unEvaluatedList.add(new NamedExpr(expr, name));
+        }
+      }
+      if (unEvaluatedList.size() == 0) {
+        iterator = null;
+      } else {
+        iterator = unEvaluatedList.iterator();
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return iterator != null && iterator.hasNext();
+    }
+
+    @Override
+    public NamedExpr next() {
+      return iterator.next();
+    }
+
+    @Override
+    public void remove() {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java
new file mode 100644
index 0000000..ebe47b4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java
@@ -0,0 +1,36 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.physical.PhysicalExec;
+import org.apache.tajo.exception.InternalException;
+
+/**
+ * This class generates a physical execution plan.
+ */
+public interface PhysicalPlanner {
+  public PhysicalExec createPlan(TaskAttemptContext context,
+                                 LogicalNode logicalPlan)
+      throws InternalException;
+}


[11/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/queryplan.css
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/queryplan.css b/tajo-core/src/main/resources/webapps/static/queryplan.css
new file mode 100644
index 0000000..b1b7dd4
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/static/queryplan.css
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+.window {
+    border:3px solid;
+    width:15em;
+    height:1.5em;
+    z-index:20;
+    position:absolute;
+}
+
+.component {
+    opacity:0.8;
+    filter:alpha(opacity=80);
+    background-color:white;
+    color:black;
+    font-family:helvetica;
+    padding:0.5em;
+}
+
+.label {
+    border:1px solid #346789;
+    padding:4px;
+    padding:4px;
+    font-size: 0.8em;
+}
+
+.label:hover {
+    border-color:#5d94a6;
+}
+
+.textborder {
+    border: 1px solid;
+    padding: 2px;
+}
+
+.component a:link {text-decoration:none;}
+.component a:visited {text-decoration:none;}
+.component a:hover {text-decoration:underline;}
+.component a:active {text-decoration:underline;}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/style.css
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/style.css b/tajo-core/src/main/resources/webapps/static/style.css
new file mode 100644
index 0000000..2af350e
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/static/style.css
@@ -0,0 +1,74 @@
+/*
+ * 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.
+ */
+html{
+    font:12px Arial, Helvetica, sans-serif; /* Sets the font size and type for the whole html page */
+    color:#333;} /* Sets the font color for the whole html page */
+
+body {margin:0;padding:0}
+
+.menu{
+    width: 100%; /* The menu should be the entire width of it's surrounding object, in this case the whole page */
+    background-color: #333;} /* dark grey bg */
+
+.menu ul{
+    margin: 0;
+    padding: 0;
+    float: left;}
+
+.menu ul li{
+    display: inline;} /* Makes the link all appear in one line, rather than on top of each other */
+
+.menu ul li a{
+    float: left;
+    text-decoration: none; /* removes the underline from the menu text */
+    color: #fff; /* text color of the menu */
+    padding: 10.5px 11px; /* 10.5px of padding to the right and left of the link and 11px to the top and bottom */
+    margin-right: 20px;
+    background-color: #333;}
+
+.menu ul li a:visited{ /* This bit just makes sure the text color doesn't change once you've visited a link */
+    color: #fff;
+    text-decoration: none;}
+
+.menu ul li a:hover, .menu ul li .current{
+    color: #fff;
+    background-color:#0b75b2;} /* change the background color of the list item when you hover over it */
+
+.page_title {
+    padding: 0px 0px 0px 10px
+}
+
+.contents {
+    padding: 0px 10px 0px 10px;
+    font-size:14px;
+}
+
+.border_table {
+    border-collapse:collapse;
+    border:1px gray solid;
+}
+
+.border_table td {
+    border:1px gray solid;
+    padding: 2px 5px 2px 5px
+}
+
+.border_table th {
+    border:1px gray solid;
+    padding: 2px 5px 2px 5px
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/WEB-INF/web.xml b/tajo-core/src/main/resources/webapps/worker/WEB-INF/web.xml
new file mode 100644
index 0000000..0ff8db6
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/WEB-INF/web.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<web-app xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xmlns="http://java.sun.com/xml/ns/javaee" xmlns:web="http://java.sun.com/xml/ns/javaee/web-app_2_5.xsd"
+	xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_2_5.xsd"
+	version="2.5">
+   <display-name>Tajo Worker</display-name>
+    <welcome-file-list>
+        <welcome-file>index.jsp</welcome-file>
+    </welcome-file-list>
+</web-app>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/conf.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/conf.jsp b/tajo-core/src/main/resources/webapps/worker/conf.jsp
new file mode 100644
index 0000000..571461b
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/conf.jsp
@@ -0,0 +1,55 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="java.util.Map" %>
+<%@ page import="org.apache.hadoop.conf.Configuration" %>
+
+<%
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  Configuration tajoConf = tajoWorker.getConfig();
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+  <hr/>
+  <table width="100%" border="1" class="border_table">
+<%
+  for(Map.Entry<String,String> entry: tajoConf) {
+%>
+    <tr><td width="200"><%=entry.getKey()%></td><td><%=entry.getValue()%></td>
+<%
+  }
+%>
+  </table>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/env.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/env.jsp b/tajo-core/src/main/resources/webapps/worker/env.jsp
new file mode 100644
index 0000000..b84f6b7
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/env.jsp
@@ -0,0 +1,69 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="java.util.Map" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="org.apache.hadoop.conf.Configuration" %>
+
+<%
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  Configuration tajoConf = tajoWorker.getConfig();
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+  <hr/>
+  <h3>System Environment</h3>
+  <table width="100%" class="border_table">
+<%
+  for(Map.Entry<String, String> entry: System.getenv().entrySet()) {
+%>
+    <tr><td width="200"><%=entry.getKey()%></td><td><%=entry.getValue()%></td>
+<%
+  }
+%>
+  </table>
+
+  <h3>Properties</h3>
+  <hr/>
+
+  <table width="100%" class="border_table">
+<%
+  for(Map.Entry<Object, Object> entry: System.getProperties().entrySet()) {
+%>
+    <tr><td width="200"><%=entry.getKey()%></td><td><%=entry.getValue()%></td>
+<%
+  }
+%>
+  </table>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/header.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/header.jsp b/tajo-core/src/main/resources/webapps/worker/header.jsp
new file mode 100644
index 0000000..f20eaf0
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/header.jsp
@@ -0,0 +1,37 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page import="org.apache.tajo.util.JSPUtil" %>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+<%
+  TajoWorker tmpTajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  String tajoMasterHttp = "http://" + JSPUtil.getTajoMasterHttpAddr(tmpTajoWorker.getConfig());
+%>
+<div class="menu">
+  <div style='float:left; margin-left:12px; margin-top:6px;'><a href='<%=tajoMasterHttp%>/index.jsp'><img src='/static/img/logo_tajo.gif' border='0'/></a></div>
+  <ul>
+    <li><a class='top_menu_item' style='margin-left:10px;' href='<%=tajoMasterHttp%>/index.jsp'>Home</a></li>
+    <li><a class='top_menu_item' href='<%=tajoMasterHttp%>/cluster.jsp'>Cluster</a></li>
+    <li><a class='top_menu_item' href='<%=tajoMasterHttp%>/query.jsp'>Query</a></li>
+    <li><a class='top_menu_item' href='<%=tajoMasterHttp%>/catalogview.jsp'>Catalog</a></li>
+    <li><a class='top_menu_item' href='<%=tajoMasterHttp%>/query_executor.jsp'>Execute Query</a></li>
+  </ul>
+  <br style="clear: left" />
+</div>
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/index.jsp b/tajo-core/src/main/resources/webapps/worker/index.jsp
new file mode 100644
index 0000000..c30a72d
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/index.jsp
@@ -0,0 +1,150 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="java.util.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.master.querymaster.Query" %>
+
+<%
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+
+  SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>tajo worker</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <%=tajoWorker.getWorkerContext().getWorkerName()%></h2>
+  <hr/>
+  <table border=0>
+    <tr><td width='100'>MaxHeap: </td><td><%=Runtime.getRuntime().maxMemory()/1024/1024%> MB</td>
+    <tr><td width='100'>TotalHeap: </td><td><%=Runtime.getRuntime().totalMemory()/1024/1024%> MB</td>
+    <tr><td width='100'>FreeHeap: </td><td><%=Runtime.getRuntime().freeMemory()/1024/1024%> MB</td>
+    <tr><td width="100">Configuration:</td><td><a href='conf.jsp'>detail...</a></td></tr>
+    <tr><td width="100">Environment:</td><td><a href='env.jsp'>detail...</a></td></tr>
+    <tr><td width="100">Threads:</td><td><a href='thread.jsp'>thread dump...</a></tr>
+  </table>
+  <hr/>
+
+<%
+if(tajoWorker.getWorkerContext().isQueryMasterMode()) {
+  List<QueryMasterTask> queryMasterTasks = JSPUtil.sortQueryMasterTask(tajoWorker.getWorkerContext()
+          .getQueryMasterManagerService().getQueryMaster().getQueryMasterTasks(), true);
+
+  List<QueryMasterTask> finishedQueryMasterTasks = JSPUtil.sortQueryMasterTask(tajoWorker.getWorkerContext()
+          .getQueryMasterManagerService().getQueryMaster().getFinishedQueryMasterTasks(), true);
+%>
+  <h3>Running Query</h3>
+  <%
+    if(queryMasterTasks.isEmpty()) {
+      out.write("No running query master");
+    } else {
+  %>
+  <table width="100%" border="1" class="border_table">
+    <tr><th>QueryId</th><th>Status</th><th>StartTime</th><th>FinishTime</th><th>Progress</th><th>RunTime</th></tr>
+    <%
+      for(QueryMasterTask eachQueryMasterTask: queryMasterTasks) {
+        Query query = eachQueryMasterTask.getQuery();
+    %>
+    <tr>
+      <td align='center'><a href='querydetail.jsp?queryId=<%=query.getId()%>'><%=query.getId()%></a></td>
+      <td align='center'><%=eachQueryMasterTask.getState()%></td>
+      <td align='center'><%=df.format(query.getStartTime())%></td>
+      <td align='center'><%=query.getFinishTime() == 0 ? "-" : df.format(query.getFinishTime())%></td>
+      <td align='center'><%=(int)(query.getProgress()*100.0f)%>%</td>
+      <td align='right'><%=JSPUtil.getElapsedTime(query.getStartTime(), query.getFinishTime())%></td>
+    </tr>
+    <%
+        } //end of for
+      } //end of if
+    %>
+  </table>
+  <p/>
+  <hr/>
+  <h3>Finished Query</h3>
+  <%
+    if(finishedQueryMasterTasks.isEmpty()) {
+      out.write("No finished query master");
+    } else {
+  %>
+  <table width="100%" border="1" class="border_table">
+    <tr><th>QueryId</th><th>Status</th><th>StartTime</th><th>FinishTime</th><th>Progress</th><th>RunTime</th></tr>
+    <%
+      for(QueryMasterTask eachQueryMasterTask: finishedQueryMasterTasks) {
+        Query query = eachQueryMasterTask.getQuery();
+        long startTime = query != null ? query.getStartTime() : eachQueryMasterTask.getQuerySubmitTime();
+    %>
+    <tr>
+      <td align='center'><a href='querydetail.jsp?queryId=<%=eachQueryMasterTask.getQueryId()%>'><%=eachQueryMasterTask.getQueryId()%></a></td>
+      <td align='center'><%=eachQueryMasterTask.getState()%></td>
+      <td align='center'><%=df.format(startTime)%></td>
+      <td align='center'><%=(query == null || query.getFinishTime() == 0) ? "-" : df.format(query.getFinishTime())%></td>
+      <td align='center'><%=(query == null) ? "-" : (int)(query.getProgress()*100.0f)%>%</td>
+      <td align='right'><%=(query == null) ? "-" : JSPUtil.getElapsedTime(query.getStartTime(), query.getFinishTime())%></td>
+    </tr>
+    <%
+        } //end of for
+      } //end of if
+    %>
+  </table>
+  <p/>
+  <hr/>
+<%
+} // end of QueryMaster
+if(tajoWorker.getWorkerContext().isTaskRunnerMode()) {
+  List<TaskRunner> taskRunners = new ArrayList<TaskRunner>(tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskRunners());
+  JSPUtil.sortTaskRunner(taskRunners);
+%>
+  <h3>Running Task Containers</h3>
+  <a href='taskcontainers.jsp'>[All Task Containers]</a>
+  <br/>
+  <table width="100%" border="1" class="border_table">
+    <tr><th>ContainerId</th><th>StartTime</th><th>FinishTime</th><th>RunTime</th><th>Status</th></tr>
+    <%
+      for(TaskRunner eachTaskRunner: taskRunners) {
+    %>
+    <tr>
+      <td><a href="tasks.jsp?containerId=<%=eachTaskRunner.getId()%>"><%=eachTaskRunner.getId()%></a></td>
+      <td><%=df.format(eachTaskRunner.getStartTime())%></td>
+      <td><%=eachTaskRunner.getFinishTime() == 0 ? "-" : df.format(eachTaskRunner.getFinishTime())%></td>
+      <td><%=JSPUtil.getElapsedTime(eachTaskRunner.getStartTime(), eachTaskRunner.getFinishTime())%></td>
+      <td><%=eachTaskRunner.getServiceState()%></td>
+<%
+      }   //end of for
+%>
+  </table>
+<%
+} //end of if
+%>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
new file mode 100644
index 0000000..3de20fe
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
@@ -0,0 +1,105 @@
+<%
+  /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements. See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership. The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License. You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.master.querymaster.*" %>
+<%@ page import="java.util.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+<%@ page import="org.apache.tajo.QueryId" %>
+<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
+
+<%
+  QueryId queryId = TajoIdUtils.parseQueryId(request.getParameter("queryId"));
+
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext()
+          .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true);
+
+  if (queryMasterTask == null) {
+    out.write("<script type='text/javascript'>alert('no query'); history.back(0); </script>");
+    return;
+  }
+  Query query = queryMasterTask.getQuery();
+  List<SubQuery> subQueries = null;
+  if (query != null) {
+    subQueries = JSPUtil.sortSubQuery(query.getSubQueries());
+  }
+
+  SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type="text/css" href="/static/style.css"/>
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Query Detail Info</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+  <hr/>
+<%
+if (query == null) {
+  String errorMessage = queryMasterTask.getErrorMessage();
+  out.write("Query Status: " + queryMasterTask.getState());
+  if (errorMessage != null && !errorMessage.isEmpty()) {
+    out.write("<p/>Message:<p/><pre>" + errorMessage + "</pre>");
+  }
+} else {
+%>
+  <h3><%=queryId.toString()%> <a href='queryplan.jsp?queryId=<%=queryId%>'>[Query Plan]</a></h3>
+  <table width="100%" border="1" class="border_table">
+    <tr><th>ID</th><th>State</th><th>Started</th><th>Finished</th><th>Running time</th><th>Progress</th><th>Tasks</th></tr>
+<%
+for(SubQuery eachSubQuery: subQueries) {
+    eachSubQuery.getSucceededObjectCount();
+    String detailLink = "querytasks.jsp?queryId=" + queryId + "&ebid=" + eachSubQuery.getId();
+%>
+  <tr>
+    <td><a href='<%=detailLink%>'><%=eachSubQuery.getId()%></a></td>
+    <td><%=eachSubQuery.getState()%></td>
+    <td><%=df.format(eachSubQuery.getStartTime())%></td>
+    <td><%=eachSubQuery.getFinishTime() == 0 ? "-" : df.format(eachSubQuery.getFinishTime())%></td>
+    <td><%=JSPUtil.getElapsedTime(eachSubQuery.getStartTime(), eachSubQuery.getFinishTime())%></td>
+    <td align='center'><%=JSPUtil.percentFormat(eachSubQuery.getProgress())%>%</td>
+    <td align='center'><a href='<%=detailLink%>&status=SUCCEEDED'><%=eachSubQuery.getSucceededObjectCount()%></a>/<a href='<%=detailLink%>&status=ALL'><%=eachSubQuery.getTotalScheduledObjectsCount()%></a></td>
+  </tr>
+  <%
+}  //end of for
+  %>
+  </table>
+  <p/>
+  <hr/>
+  <h3>Logical Plan</h3>
+  <pre style="white-space:pre-wrap;"><%=query.getPlan().getLogicalPlan().toString()%></pre>
+  <hr/>
+  <h3>Distributed Query Plan</h3>
+  <pre style="white-space:pre-wrap;"><%=query.getPlan().toString()%></pre>
+  <hr/>
+<%
+}   //end of else [if (query == null)]
+%>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/queryplan.jsp b/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
new file mode 100644
index 0000000..ec860b9
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
@@ -0,0 +1,237 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="org.apache.tajo.master.querymaster.Query" %>
+<%@ page import="org.apache.tajo.QueryId" %>
+<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.master.querymaster.SubQuery" %>
+<%@ page import="org.apache.tajo.engine.planner.global.ExecutionBlock" %>
+<%@ page import="java.util.*" %>
+<%@ page import="org.apache.tajo.ExecutionBlockId" %>
+<%@ page import="org.apache.tajo.engine.planner.global.MasterPlan" %>
+<%@ page import="org.apache.tajo.engine.planner.global.DataChannel" %>
+
+<%
+  QueryId queryId = TajoIdUtils.parseQueryId(request.getParameter("queryId"));
+
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+  QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext()
+          .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true);
+
+  if(queryMasterTask == null) {
+    out.write("<script type='text/javascript'>alert('no query'); history.back(0); </script>");
+    return;
+  }
+
+  Query query = queryMasterTask.getQuery();
+
+  Map<ExecutionBlockId, SubQuery> subQueryMap = new HashMap<ExecutionBlockId, SubQuery>();
+
+  for(SubQuery eachSubQuery: query.getSubQueries()) {
+    subQueryMap.put(eachSubQuery.getId(), eachSubQuery);
+  }
+
+  class SubQueryInfo {
+    ExecutionBlock executionBlock;
+    SubQuery subQuery;
+    ExecutionBlockId parentId;
+    int px;
+    int py;
+    int pos; // 0: mid 1: left 2: right
+    public SubQueryInfo(ExecutionBlock executionBlock, SubQuery subQuery, ExecutionBlockId parentId, int px, int py, int pos) {
+      this.executionBlock = executionBlock;
+      this.subQuery = subQuery;
+      this.parentId = parentId;
+      this.px = px;
+      this.py = py;
+      this.pos = pos;
+    }
+  }
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <link rel="stylesheet" type = "text/css" href = "/static/queryplan.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Tajo</title>
+  <script type='text/javascript' src='/static/js/jquery.js'></script>
+  <script type='text/javascript' src='/static/js/jquery-ui.min.js'></script>
+  <script type='text/javascript' src='/static/js/jquery.jsPlumb-1.3.16-all.js'></script>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+  <hr/>
+  <div>
+    <h3>Distributed Query Execution Plan</h3>
+    <div style='float:left'><span class="textborder" style="color:black;font-size:9px">NEW</span></div>
+    <div style='float:left;margin-left:5px;'><span class="textborder" style="color:gray;font-size:9px">ALLOCATED</span></div>
+    <div style='float:left;margin-left:5px;'><span class="textborder" style="color:skyblue;font-size:9px">INIT</span></div>
+    <div style='float:left;margin-left:5px;'><span class="textborder" style="color:blue;font-size:9px">RUNNING</span></div>
+    <div style='float:left;margin-left:5px;'><span class="textborder" style="color:green;font-size:9px">SUCCEEDED</span></div>
+    <div style='float:left;margin-left:5px;'><span class="textborder" style="color:red;font-size:9px">FAILED</span></div>
+  </div>
+
+<!-- draw the query plan -->
+<%
+  MasterPlan masterPlan = query.getPlan();
+  String curIdStr = null;
+  int x=35, y=1;
+  int pos;
+  List<SubQueryInfo> subQueryInfos = new ArrayList<SubQueryInfo>();
+
+  subQueryInfos.add(new SubQueryInfo(masterPlan.getRoot(), null, null, x, y, 0));
+
+  while (!subQueryInfos.isEmpty()) {
+    SubQueryInfo eachSubQueryInfo = subQueryInfos.remove(0);
+    curIdStr = eachSubQueryInfo.executionBlock.getId().toString();
+
+    y = eachSubQueryInfo.py + 13;
+    if (eachSubQueryInfo.pos == 0) {
+      x = eachSubQueryInfo.px;
+    } else if (eachSubQueryInfo.pos == 1) {
+      x = eachSubQueryInfo.px - 20;
+    } else if (eachSubQueryInfo.pos == 2) {
+      x = eachSubQueryInfo.px + 20;
+    }
+%>
+  <script type='text/javascript'>
+    jsPlumb.setRenderMode(jsPlumb.CANVAS);
+  </script>
+
+  <div class="component window" id="<%=curIdStr%>" style="left:<%=x%>em;top:<%=y%>em;">
+    <a style="font-size:0.9em;" href="./querytasks.jsp?queryId=<%=queryId%>&ebid=<%=curIdStr%>"><%=curIdStr%></a></p>
+  </div>
+
+<%
+    if (eachSubQueryInfo.parentId != null) {
+      String outgoing = "";
+      String prefix = "";
+      for (DataChannel channel : masterPlan.getOutgoingChannels(eachSubQueryInfo.executionBlock.getId())) {
+        outgoing += prefix + channel.getShuffleType();
+        prefix = "; ";
+      }
+%>
+  <script type="text/javascript">
+    var srcId = "<%=curIdStr%>";
+    var destId = "<%=eachSubQueryInfo.parentId.toString()%>";
+    var src = window.jsPlumb.addEndpoint(srcId, {
+        anchor:"AutoDefault",
+        paintStyle:{
+          fillStyle:"CornflowerBlue "
+        },
+        hoverPaintStyle:{
+          fillStyle:"red"
+        }
+      }
+    );
+
+    var dst = jsPlumb.addEndpoint(destId, {
+        anchor:"AutoDefault",
+        paintStyle:{
+          fillStyle:"CornflowerBlue "
+        },
+        hoverPaintStyle:{
+          fillStyle:"red"
+        }
+      }
+    );
+
+    var con = jsPlumb.connect({
+      source:src,
+      target:dst,
+      paintStyle:{ strokeStyle:"CornflowerBlue ", lineWidth:3  },
+      hoverPaintStyle:{ strokeStyle:"red", lineWidth:4 },
+      overlays : [ <!-- overlays start -->
+        [ "Arrow", { location:1 } ],
+        ["Label", {
+          cssClass:"l1 component label",
+          label : "<%=outgoing%>",
+          location:0.5,
+          id:"label",
+          events:{
+            "click":function(label, evt) {
+            }
+          }
+        }] <!-- label end -->
+      ] <!-- overlays end -->
+    });
+  </script>
+<%
+    } //end of if
+%>
+
+  <script type='text/javascript'>
+    var e = document.getElementById("<%=curIdStr%>");
+    var state = "<%=eachSubQueryInfo.subQuery != null ? eachSubQueryInfo.subQuery.getState().name(): ""%>";
+    switch (state) {
+      case 'NEW':
+        e.style.borderColor = "black";
+        e.style.color = "black";
+        break;
+      case 'CONTAINER_ALLOCATED':
+        e.style.borderColor = "gray";
+        e.style.color = "gray";
+        break;
+      case 'INIT':
+        e.style.borderColor = "skyblue";
+        e.style.color = "skyblue";
+        break;
+      case 'RUNNING':
+        e.style.borderColor = "blue";
+        e.style.color = "blue";
+        break;
+      case 'SUCCEEDED':
+        e.style.borderColor = "green";
+        e.style.color = "green";
+        break;
+      case 'FAILED':
+        e.style.borderColor = "red";
+        e.style.color = "red";
+        break;
+      default:
+        break;
+    }
+  </script>
+
+<%
+    List<ExecutionBlock> children = masterPlan.getChilds(eachSubQueryInfo.executionBlock.getId());
+
+    if (children.size() == 1) {
+      pos = 0;
+    } else {
+      pos = 1;
+    }
+    for (ExecutionBlock child : children) {
+      subQueryInfos.add(new SubQueryInfo(child, subQueryMap.get(child.getId()), eachSubQueryInfo.executionBlock.getId(), x, y, pos++));
+    }
+  } //end of while
+%>
+
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
new file mode 100644
index 0000000..ab6ff26
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
@@ -0,0 +1,232 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.master.querymaster.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+<%@ page import="org.apache.tajo.QueryId" %>
+<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
+<%@ page import="org.apache.tajo.ExecutionBlockId" %>
+<%@ page import="org.apache.tajo.ipc.TajoMasterProtocol" %>
+<%@ page import="java.util.List" %>
+<%@ page import="java.util.Map" %>
+<%@ page import="java.util.HashMap" %>
+<%@ page import="org.apache.tajo.QueryUnitAttemptId" %>
+<%@ page import="org.apache.tajo.catalog.statistics.TableStats" %>
+<%@ page import="java.util.Locale" %>
+<%@ page import="java.text.NumberFormat" %>
+<%@ page import="org.apache.tajo.engine.planner.PlannerUtil" %>
+<%@ page import="org.apache.tajo.util.FileUtil" %>
+
+<%
+  String paramQueryId = request.getParameter("queryId");
+  String paramEbId = request.getParameter("ebid");
+
+  QueryId queryId = TajoIdUtils.parseQueryId(paramQueryId);
+  ExecutionBlockId ebid = TajoIdUtils.createExecutionBlockId(paramEbId);
+  String sort = request.getParameter("sort");
+  if(sort == null) {
+    sort = "id";
+  }
+  String sortOrder = request.getParameter("sortOrder");
+  if(sortOrder == null) {
+    sortOrder = "asc";
+  }
+
+  String nextSortOrder = "asc";
+  if("asc".equals(sortOrder)) {
+    nextSortOrder = "desc";
+  }
+
+  String status = request.getParameter("status");
+  if(status == null || status.isEmpty() || "null".equals(status)) {
+    status = "ALL";
+  }
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+
+  List<TajoMasterProtocol.WorkerResourceProto> allWorkers = tajoWorker.getWorkerContext()
+            .getQueryMasterManagerService().getQueryMaster().getAllWorker();
+
+  Map<String, TajoMasterProtocol.WorkerResourceProto> workerMap = new HashMap<String, TajoMasterProtocol.WorkerResourceProto>();
+  if(allWorkers != null) {
+    for(TajoMasterProtocol.WorkerResourceProto eachWorker: allWorkers) {
+      workerMap.put(eachWorker.getHost(), eachWorker);
+    }
+  }
+  QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext()
+          .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true);
+
+  if(queryMasterTask == null) {
+    out.write("<script type='text/javascript'>alert('no query'); history.back(0); </script>");
+    return;
+  }
+
+  Query query = queryMasterTask.getQuery();
+  SubQuery subQuery = query.getSubQuery(ebid);
+
+  if(subQuery == null) {
+    out.write("<script type='text/javascript'>alert('no sub-query'); history.back(0); </script>");
+    return;
+  }
+
+  if(subQuery == null) {
+%>
+<script type="text/javascript">
+  alert("No Execution Block for" + ebid);
+  document.history.back();
+</script>
+<%
+    return;
+  }
+
+  SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  String url = "querytasks.jsp?queryId=" + queryId + "&ebid=" + ebid + "&status=" + status + "&sortOrder=" + nextSortOrder + "&sort=";
+  QueryUnit[] queryUnits = subQuery.getQueryUnits();
+
+
+  long totalInputBytes = 0;
+  long totalReadBytes = 0;
+  long totalReadRows = 0;
+  long totalWriteBytes = 0;
+  long totalWriteRows = 0;
+  int numTasks = queryUnits.length;
+//  int numSucceededTasks = 0;
+//  int localReadTasks = subQuery.;
+  int numShuffles = 0;
+
+  float totalProgress = 0.0f;
+  for(QueryUnit eachQueryUnit: queryUnits) {
+    totalProgress += eachQueryUnit.getLastAttempt() != null ? eachQueryUnit.getLastAttempt().getProgress(): 0.0f;
+    numShuffles = eachQueryUnit.getShuffleOutpuNum();
+    if (eachQueryUnit.getLastAttempt() != null) {
+      TableStats inputStats = eachQueryUnit.getLastAttempt().getInputStats();
+      if (inputStats != null) {
+        totalInputBytes += inputStats.getNumBytes();
+        totalReadBytes += inputStats.getReadBytes();
+        totalReadRows += inputStats.getNumRows();
+      }
+      TableStats outputStats = eachQueryUnit.getLastAttempt().getResultStats();
+      if (outputStats != null) {
+        totalWriteBytes += outputStats.getNumBytes();
+        totalWriteRows += outputStats.getNumRows();
+      }
+    }
+  }
+
+    NumberFormat nf = NumberFormat.getInstance(Locale.US);
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type="text/css" href="/static/style.css"/>
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>Query Detail Info</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+  <hr/>
+  <h3><a href='querydetail.jsp?queryId=<%=paramQueryId%>'><%=ebid.toString()%></a></h3>
+  <hr/>
+  <p/>
+  <pre style="white-space:pre-wrap;"><%=PlannerUtil.buildExplainString(subQuery.getBlock().getPlan())%></pre>
+  <p/>
+  <table border="1" width="100%" class="border_table">
+    <tr><td align='right' width='180px'>Status:</td><td><%=subQuery.getState()%></td></tr>
+    <tr><td align='right'>Started:</td><td><%=df.format(subQuery.getStartTime())%> ~ <%=subQuery.getFinishTime() == 0 ? "-" : df.format(subQuery.getFinishTime())%></td></tr>
+    <tr><td align='right'># Tasks:</td><td><%=numTasks%> (Local Tasks: <%=subQuery.getTaskScheduler().getHostLocalAssigned()%>, Rack Local Tasks: <%=subQuery.getTaskScheduler().getRackLocalAssigned()%>)</td></tr>
+    <tr><td align='right'>Progress:</td><td><%=JSPUtil.percentFormat((float)(totalProgress/numTasks))%>%</td></tr>
+    <tr><td align='right'># Shuffles:</td><td><%=numShuffles%></td></tr>
+    <tr><td align='right'>Input Bytes:</td><td><%=FileUtil.humanReadableByteCount(totalInputBytes, false) + " (" + nf.format(totalInputBytes) + " B)"%></td></tr>
+    <tr><td align='right'>Actual Processed Bytes:</td><td><%=totalReadBytes == 0 ? "-" : FileUtil.humanReadableByteCount(totalReadBytes, false) + " (" + nf.format(totalReadBytes) + " B)"%></td></tr>
+    <tr><td align='right'>Input Rows:</td><td><%=nf.format(totalReadRows)%></td></tr>
+    <tr><td align='right'>Output Bytes:</td><td><%=FileUtil.humanReadableByteCount(totalWriteBytes, false) + " (" + nf.format(totalWriteBytes) + " B)"%></td></tr>
+    <tr><td align='right'>Output Rows:</td><td><%=nf.format(totalWriteRows)%></td></tr>
+  </table>
+  <hr/>
+
+
+  <form action='querytasks.jsp' method='GET'>
+  Status:
+    <select name="status" onchange="this.form.submit()">
+        <option value="ALL" <%="ALL".equals(status) ? "selected" : ""%>>ALL</option>
+        <option value="SCHEDULED" <%="SCHEDULED".equals(status) ? "selected" : ""%>>SCHEDULED</option>
+        <option value="RUNNING" <%="RUNNING".equals(status) ? "selected" : ""%>>RUNNING</option>
+        <option value="SUCCEEDED" <%="SUCCEEDED".equals(status) ? "selected" : ""%>>SUCCEEDED</option>
+    </select>
+    &nbsp;&nbsp;
+    <input type="submit" value="Filter">
+    <input type="hidden" name="queryId" value="<%=paramQueryId%>"/>
+    <input type="hidden" name="ebid" value="<%=paramEbId%>"/>
+    <input type="hidden" name="sort" value="<%=sort%>"/>
+    <input type="hidden" name="sortOrder" value="<%=sortOrder%>"/>
+  </form>
+  <table border="1" width="100%" class="border_table">
+    <tr><th>No</th><th><a href='<%=url%>id'>Id</a></th><th>Status</th><th>Progress</th><th><a href='<%=url%>startTime'>Started</a></th><th><a href='<%=url%>runTime'>Running Time</a></th><th><a href='<%=url%>host'>Host</a></th></tr>
+    <%
+      JSPUtil.sortQueryUnit(queryUnits, sort, sortOrder);
+      int rowNo = 1;
+      for(QueryUnit eachQueryUnit: queryUnits) {
+          if(!"ALL".equals(status)) {
+            if(!status.equals(eachQueryUnit.getState().toString())) {
+              continue;
+            }
+          }
+          int queryUnitSeq = eachQueryUnit.getId().getId();
+          String queryUnitDetailUrl = "queryunit.jsp?queryId=" + paramQueryId + "&ebid=" + paramEbId +
+                  "&queryUnitSeq=" + queryUnitSeq + "&sort=" + sort + "&sortOrder=" + sortOrder;
+
+          String queryUnitHost = eachQueryUnit.getSucceededHost() == null ? "-" : eachQueryUnit.getSucceededHost();
+          if(eachQueryUnit.getSucceededHost() != null) {
+              TajoMasterProtocol.WorkerResourceProto worker = workerMap.get(eachQueryUnit.getSucceededHost());
+              if(worker != null) {
+                  QueryUnitAttempt lastAttempt = eachQueryUnit.getLastAttempt();
+                  if(lastAttempt != null) {
+                    QueryUnitAttemptId lastAttemptId = lastAttempt.getId();
+                    queryUnitHost = "<a href='http://" + eachQueryUnit.getSucceededHost() + ":" + worker.getInfoPort() + "/taskdetail.jsp?queryUnitAttemptId=" + lastAttemptId + "'>" + eachQueryUnit.getSucceededHost() + "</a>";
+                  }
+              }
+          }
+
+    %>
+    <tr>
+      <td><%=rowNo%></td>
+      <td><a href="<%=queryUnitDetailUrl%>"><%=eachQueryUnit.getId()%></a></td>
+      <td><%=eachQueryUnit.getState()%></td>
+      <td><%=JSPUtil.percentFormat(eachQueryUnit.getLastAttempt().getProgress())%>%</td>
+      <td><%=eachQueryUnit.getLaunchTime() == 0 ? "-" : df.format(eachQueryUnit.getLaunchTime())%></td>
+      <td align='right'><%=eachQueryUnit.getLaunchTime() == 0 ? "-" : eachQueryUnit.getRunningTime() + " ms"%></td>
+      <td><%=queryUnitHost%></td>
+    </tr>
+    <%
+        rowNo++;
+      }
+    %>
+  </table>
+  <%
+  %>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/queryunit.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/queryunit.jsp b/tajo-core/src/main/resources/webapps/worker/queryunit.jsp
new file mode 100644
index 0000000..3e8dfef
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/queryunit.jsp
@@ -0,0 +1,171 @@
+<%
+    /*
+    * 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.
+    */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.ExecutionBlockId" %>
+<%@ page import="org.apache.tajo.QueryId" %>
+<%@ page import="org.apache.tajo.QueryUnitId" %>
+<%@ page import="org.apache.tajo.catalog.proto.CatalogProtos" %>
+<%@ page import="org.apache.tajo.ipc.TajoWorkerProtocol" %>
+<%@ page import="org.apache.tajo.master.querymaster.Query" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryUnit" %>
+<%@ page import="org.apache.tajo.master.querymaster.SubQuery" %>
+<%@ page import="org.apache.tajo.storage.DataLocation" %>
+<%@ page import="org.apache.tajo.storage.fragment.FileFragment" %>
+<%@ page import="org.apache.tajo.storage.fragment.FragmentConvertor" %>
+<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.TajoWorker" %>
+<%@ page import="java.net.URI" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+<%@ page import="java.util.Map" %>
+<%@ page import="java.util.Set" %>
+<%@ page import="org.apache.tajo.catalog.statistics.TableStats" %>
+<%@ page import="org.apache.tajo.worker.TaskHistory" %>
+
+<%
+    String paramQueryId = request.getParameter("queryId");
+    String paramEbId = request.getParameter("ebid");
+    String status = request.getParameter("status");
+    if(status == null || status.isEmpty() || "null".equals(status)) {
+        status = "ALL";
+    }
+
+    QueryId queryId = TajoIdUtils.parseQueryId(paramQueryId);
+    ExecutionBlockId ebid = TajoIdUtils.createExecutionBlockId(paramEbId);
+
+    int queryUnitSeq = Integer.parseInt(request.getParameter("queryUnitSeq"));
+    TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+    QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext()
+            .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true);
+
+    if(queryMasterTask == null) {
+        out.write("<script type='text/javascript'>alert('no query'); history.back(0); </script>");
+        return;
+    }
+
+    Query query = queryMasterTask.getQuery();
+    SubQuery subQuery = query.getSubQuery(ebid);
+
+    if(subQuery == null) {
+        out.write("<script type='text/javascript'>alert('no sub-query'); history.back(0); </script>");
+        return;
+    }
+
+    if(subQuery == null) {
+%>
+<script type="text/javascript">
+    alert("No Execution Block for" + ebid);
+    document.history.back();
+</script>
+<%
+        return;
+    }
+    SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+    QueryUnitId queryUnitId = new QueryUnitId(ebid, queryUnitSeq);
+    QueryUnit queryUnit = subQuery.getQueryUnit(queryUnitId);
+    if(queryUnit == null) {
+%>
+<script type="text/javascript">
+    alert("No QueryUnit for" + queryUnitId);
+    document.history.back();
+</script>
+<%
+        return;
+    }
+
+    String sort = request.getParameter("sort");
+    String sortOrder = request.getParameter("sortOrder");
+
+    String backUrl = "querytasks.jsp?queryId=" + paramQueryId + "&ebid=" + paramEbId + "&sort=" + sort + "&sortOrder=" + sortOrder + "&status=" + status;
+
+    String fragmentInfo = "";
+    String delim = "";
+    for (CatalogProtos.FragmentProto eachFragment : queryUnit.getAllFragments()) {
+        FileFragment fileFragment = FragmentConvertor.convert(FileFragment.class, eachFragment);
+        fragmentInfo += delim + fileFragment.toString();
+        delim = "<br/>";
+    }
+
+    String fetchInfo = "";
+    delim = "";
+    for (Map.Entry<String, Set<URI>> e : queryUnit.getFetchMap().entrySet()) {
+        fetchInfo += delim + "<b>" + e.getKey() + "</b>";
+        delim = "<br/>";
+        for (URI t : e.getValue()) {
+            fetchInfo += delim + t;
+        }
+    }
+
+    String dataLocationInfos = "";
+    delim = "";
+    for(DataLocation eachLocation: queryUnit.getDataLocations()) {
+        dataLocationInfos += delim + eachLocation.toString();
+        delim = "<br/>";
+    }
+
+    int numShuffles = queryUnit.getShuffleOutpuNum();
+    String shuffleKey = "-";
+    String shuffleFileName = "-";
+    if(numShuffles > 0) {
+        TajoWorkerProtocol.ShuffleFileOutput shuffleFileOutputs = queryUnit.getShuffleFileOutputs().get(0);
+        shuffleKey = "" + shuffleFileOutputs.getPartId();
+        shuffleFileName = shuffleFileOutputs.getFileName();
+    }
+
+    //int numIntermediateData = queryUnit.getIntermediateData() == null ? 0 : queryUnit.getIntermediateData().size();
+    TableStats inputStat = queryUnit.getLastAttempt().getInputStats();
+    TableStats outputStat = queryUnit.getLastAttempt().getResultStats();
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+    <link rel="stylesheet" type="text/css" href="/static/style.css"/>
+    <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+    <title>Query Unit Detail</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+    <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+    <hr/>
+    <h3><a href='<%=backUrl%>'><%=ebid.toString()%></a></h3>
+    <hr/>
+    <table border="1" width="100%" class="border_table">
+        <tr><td width="200" align="right">ID</td><td><%=queryUnit.getId()%></td></tr>
+        <tr><td align="right">Progress</td><td><%=JSPUtil.percentFormat(queryUnit.getLastAttempt().getProgress())%>%</td></tr>
+        <tr><td align="right">State</td><td><%=queryUnit.getState()%></td></tr>
+        <tr><td align="right">Launch Time</td><td><%=queryUnit.getLaunchTime() == 0 ? "-" : df.format(queryUnit.getLaunchTime())%></td></tr>
+        <tr><td align="right">Finish Time</td><td><%=queryUnit.getFinishTime() == 0 ? "-" : df.format(queryUnit.getFinishTime())%></td></tr>
+        <tr><td align="right">Running Time</td><td><%=queryUnit.getLaunchTime() == 0 ? "-" : queryUnit.getRunningTime() + " ms"%></td></tr>
+        <tr><td align="right">Host</td><td><%=queryUnit.getSucceededHost() == null ? "-" : queryUnit.getSucceededHost()%></td></tr>
+        <tr><td align="right">Shuffles</td><td># Shuffle Outputs: <%=numShuffles%>, Shuffle Key: <%=shuffleKey%>, Shuffle file: <%=shuffleFileName%></td></tr>
+        <tr><td align="right">Data Locations</td><td><%=dataLocationInfos%></td></tr>
+        <tr><td align="right">Fragment</td><td><%=fragmentInfo%></td></tr>
+        <tr><td align="right">Input Statistics</td><td><%=TaskHistory.toInputStatsString(inputStat)%></td></tr>
+        <tr><td align="right">Output Statistics</td><td><%=TaskHistory.toOutputStatsString(outputStat)%></td></tr>
+        <tr><td align="right">Fetches</td><td><%=fetchInfo%></td></tr>
+    </table>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp b/tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp
new file mode 100644
index 0000000..be19a42
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp
@@ -0,0 +1,87 @@
+<%
+  /*
+  * 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.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="java.util.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+
+<%
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+
+  List<TaskRunner> taskRunners = new ArrayList<TaskRunner>(tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskRunners());
+  List<TaskRunner> finishedTaskRunners = new ArrayList<TaskRunner>(tajoWorker.getWorkerContext().getTaskRunnerManager().getFinishedTaskRunners());
+
+  JSPUtil.sortTaskRunner(taskRunners);
+  JSPUtil.sortTaskRunner(finishedTaskRunners);
+
+  SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>tajo worker</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+  <hr/>
+  <h3>Running Task Containers</h3>
+  <table width="100%" border="1" class="border_table">
+    <tr><th>ContainerId</th><th>StartTime</th><th>FinishTime</th><th>RunTime</th><th>Status</th></tr>
+<%
+      for(TaskRunner eachTaskRunner: taskRunners) {
+%>
+    <tr>
+      <td><a href="tasks.jsp?containerId=<%=eachTaskRunner.getId()%>"><%=eachTaskRunner.getId()%></a></td>
+      <td><%=df.format(eachTaskRunner.getStartTime())%></td>
+      <td><%=eachTaskRunner.getFinishTime() == 0 ? "-" : df.format(eachTaskRunner.getFinishTime())%></td>
+      <td><%=JSPUtil.getElapsedTime(eachTaskRunner.getStartTime(), eachTaskRunner.getFinishTime())%></td>
+      <td><%=eachTaskRunner.getServiceState()%></td>
+<%
+  }
+%>
+  </table>
+  <p/>
+  <hr/>
+  <h3>Finished Task Containers</h3>
+  <table width="100%" border="1" class="border_table">
+    <tr><th>ContainerId</th><th>StartTime</th><th>FinishTime</th><th>RunTime</th><th>Status</th></tr>
+<%
+      for(TaskRunner eachTaskRunner: finishedTaskRunners) {
+%>
+    <tr>
+      <td><a href="tasks.jsp?containerId=<%=eachTaskRunner.getId()%>"><%=eachTaskRunner.getId()%></a></td>
+      <td><%=df.format(eachTaskRunner.getStartTime())%></td>
+      <td><%=eachTaskRunner.getFinishTime() == 0 ? "-" : df.format(eachTaskRunner.getFinishTime())%></td>
+      <td><%=JSPUtil.getElapsedTime(eachTaskRunner.getStartTime(), eachTaskRunner.getFinishTime())%></td>
+      <td><%=eachTaskRunner.getServiceState()%></td>
+<%
+  }
+%>
+  </table>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp
new file mode 100644
index 0000000..b264081
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp
@@ -0,0 +1,127 @@
+<%
+    /*
+    * 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.
+    */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.QueryUnitAttemptId" %>
+<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.TajoWorker" %>
+<%@ page import="org.apache.tajo.worker.Task" %>
+<%@ page import="org.apache.tajo.worker.TaskHistory" %>
+<%@ page import="org.apache.tajo.worker.TaskRunner" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+
+<%
+    TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+
+    String containerId = request.getParameter("containerId");
+    String quAttemptId = request.getParameter("queryUnitAttemptId");
+    QueryUnitAttemptId queryUnitAttemptId = TajoIdUtils.parseQueryUnitAttemptId(quAttemptId);
+    Task task = null;
+    TaskHistory taskHistory = null;
+    if(containerId == null || containerId.isEmpty() || "null".equals(containerId)) {
+        task = tajoWorker.getWorkerContext().getTaskRunnerManager().findTaskByQueryUnitAttemptId(queryUnitAttemptId);
+        if (task != null) {
+            taskHistory = task.getTaskHistory();
+        } else {
+            taskHistory = tajoWorker.getWorkerContext().getTaskRunnerManager().findTaskHistoryByQueryUnitAttemptId(queryUnitAttemptId);
+        }
+    } else {
+        TaskRunner taskRunner = tajoWorker.getWorkerContext().getTaskRunnerManager().findTaskRunner(containerId);
+        if(taskRunner != null) {
+            task = taskRunner.getContext().getTask(queryUnitAttemptId);
+            if (task != null) {
+                taskHistory = task.getTaskHistory();
+            } else {
+                taskHistory = taskRunner.getContext().getTaskHistory(queryUnitAttemptId);
+            }
+        }
+    }
+    if(taskHistory == null) {
+%>
+<script type="text/javascript">
+    alert("No Task Info for" + quAttemptId);
+    document.history.back();
+</script>
+<%
+        return;
+    }
+
+    SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+    <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+    <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+    <title>tajo worker</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+    <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+    <hr/>
+    <h3>Task Detail: <%=quAttemptId%></h3>
+    <table border="1" width="100%" class="border_table">
+        <tr><td width="200" align="right">ID</td><td><%=quAttemptId%></td></tr>
+        <tr><td align="right">State</td><td><%=taskHistory.getStatus()%></td></tr>
+        <tr><td align="right">Start Time</td><td><%=taskHistory.getStartTime() == 0 ? "-" : df.format(taskHistory.getStartTime())%></td></tr>
+        <tr><td align="right">Finish Time</td><td><%=taskHistory.getFinishTime() == 0 ? "-" : df.format(taskHistory.getFinishTime())%></td></tr>
+        <tr><td align="right">Running Time</td><td><%=JSPUtil.getElapsedTime(taskHistory.getStartTime(), taskHistory.getFinishTime())%></td></tr>
+        <tr><td align="right">Progress</td><td><%=JSPUtil.percentFormat(taskHistory.getProgress())%>%</td></tr>
+        <tr><td align="right">Output Path</td><td><%=taskHistory.getOutputPath()%></td></tr>
+        <tr><td align="right">Working Path</td><td><%=taskHistory.getWorkingPath()%></td></tr>
+        <tr><td align="right">Input Statistics</td><td><%=TaskHistory.toInputStatsString(taskHistory.getInputStats())%></td></tr>
+        <tr><td align="right">Output Statistics</td><td><%=TaskHistory.toOutputStatsString(taskHistory.getOutputStats())%></td></tr>
+    </table>
+
+<%
+    if(taskHistory.hasFetcher()) {
+%>
+    <hr/>
+    <h3>Fetch Status</h3>
+    <table border="1" width="100%" class="border_table">
+        <tr><th>No</th><th>StartTime</th><th>FinishTime</th><th>RunTime</th><th>Status</th><th>File Length</th><th># Messages</th><th>URI</th></tr>
+<%
+    int index = 1;
+    for(TaskHistory.FetcherHistory eachFetcher: taskHistory.getFetchers()) {
+%>
+        <tr>
+            <td><%=index%></td>
+            <td><%=df.format(eachFetcher.getStartTime())%></td>
+            <td><%=eachFetcher.getFinishTime() == 0 ? "-" : df.format(eachFetcher.getFinishTime())%></td>
+            <td><%=JSPUtil.getElapsedTime(eachFetcher.getStartTime(), eachFetcher.getFinishTime())%></td>
+            <td><%=eachFetcher.getStatus()%></td>
+            <td align="right"><%=eachFetcher.getFileLen()%></td>
+            <td align="right"><%=eachFetcher.getMessageReceiveCount()%></td>
+            <td><a href="<%=eachFetcher.getUri()%>"><%=eachFetcher.getUri()%></a></td>
+        </tr>
+<%
+        index++;
+    }
+%>
+    </table>
+<%
+    }
+%>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/tasks.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/tasks.jsp b/tajo-core/src/main/resources/webapps/worker/tasks.jsp
new file mode 100644
index 0000000..7b65989
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/tasks.jsp
@@ -0,0 +1,94 @@
+<%
+    /*
+    * 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.
+    */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="java.util.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.*" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+<%@ page import="org.apache.tajo.QueryUnitAttemptId" %>
+
+<%
+    String containerId = request.getParameter("containerId");
+    TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+
+    TaskRunner taskRunner = tajoWorker.getWorkerContext().getTaskRunnerManager().findTaskRunner(containerId);
+    if(taskRunner == null) {
+%>
+<script type="text/javascript">
+    alert("No Task Container for" + containerId);
+    document.history.back();
+</script>
+<%
+        return;
+    }
+
+    TaskRunner.TaskRunnerContext taskRunnerContext = taskRunner.getContext();
+    SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+    <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+    <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+    <title>tajo worker</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+    <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+    <hr/>
+    <h3>Tasks</h3>
+    <table width="100%" border="1" class="border_table">
+        <tr><th>Id</th><th>StartTime</th><th>FinishTime</th><th>RunTime</th><th>Status</th></tr>
+<%
+    for(Map.Entry<QueryUnitAttemptId, Task> entry: taskRunnerContext.getTasks().entrySet()) {
+        QueryUnitAttemptId queryUnitId = entry.getKey();
+        TaskHistory eachTask = entry.getValue().getTaskHistory();
+%>
+        <tr>
+            <td><a href="taskdetail.jsp?containerId=<%=containerId%>&queryUnitAttemptId=<%=queryUnitId%>"><%=queryUnitId%></a></td>
+            <td><%=df.format(eachTask.getStartTime())%></td>
+            <td><%=eachTask.getFinishTime() == 0 ? "-" : df.format(eachTask.getFinishTime())%></td>
+            <td><%=JSPUtil.getElapsedTime(eachTask.getStartTime(), eachTask.getFinishTime())%></td>
+            <td><%=eachTask.getStatus()%></td>
+        </tr>
+<%
+    }
+
+    for(Map.Entry<QueryUnitAttemptId, TaskHistory> entry: taskRunnerContext.getTaskHistories().entrySet()) {
+        QueryUnitAttemptId queryUnitId = entry.getKey();
+        TaskHistory eachTask = entry.getValue();
+%>
+        <tr>
+            <td><a href="taskdetail.jsp?containerId=<%=containerId%>&queryUnitAttemptId=<%=queryUnitId%>"><%=queryUnitId%></a></td>
+            <td><%=df.format(eachTask.getStartTime())%></td>
+            <td><%=eachTask.getFinishTime() == 0 ? "-" : df.format(eachTask.getFinishTime())%></td>
+            <td><%=JSPUtil.getElapsedTime(eachTask.getStartTime(), eachTask.getFinishTime())%></td>
+            <td><%=eachTask.getStatus()%></td>
+        </tr>
+<%
+    }
+%>
+    </table>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/worker/thread.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/thread.jsp b/tajo-core/src/main/resources/webapps/worker/thread.jsp
new file mode 100644
index 0000000..7489741
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/worker/thread.jsp
@@ -0,0 +1,45 @@
+<%
+  /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements. See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership. The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License. You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+%>
+<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
+
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.TajoWorker" %>
+
+<%
+  TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
+%>
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+  <link rel="stylesheet" type = "text/css" href = "/static/style.css" />
+  <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>tajo worker</title>
+</head>
+<body>
+<%@ include file="header.jsp"%>
+<div class='contents'>
+  <h2>Tajo Worker: <a href='index.jsp'><%=tajoWorker.getWorkerContext().getWorkerName()%></a></h2>
+  <hr/>
+  <h3>Thread Dump</h3>
+  <pre><%tajoWorker.dumpThread(out);%></pre>
+</div>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/log4j.properties b/tajo-core/src/test/java/log4j.properties
new file mode 100644
index 0000000..749124c
--- /dev/null
+++ b/tajo-core/src/test/java/log4j.properties
@@ -0,0 +1,28 @@
+##
+# 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.
+#
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java b/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
new file mode 100644
index 0000000..0064e41
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
@@ -0,0 +1,76 @@
+/**
+ * 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;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.*;
+
+import java.io.IOException;
+
+public class BackendTestingUtil {
+	public final static Schema mockupSchema;
+	public final static TableMeta mockupMeta;
+
+	static {
+    mockupSchema = new Schema();
+    mockupSchema.addColumn("deptname", Type.TEXT);
+    mockupSchema.addColumn("score", Type.INT4);
+    mockupMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+	}
+
+  public static void writeTmpTable(TajoConf conf, Path tablePath)
+      throws IOException {
+    AbstractStorageManager sm = StorageManagerFactory.getStorageManager(conf, tablePath);
+    FileSystem fs = sm.getFileSystem();
+
+    Appender appender;
+
+    Path filePath = new Path(tablePath, "table.csv");
+    if (fs.exists(tablePath)) {
+      fs.delete(tablePath, true);
+    }
+    fs.mkdirs(tablePath);
+
+    appender = sm.getAppender(mockupMeta, mockupSchema, filePath);
+    appender.init();
+
+    int deptSize = 10000;
+    int tupleNum = 100;
+    Tuple tuple;
+    for (int i = 0; i < tupleNum; i++) {
+      tuple = new VTuple(2);
+      String key = "test" + (i % deptSize);
+      tuple.put(0, DatumFactory.createText(key));
+      tuple.put(1, DatumFactory.createInt4(i + 1));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/IntegrationTest.java b/tajo-core/src/test/java/org/apache/tajo/IntegrationTest.java
new file mode 100644
index 0000000..755b90a
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/IntegrationTest.java
@@ -0,0 +1,22 @@
+/**
+ * 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;
+
+public interface IntegrationTest {
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java b/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
new file mode 100644
index 0000000..e651313
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
@@ -0,0 +1,136 @@
+/**
+ * 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;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.util.UUID;
+
+public class LocalTajoTestingUtility {
+  private static final Log LOG = LogFactory.getLog(LocalTajoTestingUtility.class);
+
+  private TajoTestingCluster util;
+  private TajoConf conf;
+  private TajoClient client;
+
+  private static UserGroupInformation dummyUserInfo;
+
+  static {
+    try {
+      dummyUserInfo = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private static int taskAttemptId;
+
+  public static QueryUnitAttemptId newQueryUnitAttemptId() {
+    return QueryIdFactory.newQueryUnitAttemptId(
+        QueryIdFactory.newQueryUnitId(new MasterPlan(newQueryId(), null, null).newExecutionBlockId()), taskAttemptId++);
+  }
+  public static QueryUnitAttemptId newQueryUnitAttemptId(MasterPlan plan) {
+    return QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(plan.newExecutionBlockId()), 0);
+  }
+  public static Session createDummySession() {
+    return new Session(UUID.randomUUID().toString(), dummyUserInfo.getUserName(), TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  /**
+   * for test
+   * @return The generated QueryId
+   */
+  public synchronized static QueryId newQueryId() {
+    return QueryIdFactory.newQueryId(TajoIdUtils.MASTER_ID_FORMAT.format(0));
+  }
+
+  public void setup(String[] names,
+                    String[] tablepaths,
+                    Schema[] schemas,
+                    Options option) throws Exception {
+    LOG.info("===================================================");
+    LOG.info("Starting Test Cluster.");
+    LOG.info("===================================================");
+
+    util = new TajoTestingCluster();
+    util.startMiniCluster(1);
+    conf = util.getConfiguration();
+    client = new TajoClient(conf);
+
+    FileSystem fs = util.getDefaultFileSystem();
+    Path rootDir = util.getMaster().getStorageManager().getWarehouseDir();
+    fs.mkdirs(rootDir);
+    for (int i = 0; i < tablepaths.length; i++) {
+      Path localPath = new Path(tablepaths[i]);
+      Path tablePath = new Path(rootDir, names[i]);
+      fs.mkdirs(tablePath);
+      Path dfsPath = new Path(tablePath, localPath.getName());
+      fs.copyFromLocalFile(localPath, dfsPath);
+      TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV, option);
+
+      // Add fake table statistic data to tables.
+      // It gives more various situations to unit tests.
+      TableStats stats = new TableStats();
+      stats.setNumBytes(TPCH.tableVolumes.get(names[i]));
+      TableDesc tableDesc = new TableDesc(
+          CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, names[i]), schemas[i], meta,
+          tablePath);
+      tableDesc.setStats(stats);
+      util.getMaster().getCatalog().createTable(tableDesc);
+    }
+
+    LOG.info("===================================================");
+    LOG.info("Test Cluster ready and test table created.");
+    LOG.info("===================================================");
+
+  }
+
+  public TajoTestingCluster getTestingCluster() {
+    return util;
+  }
+
+  public ResultSet execute(String query) throws IOException, ServiceException {
+    return client.executeQueryAndGetResult(query);
+  }
+
+  public void shutdown() throws IOException {
+    if(client != null) {
+      client.close();
+    }
+    if(util != null) {
+      util.shutdownMiniCluster();
+    }
+  }
+}


[20/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java
new file mode 100644
index 0000000..64c6fc6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java
@@ -0,0 +1,24 @@
+/**
+ * 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.master.session;
+
+public enum SessionEventType {
+  EXPIRE,
+  PING
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
new file mode 100644
index 0000000..483920f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
@@ -0,0 +1,53 @@
+/**
+ * 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.master.session;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tajo.conf.TajoConf;
+
+public class SessionLivelinessMonitor extends AbstractLivelinessMonitor<String> {
+
+  private EventHandler dispatcher;
+
+  public SessionLivelinessMonitor(Dispatcher d) {
+    super(SessionLivelinessMonitor.class.getSimpleName(), new SystemClock());
+    this.dispatcher = d.getEventHandler();
+  }
+
+  public void serviceInit(Configuration conf) throws Exception {
+    Preconditions.checkArgument(conf instanceof TajoConf);
+    TajoConf systemConf = (TajoConf) conf;
+
+    // seconds
+    int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.CLIENT_SESSION_EXPIRY_TIME);
+    setExpireInterval(expireIntvl);
+    setMonitorInterval(expireIntvl / 3);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void expire(String id) {
+    dispatcher.handle(new SessionEvent(id, SessionEventType.EXPIRE));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java
new file mode 100644
index 0000000..24df9d8
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java
@@ -0,0 +1,139 @@
+/**
+ * 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.master.session;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class SessionManager extends CompositeService implements EventHandler<SessionEvent> {
+  private static final Log LOG = LogFactory.getLog(SessionManager.class);
+
+  public final ConcurrentHashMap<String, Session> sessions = new ConcurrentHashMap<String, Session>();
+  private final Dispatcher dispatcher;
+  private SessionLivelinessMonitor sessionLivelinessMonitor;
+
+
+  public SessionManager(Dispatcher dispatcher) {
+    super(SessionManager.class.getSimpleName());
+    this.dispatcher = dispatcher;
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    sessionLivelinessMonitor = new SessionLivelinessMonitor(dispatcher);
+    addIfService(sessionLivelinessMonitor);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    super.serviceStop();
+  }
+
+  private void assertSessionExistence(String sessionId) throws InvalidSessionException {
+    if (!sessions.containsKey(sessionId)) {
+      throw new InvalidSessionException(sessionId);
+    }
+  }
+
+  public String createSession(String username, String baseDatabaseName) throws InvalidSessionException {
+    String sessionId;
+    Session oldSession;
+
+    sessionId = UUID.randomUUID().toString();
+    Session newSession = new Session(sessionId, username, baseDatabaseName);
+    oldSession = sessions.putIfAbsent(sessionId, newSession);
+    if (oldSession != null) {
+      throw new InvalidSessionException("Session id is duplicated: " + oldSession.getSessionId());
+    }
+    LOG.info("Session " + sessionId + " is created." );
+    return sessionId;
+  }
+
+  public void removeSession(String sessionId) {
+    if (sessions.containsKey(sessionId)) {
+      sessions.remove(sessionId);
+      LOG.info("Session " + sessionId + " is removed.");
+    } else {
+      LOG.error("No such session id: " + sessionId);
+    }
+  }
+
+  public Session getSession(String sessionId) throws InvalidSessionException {
+    assertSessionExistence(sessionId);
+    touch(sessionId);
+    return sessions.get(sessionId);
+  }
+
+  public void setVariable(String sessionId, String name, String value) throws InvalidSessionException {
+    assertSessionExistence(sessionId);
+    touch(sessionId);
+    sessions.get(sessionId).setVariable(name, value);
+  }
+
+  public String getVariable(String sessionId, String name)
+      throws InvalidSessionException, NoSuchSessionVariableException {
+    assertSessionExistence(sessionId);
+    touch(sessionId);
+    return sessions.get(sessionId).getVariable(name);
+  }
+
+  public void removeVariable(String sessionId, String name) throws InvalidSessionException {
+    assertSessionExistence(sessionId);
+    touch(sessionId);
+    sessions.get(sessionId).removeVariable(name);
+  }
+
+  public Map<String, String> getAllVariables(String sessionId) throws InvalidSessionException {
+    assertSessionExistence(sessionId);
+    touch(sessionId);
+    return sessions.get(sessionId).getAllVariables();
+  }
+
+  public void touch(String sessionId) throws InvalidSessionException {
+    assertSessionExistence(sessionId);
+    sessions.get(sessionId).updateLastAccessTime();
+    sessionLivelinessMonitor.receivedPing(sessionId);
+  }
+
+  @Override
+  public void handle(SessionEvent event) {
+    LOG.info("Processing " + event.getSessionId() + " of type " + event.getType());
+
+    try {
+      assertSessionExistence(event.getSessionId());
+      touch(event.getSessionId());
+    } catch (InvalidSessionException e) {
+      LOG.error(e);
+    }
+
+    if (event.getType() == SessionEventType.EXPIRE) {
+      Session session = sessions.remove(event.getSessionId());
+      LOG.info("[Expired] Session username=" + session.getUserName() + ",sessionid=" + event.getSessionId());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/net/CachedDNSResolver.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/net/CachedDNSResolver.java b/tajo-core/src/main/java/org/apache/tajo/net/CachedDNSResolver.java
new file mode 100644
index 0000000..2a53c47
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/net/CachedDNSResolver.java
@@ -0,0 +1,64 @@
+/**
+ * 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.net;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class CachedDNSResolver {
+  private static Map<String, String> hostNameToIPAddrMap
+      = new ConcurrentHashMap<String, String>();
+
+  private static CachedDNSResolver instance;
+
+  static {
+    instance = new CachedDNSResolver();
+  }
+
+  public static String resolve(String hostName) {
+
+    if (hostNameToIPAddrMap.containsKey(hostName)) {
+      return hostNameToIPAddrMap.get(hostName);
+    }
+
+    String ipAddress = null;
+    try {
+      ipAddress = InetAddress.getByName(hostName).getHostAddress();
+    } catch (UnknownHostException e) {
+      e.printStackTrace();
+    }
+    hostNameToIPAddrMap.put(hostName, ipAddress);
+
+    return ipAddress;
+  }
+
+  public static String [] resolve(String [] hostNames) {
+    if (hostNames == null) {
+      return null;
+    }
+
+    String [] resolved = new String[hostNames.length];
+    for (int i = 0; i < hostNames.length; i++) {
+      resolved[i] = resolve(hostNames[i]);
+    }
+    return resolved;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java b/tajo-core/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
new file mode 100644
index 0000000..1db0c3b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
@@ -0,0 +1,44 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+
+public class ApplicationIdUtils {
+  public static ApplicationAttemptId createApplicationAttemptId(QueryId queryId, int attemptId) {
+    return BuilderUtils.newApplicationAttemptId(queryIdToAppId(queryId), attemptId);
+  }
+
+  public static ApplicationAttemptId createApplicationAttemptId(QueryId queryId) {
+    return BuilderUtils.newApplicationAttemptId(queryIdToAppId(queryId), 1);
+  }
+
+  public static ApplicationId queryIdToAppId(QueryId queryId) {
+    return BuilderUtils.newApplicationId(Long.parseLong(queryId.getId()), queryId.getSeq());
+  }
+
+  public static QueryId appIdToQueryId(YarnProtos.ApplicationIdProto appId) {
+    return QueryIdFactory.newQueryId(appId.getClusterTimestamp(), appId.getId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/ClassUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/ClassUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/ClassUtil.java
new file mode 100644
index 0000000..160b585
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/ClassUtil.java
@@ -0,0 +1,155 @@
+/**
+ * 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.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.File;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
+public abstract class ClassUtil {
+  private static final Log LOG = LogFactory.getLog(ClassUtil.class);
+
+  public static Set<Class> findClasses(Class type, String packageFilter) {
+    Set<Class> classSet = new HashSet<Class>();
+
+    String classpath = System.getProperty("java.class.path");
+    String[] paths = classpath.split(System.getProperty("path.separator"));
+
+    for (String path : paths) {
+      File file = new File(path);
+      if (file.exists()) {
+        findClasses(classSet, file, file, true, type, packageFilter);
+      }
+    }
+
+    return classSet;
+  }
+
+  private static void findClasses(Set<Class> matchedClassSet, File root, File file, boolean includeJars, Class type, String packageFilter) {
+    if (file.isDirectory()) {
+      for (File child : file.listFiles()) {
+        findClasses(matchedClassSet, root, child, includeJars, type, packageFilter);
+      }
+    } else {
+      if (file.getName().toLowerCase().endsWith(".jar") && includeJars) {
+        JarFile jar = null;
+        try {
+          jar = new JarFile(file);
+        } catch (Exception ex) {
+          LOG.error(ex.getMessage(), ex);
+          return;
+        }
+        Enumeration<JarEntry> entries = jar.entries();
+        while (entries.hasMoreElements()) {
+          JarEntry entry = entries.nextElement();
+          String name = entry.getName();
+          int extIndex = name.lastIndexOf(".class");
+          if (extIndex > 0) {
+            String qualifiedClassName = name.substring(0, extIndex).replace("/", ".");
+            if (qualifiedClassName.indexOf(packageFilter) >= 0 && !isTestClass(qualifiedClassName)) {
+              try {
+                Class clazz = Class.forName(qualifiedClassName);
+
+                if (!clazz.isInterface() && isMatch(type, clazz)) {
+                  matchedClassSet.add(clazz);
+                }
+              } catch (ClassNotFoundException e) {
+                LOG.error(e.getMessage(), e);
+              }
+            }
+          }
+        }
+      } else if (file.getName().toLowerCase().endsWith(".class")) {
+        String qualifiedClassName = createClassName(root, file);
+        if (qualifiedClassName.indexOf(packageFilter) >= 0 && !isTestClass(qualifiedClassName)) {
+          try {
+            Class clazz = Class.forName(qualifiedClassName);
+            if (!clazz.isInterface() && isMatch(type, clazz)) {
+              matchedClassSet.add(clazz);
+            }
+          } catch (ClassNotFoundException e) {
+            LOG.error(e.getMessage(), e);
+          }
+        }
+      }
+    }
+  }
+
+  private static boolean isTestClass(String qualifiedClassName) {
+    String className = getClassName(qualifiedClassName);
+    if(className == null) {
+      return false;
+    }
+
+    return className.startsWith("Test");
+  }
+
+  private static boolean isMatch(Class targetClass, Class loadedClass) {
+    if (targetClass.equals(loadedClass)) {
+      return true;
+    }
+
+    Class[] classInterfaces = loadedClass.getInterfaces();
+    if (classInterfaces != null) {
+      for (Class eachInterfaceClass : classInterfaces) {
+        if (eachInterfaceClass.equals(targetClass)) {
+          return true;
+        }
+
+        if (isMatch(targetClass, eachInterfaceClass)) {
+          return true;
+        }
+      }
+    }
+
+    Class superClass = loadedClass.getSuperclass();
+    if (superClass != null) {
+      if (isMatch(targetClass, superClass)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static String getClassName(String qualifiedClassName) {
+    String[] tokens = qualifiedClassName.split("\\.");
+    if (tokens.length == 0) {
+      return qualifiedClassName;
+    }
+    return tokens[tokens.length - 1];
+  }
+
+  private static String createClassName(File root, File file) {
+    StringBuffer sb = new StringBuffer();
+    String fileName = file.getName();
+    sb.append(fileName.substring(0, fileName.lastIndexOf(".class")));
+    file = file.getParentFile();
+    while (file != null && !file.equals(root)) {
+      sb.insert(0, '.').insert(0, file.getName());
+      file = file.getParentFile();
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/GeoIPUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/GeoIPUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/GeoIPUtil.java
new file mode 100644
index 0000000..859b37d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/GeoIPUtil.java
@@ -0,0 +1,46 @@
+/**
+ * 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.util;
+
+import com.maxmind.geoip.LookupService;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+
+import java.io.IOException;
+
+public class GeoIPUtil {
+  private static final Log LOG = LogFactory.getLog(GeoIPUtil.class);
+  private static LookupService lookup;
+
+  static {
+    try {
+      TajoConf conf = new TajoConf();
+      lookup = new LookupService(conf.getVar(ConfVars.GEOIP_DATA),
+          LookupService.GEOIP_MEMORY_CACHE);
+    } catch (IOException e) {
+      LOG.error("Cannot open the geoip data", e);
+    }
+  }
+
+  public static String getCountryCode(String host) {
+    return lookup.getCountry(host).getCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java
new file mode 100644
index 0000000..8816f8f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java
@@ -0,0 +1,148 @@
+/**
+ * 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.util;
+
+import com.google.gson.Gson;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.logical.IndexScanNode;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.storage.fragment.FileFragment;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map.Entry;
+
+public class IndexUtil {
+  public static String getIndexNameOfFrag(FileFragment fragment, SortSpec[] keys) {
+    StringBuilder builder = new StringBuilder(); 
+    builder.append(fragment.getPath().getName() + "_");
+    builder.append(fragment.getStartKey() + "_" + fragment.getEndKey() + "_");
+    for(int i = 0 ; i < keys.length ; i ++) {
+      builder.append(keys[i].getSortKey().getSimpleName()+"_");
+    }
+    builder.append("_index");
+    return builder.toString();
+       
+  }
+  
+  public static String getIndexName(String indexName , SortSpec[] keys) {
+    StringBuilder builder = new StringBuilder();
+    builder.append(indexName + "_");
+    for(int i = 0 ; i < keys.length ; i ++) {
+      builder.append(keys[i].getSortKey().getSimpleName() + "_");
+    }
+    return builder.toString();
+  }
+  
+  public static IndexScanNode indexEval(LogicalPlan plan, ScanNode scanNode,
+      Iterator<Entry<String, String>> iter ) {
+   
+    EvalNode qual = scanNode.getQual();
+    Gson gson = CoreGsonHelper.getInstance();
+    
+    FieldAndValueFinder nodeFinder = new FieldAndValueFinder();
+    qual.preOrder(nodeFinder);
+    LinkedList<EvalNode> nodeList = nodeFinder.getNodeList();
+    
+    int maxSize = Integer.MIN_VALUE;
+    SortSpec[] maxIndex = null;
+    
+    String json;
+    while(iter.hasNext()) {
+      Entry<String , String> entry = iter.next();
+      json = entry.getValue();
+      SortSpec[] sortKey = gson.fromJson(json, SortSpec[].class);
+      if(sortKey.length > nodeList.size()) {
+        /* If the number of the sort key is greater than where condition, 
+         * this index cannot be used
+         * */
+        continue; 
+      } else {
+        boolean[] equal = new boolean[sortKey.length];
+        for(int i = 0 ; i < sortKey.length ; i ++) {
+          for(int j = 0 ; j < nodeList.size() ; j ++) {
+            Column col = ((FieldEval)(nodeList.get(j).getLeftExpr())).getColumnRef();
+            if(col.equals(sortKey[i].getSortKey())) {
+              equal[i] = true;
+            }
+          }
+        }
+        boolean chk = true;
+        for(int i = 0 ; i < equal.length ; i ++) {
+          chk = chk && equal[i];
+        }
+        if(chk) {
+          if(maxSize < sortKey.length) {
+            maxSize = sortKey.length;
+            maxIndex = sortKey;
+          }
+        }
+      }
+    }
+    if(maxIndex == null) {
+      return null;
+    } else {
+      Schema keySchema = new Schema();
+      for(int i = 0 ; i < maxIndex.length ; i ++ ) {
+        keySchema.addColumn(maxIndex[i].getSortKey());
+      }
+      Datum[] datum = new Datum[nodeList.size()];
+      for(int i = 0 ; i < nodeList.size() ; i ++ ) {
+        datum[i] = ((ConstEval)(nodeList.get(i).getRightExpr())).getValue();
+      }
+      
+      return new IndexScanNode(plan.newPID(), scanNode, keySchema , datum , maxIndex);
+    }
+
+  }
+  
+  
+  private static class FieldAndValueFinder implements EvalNodeVisitor {
+    private LinkedList<EvalNode> nodeList = new LinkedList<EvalNode>();
+    
+    public LinkedList<EvalNode> getNodeList () {
+      return this.nodeList;
+    }
+    
+    @Override
+    public void visit(EvalNode node) {
+      switch(node.getType()) {
+      case AND:
+        break;
+      case EQUAL:
+        if( node.getLeftExpr().getType() == EvalType.FIELD
+          && node.getRightExpr().getType() == EvalType.CONST ) {
+          nodeList.add(node);
+        }
+        break;
+      case IS_NULL:
+        if( node.getLeftExpr().getType() == EvalType.FIELD
+          && node.getRightExpr().getType() == EvalType.CONST) {
+          nodeList.add(node);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
new file mode 100644
index 0000000..58a3550
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
@@ -0,0 +1,209 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.FunctionDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.querymaster.QueryUnit;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.worker.TaskRunner;
+
+import java.text.DecimalFormat;
+import java.util.*;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+
+public class JSPUtil {
+  static DecimalFormat decimalF = new DecimalFormat("###.0");
+
+  public static void sortQueryUnit(QueryUnit[] queryUnits, String sortField, String sortOrder) {
+    if(sortField == null || sortField.isEmpty()) {
+      sortField = "id";
+    }
+
+    Arrays.sort(queryUnits, new QueryUnitComparator(sortField, "asc".equals(sortOrder)));
+  }
+
+  public static void sortTaskRunner(List<TaskRunner> taskRunners) {
+    Collections.sort(taskRunners, new Comparator<TaskRunner>() {
+      @Override
+      public int compare(TaskRunner taskRunner, TaskRunner taskRunner2) {
+        return taskRunner.getId().compareTo(taskRunner2.getId());
+      }
+    });
+  }
+
+  public static String getElapsedTime(long startTime, long finishTime) {
+    if(startTime == 0) {
+      return "-";
+    }
+    return finishTime == 0 ? decimalF.format((System.currentTimeMillis() - startTime) / 1000) + " sec"
+        : decimalF.format((finishTime - startTime) / 1000) + " sec";
+  }
+
+  public static String getTajoMasterHttpAddr(Configuration config) {
+    try {
+      TajoConf conf = (TajoConf) config;
+      String [] masterAddr = conf.getVar(ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS).split(":");
+      return masterAddr[0] + ":" + conf.getVar(ConfVars.TAJO_MASTER_INFO_ADDRESS).split(":")[1];
+    } catch (Exception e) {
+      e.printStackTrace();
+      return e.getMessage();
+    }
+  }
+
+  public static List<QueryMasterTask> sortQueryMasterTask(Collection<QueryMasterTask> queryMasterTasks,
+                                                          final boolean desc) {
+    List<QueryMasterTask> queryMasterTaskList = new ArrayList<QueryMasterTask>(queryMasterTasks);
+
+    Collections.sort(queryMasterTaskList, new Comparator<QueryMasterTask>() {
+
+      @Override
+      public int compare(QueryMasterTask task1, QueryMasterTask task2) {
+        if(desc) {
+          return task2.getQueryId().toString().compareTo(task1.getQueryId().toString());
+        } else {
+          return task1.getQueryId().toString().compareTo(task2.getQueryId().toString());
+        }
+      }
+    });
+
+    return queryMasterTaskList;
+  }
+
+  public static List<QueryInProgress> sortQueryInProgress(Collection<QueryInProgress> queryInProgresses,
+                                                          final boolean desc) {
+    List<QueryInProgress> queryProgressList = new ArrayList<QueryInProgress>(queryInProgresses);
+
+    Collections.sort(queryProgressList, new Comparator<QueryInProgress>() {
+      @Override
+      public int compare(QueryInProgress query1, QueryInProgress query2) {
+        if(desc) {
+          return query2.getQueryId().toString().compareTo(query1.getQueryId().toString());
+        } else {
+          return query1.getQueryId().toString().compareTo(query2.getQueryId().toString());
+        }
+      }
+    });
+
+    return queryProgressList;
+  }
+
+  public static List<SubQuery> sortSubQuery(Collection<SubQuery> subQueries) {
+    List<SubQuery> subQueryList = new ArrayList<SubQuery>(subQueries);
+    Collections.sort(subQueryList, new Comparator<SubQuery>() {
+      @Override
+      public int compare(SubQuery subQuery1, SubQuery subQuery2) {
+        long q1StartTime = subQuery1.getStartTime();
+        long q2StartTime = subQuery2.getStartTime();
+
+        q1StartTime = (q1StartTime == 0 ? Long.MAX_VALUE : q1StartTime);
+        q2StartTime = (q2StartTime == 0 ? Long.MAX_VALUE : q2StartTime);
+
+        int result = compareLong(q1StartTime, q2StartTime);
+        if (result == 0) {
+          return subQuery1.getId().toString().compareTo(subQuery2.getId().toString());
+        } else {
+          return result;
+        }
+      }
+    });
+
+    return subQueryList;
+  }
+
+  static class QueryUnitComparator implements Comparator<QueryUnit> {
+    private String sortField;
+    private boolean asc;
+    public QueryUnitComparator(String sortField, boolean asc) {
+      this.sortField = sortField;
+      this.asc = asc;
+    }
+
+    @Override
+    public int compare(QueryUnit queryUnit, QueryUnit queryUnit2) {
+      if(asc) {
+        if("id".equals(sortField)) {
+          return queryUnit.getId().compareTo(queryUnit2.getId());
+        } else if("host".equals(sortField)) {
+          String host1 = queryUnit.getSucceededHost() == null ? "-" : queryUnit.getSucceededHost();
+          String host2 = queryUnit2.getSucceededHost() == null ? "-" : queryUnit2.getSucceededHost();
+          return host1.compareTo(host2);
+        } else if("runTime".equals(sortField)) {
+          return compareLong(queryUnit.getRunningTime(), queryUnit2.getRunningTime());
+        } else if("startTime".equals(sortField)) {
+          return compareLong(queryUnit.getLaunchTime(), queryUnit2.getLaunchTime());
+        } else {
+          return queryUnit.getId().compareTo(queryUnit2.getId());
+        }
+      } else {
+        if("id".equals(sortField)) {
+          return queryUnit2.getId().compareTo(queryUnit.getId());
+        } else if("host".equals(sortField)) {
+          String host1 = queryUnit.getSucceededHost() == null ? "-" : queryUnit.getSucceededHost();
+          String host2 = queryUnit2.getSucceededHost() == null ? "-" : queryUnit2.getSucceededHost();
+          return host2.compareTo(host1);
+        } else if("runTime".equals(sortField)) {
+          if(queryUnit2.getLaunchTime() == 0) {
+            return -1;
+          } else if(queryUnit.getLaunchTime() == 0) {
+            return 1;
+          }
+          return compareLong(queryUnit2.getRunningTime(), queryUnit.getRunningTime());
+        } else if("startTime".equals(sortField)) {
+          return compareLong(queryUnit2.getLaunchTime(), queryUnit.getLaunchTime());
+        } else {
+          return queryUnit2.getId().compareTo(queryUnit.getId());
+        }
+      }
+    }
+  }
+
+  static int compareLong(long a, long b) {
+    if(a > b) {
+      return 1;
+    } else if(a < b) {
+      return -1;
+    } else {
+      return 0;
+    }
+  }
+
+  public static void sortFunctionDesc(List<FunctionDesc> functions) {
+    Collections.sort(functions, new java.util.Comparator<FunctionDesc>() {
+      @Override
+      public int compare(FunctionDesc f1, FunctionDesc f2) {
+        int nameCompared = f1.getSignature().compareTo(f2.getSignature());
+        if(nameCompared != 0) {
+          return nameCompared;
+        } else {
+          return f1.getReturnType().getType().compareTo(f2.getReturnType().getType());
+        }
+      }
+    });
+  }
+
+  static final DecimalFormat PERCENT_FORMAT = new DecimalFormat("###.0");
+  public static String percentFormat(float value) {
+    return PERCENT_FORMAT.format(value * 100.0f);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/GroupNameMetricsFilter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/GroupNameMetricsFilter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/GroupNameMetricsFilter.java
new file mode 100644
index 0000000..a273475
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/GroupNameMetricsFilter.java
@@ -0,0 +1,43 @@
+/**
+ * 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.util.metrics;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+
+public class GroupNameMetricsFilter implements MetricFilter {
+  String groupName;
+
+  public GroupNameMetricsFilter(String groupName) {
+    this.groupName = groupName;
+  }
+  @Override
+  public boolean matches(String name, Metric metric) {
+    if(name != null) {
+      String[] tokens = name.split("\\.");
+      if(groupName.equals(tokens[0])) {
+        return true;
+      } else {
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/LogEventGaugeSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/LogEventGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/LogEventGaugeSet.java
new file mode 100644
index 0000000..6e130ff
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/LogEventGaugeSet.java
@@ -0,0 +1,64 @@
+/**
+ * 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.util.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class LogEventGaugeSet implements MetricSet {
+
+  @Override
+  public Map<String, Metric> getMetrics() {
+    final Map<String, Metric> gauges = new HashMap<String, Metric>();
+
+    gauges.put("Fatal", new Gauge<Long>() {
+      @Override
+      public Long getValue() {
+        return TajoLogEventCounter.getFatal();
+      }
+    });
+
+    gauges.put("Error", new Gauge<Long>() {
+      @Override
+      public Long getValue() {
+        return TajoLogEventCounter.getError();
+      }
+    });
+
+    gauges.put("Warn", new Gauge<Long>() {
+      @Override
+      public Long getValue() {
+        return TajoLogEventCounter.getWarn();
+      }
+    });
+
+    gauges.put("Info", new Gauge<Long>() {
+      @Override
+      public Long getValue() {
+        return TajoLogEventCounter.getInfo();
+      }
+    });
+
+    return gauges;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/MetricsFilterList.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/MetricsFilterList.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/MetricsFilterList.java
new file mode 100644
index 0000000..b2fc6e4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/MetricsFilterList.java
@@ -0,0 +1,43 @@
+/**
+ * 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.util.metrics;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class MetricsFilterList implements MetricFilter {
+  List<MetricFilter> filters = new ArrayList<MetricFilter>();
+
+  public void addMetricFilter(MetricFilter filter) {
+    filters.add(filter);
+  }
+
+  @Override
+  public boolean matches(String name, Metric metric) {
+    for (MetricFilter eachFilter: filters) {
+      if (!eachFilter.matches(name, metric)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/RegexpMetricsFilter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/RegexpMetricsFilter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/RegexpMetricsFilter.java
new file mode 100644
index 0000000..4faa3e7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/RegexpMetricsFilter.java
@@ -0,0 +1,51 @@
+/**
+ * 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.util.metrics;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.regex.Pattern;
+
+public class RegexpMetricsFilter implements MetricFilter {
+  List<Pattern> filterPatterns = new ArrayList<Pattern>();
+
+  public RegexpMetricsFilter(Collection<String> filterExpressions) {
+    for(String eachExpression: filterExpressions) {
+      filterPatterns.add(Pattern.compile(eachExpression));
+    }
+  }
+
+  @Override
+  public boolean matches(String name, Metric metric) {
+    if(filterPatterns.isEmpty()) {
+      return true;
+    }
+
+    for(Pattern eachPattern: filterPatterns) {
+      if(eachPattern.matcher(name).find()) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoLogEventCounter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoLogEventCounter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoLogEventCounter.java
new file mode 100644
index 0000000..3e44b02
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoLogEventCounter.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.util.metrics;
+
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.Level;
+import org.apache.log4j.spi.LoggingEvent;
+
+public class TajoLogEventCounter extends AppenderSkeleton {
+  private static final int FATAL = 0;
+  private static final int ERROR = 1;
+  private static final int WARN = 2;
+  private static final int INFO = 3;
+
+  private static class EventCounts {
+
+    private final long[] counts = {0, 0, 0, 0};
+
+    private synchronized void incr(int i) {
+      ++counts[i];
+    }
+
+    private synchronized long get(int i) {
+      return counts[i];
+    }
+  }
+
+  private static EventCounts counts = new EventCounts();
+
+  public static long getFatal() {
+    return counts.get(FATAL);
+  }
+
+  public static long getError() {
+    return counts.get(ERROR);
+  }
+
+  public static long getWarn() {
+    return counts.get(WARN);
+  }
+
+  public static long getInfo() {
+    return counts.get(INFO);
+  }
+
+  @Override
+  public void append(LoggingEvent event) {
+    Level level = event.getLevel();
+    String levelStr = level.toString();
+
+    if (level == Level.INFO || "INFO".equalsIgnoreCase(levelStr)) {
+      counts.incr(INFO);
+    } else if (level == Level.WARN || "WARN".equalsIgnoreCase(levelStr)) {
+      counts.incr(WARN);
+    } else if (level == Level.ERROR || "ERROR".equalsIgnoreCase(levelStr)) {
+      counts.incr(ERROR);
+    } else if (level == Level.FATAL || "FATAL".equalsIgnoreCase(levelStr)) {
+      counts.incr(FATAL);
+    }
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public boolean requiresLayout() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoMetrics.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoMetrics.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoMetrics.java
new file mode 100644
index 0000000..0e378b2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoMetrics.java
@@ -0,0 +1,133 @@
+/**
+ * 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.util.metrics;
+
+import com.codahale.metrics.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.util.metrics.reporter.TajoMetricsReporter;
+
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TajoMetrics {
+  private static final Log LOG = LogFactory.getLog(TajoMetrics.class);
+
+  protected MetricRegistry metricRegistry;
+  protected AtomicBoolean stop = new AtomicBoolean(false);
+  protected String metricsGroupName;
+
+  public TajoMetrics(String metricsGroupName) {
+    this.metricsGroupName = metricsGroupName;
+    this.metricRegistry = new MetricRegistry();
+  }
+
+  public void stop() {
+    stop.set(true);
+  }
+
+  public MetricRegistry getRegistry() {
+    return metricRegistry;
+  }
+
+  public void report(TajoMetricsReporter reporter) {
+    try {
+      reporter.report(metricRegistry.getGauges(),
+          metricRegistry.getCounters(),
+          metricRegistry.getHistograms(),
+          metricRegistry.getMeters(),
+          metricRegistry.getTimers());
+    } catch (Exception e) {
+      if(LOG.isDebugEnabled()) {
+        LOG.warn("Metric report error:" + e.getMessage(), e);
+      } else {
+        LOG.warn("Metric report error:" + e.getMessage());
+      }
+    }
+  }
+
+  public Map<String, Metric> getMetrics() {
+    return metricRegistry.getMetrics();
+  }
+
+  public SortedMap<String, Gauge> getGuageMetrics(MetricFilter filter) {
+    if(filter == null) {
+      filter = MetricFilter.ALL;
+    }
+    return metricRegistry.getGauges(filter);
+  }
+
+  public SortedMap<String, Counter> getCounterMetrics(MetricFilter filter) {
+    if(filter == null) {
+      filter = MetricFilter.ALL;
+    }
+    return metricRegistry.getCounters(filter);
+  }
+
+  public SortedMap<String, Histogram> getHistogramMetrics(MetricFilter filter) {
+    if(filter == null) {
+      filter = MetricFilter.ALL;
+    }
+    return metricRegistry.getHistograms(filter);
+  }
+
+  public SortedMap<String, Meter> getMeterMetrics(MetricFilter filter) {
+    if(filter == null) {
+      filter = MetricFilter.ALL;
+    }
+    return metricRegistry.getMeters(filter);
+  }
+
+  public SortedMap<String, Timer> getTimerMetrics(MetricFilter filter) {
+    if(filter == null) {
+      filter = MetricFilter.ALL;
+    }
+    return metricRegistry.getTimers(filter);
+  }
+
+  public void register(String contextName, MetricSet metricSet) {
+    metricRegistry.register(MetricRegistry.name(metricsGroupName, contextName), metricSet);
+  }
+
+  public void register(String contextName, String itemName, Gauge gauge) {
+    metricRegistry.register(makeMetricsName(metricsGroupName, contextName, itemName), gauge);
+  }
+
+  public Counter counter(String contextName, String itemName) {
+    return metricRegistry.counter(makeMetricsName(metricsGroupName, contextName, itemName));
+  }
+
+  public Histogram histogram(String contextName, String itemName) {
+    return metricRegistry.histogram(makeMetricsName(metricsGroupName, contextName, itemName));
+  }
+
+  public Meter meter(String contextName, String itemName) {
+    return metricRegistry.meter(makeMetricsName(metricsGroupName, contextName, itemName));
+  }
+
+  public Timer timer(String contextName, String itemName) {
+    return metricRegistry.timer(makeMetricsName(metricsGroupName, contextName, itemName));
+  }
+
+  public static String makeMetricsName(String metricsGroupName, String contextName, String itemName) {
+    return MetricRegistry.name(metricsGroupName, contextName, itemName);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoSystemMetrics.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoSystemMetrics.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoSystemMetrics.java
new file mode 100644
index 0000000..4192ca0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/TajoSystemMetrics.java
@@ -0,0 +1,213 @@
+/**
+ * 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.util.metrics;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.configuration.event.ConfigurationEvent;
+import org.apache.commons.configuration.event.ConfigurationListener;
+import org.apache.commons.configuration.reloading.FileChangedReloadingStrategy;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.util.metrics.reporter.TajoMetricsScheduledReporter;
+
+import java.util.*;
+
+public class TajoSystemMetrics extends TajoMetrics {
+  private static final Log LOG = LogFactory.getLog(TajoSystemMetrics.class);
+
+  private PropertiesConfiguration metricsProps;
+
+  private Thread propertyChangeChecker;
+
+  private String hostAndPort;
+
+  private List<TajoMetricsScheduledReporter> metricsReporters = new ArrayList<TajoMetricsScheduledReporter>();
+
+  private boolean inited = false;
+
+  private String metricsPropertyFileName;
+
+  public TajoSystemMetrics(TajoConf tajoConf, String metricsGroupName, String hostAndPort) {
+    super(metricsGroupName);
+
+    this.hostAndPort = hostAndPort;
+    try {
+      this.metricsPropertyFileName = tajoConf.getVar(TajoConf.ConfVars.METRICS_PROPERTY_FILENAME);
+      this.metricsProps = new PropertiesConfiguration(metricsPropertyFileName);
+      this.metricsProps.addConfigurationListener(new MetricsReloadListener());
+      FileChangedReloadingStrategy reloadingStrategy = new FileChangedReloadingStrategy();
+      reloadingStrategy.setRefreshDelay(5 * 1000);
+      this.metricsProps.setReloadingStrategy(reloadingStrategy);
+    } catch (ConfigurationException e) {
+      LOG.warn(e.getMessage(), e);
+    }
+
+    //PropertiesConfiguration fire configurationChanged after getXXX()
+    //So neeaded calling getXXX periodically
+    propertyChangeChecker = new Thread() {
+      public void run() {
+        while(!stop.get()) {
+          String value = metricsProps.getString("reporter.file");
+          try {
+            Thread.sleep(10 * 1000);
+          } catch (InterruptedException e) {
+          }
+        }
+      }
+    };
+
+    propertyChangeChecker.start();
+  }
+
+  public Collection<TajoMetricsScheduledReporter> getMetricsReporters() {
+    synchronized (metricsReporters) {
+      return Collections.unmodifiableCollection(metricsReporters);
+    }
+  }
+
+  @Override
+  public void stop() {
+    super.stop();
+    if(propertyChangeChecker != null) {
+      propertyChangeChecker.interrupt();
+    }
+    stopAndClearReporter();
+  }
+
+  protected void stopAndClearReporter() {
+    synchronized(metricsReporters) {
+      for(TajoMetricsScheduledReporter eachReporter: metricsReporters) {
+        eachReporter.close();
+      }
+
+      metricsReporters.clear();
+    }
+  }
+
+  public void start() {
+    setMetricsReporter(metricsGroupName);
+
+    String jvmMetricsName = metricsGroupName + "-jvm";
+    setMetricsReporter(jvmMetricsName);
+
+    if(!inited) {
+      metricRegistry.register(MetricRegistry.name(jvmMetricsName, "Heap"), new MemoryUsageGaugeSet());
+      metricRegistry.register(MetricRegistry.name(jvmMetricsName, "File"), new FileDescriptorRatioGauge());
+      metricRegistry.register(MetricRegistry.name(jvmMetricsName, "GC"), new GarbageCollectorMetricSet());
+      metricRegistry.register(MetricRegistry.name(jvmMetricsName, "Thread"), new ThreadStatesGaugeSet());
+      metricRegistry.register(MetricRegistry.name(jvmMetricsName, "Log"), new LogEventGaugeSet());
+    }
+    inited = true;
+  }
+
+  private void setMetricsReporter(String groupName) {
+    //reporter name -> class name
+    Map<String, String> reporters = new HashMap<String, String>();
+
+    List<String> reporterNames = metricsProps.getList(groupName + ".reporters");
+    if(reporterNames.isEmpty()) {
+      LOG.warn("No property " + groupName + ".reporters in " + metricsPropertyFileName);
+      return;
+    }
+
+    Map<String, String> allReporterProperties = new HashMap<String, String>();
+
+    Iterator<String> keys = metricsProps.getKeys();
+    while (keys.hasNext()) {
+      String key = keys.next();
+      String value = metricsProps.getString(key);
+      if(key.indexOf("reporter.") == 0) {
+        String[] tokens = key.split("\\.");
+        if(tokens.length == 2) {
+          reporters.put(tokens[1], value);
+        }
+      } else if(key.indexOf(groupName + ".") == 0) {
+        String[] tokens = key.split("\\.");
+        if(tokens.length > 2) {
+          allReporterProperties.put(key, value);
+        }
+      }
+    }
+
+    synchronized(metricsReporters) {
+      for(String eachReporterName: reporterNames) {
+        if("null".equals(eachReporterName)) {
+          continue;
+        }
+        String reporterClass = reporters.get(eachReporterName);
+        if(reporterClass == null) {
+          LOG.warn("No metrics reporter definition[" + eachReporterName + "] in " + metricsPropertyFileName);
+          continue;
+        }
+
+        Map<String, String> eachMetricsReporterProperties = findMetircsProperties(allReporterProperties,
+            groupName + "." + eachReporterName);
+
+        try {
+          Object reporterObject = Class.forName(reporterClass).newInstance();
+          if(!(reporterObject instanceof TajoMetricsScheduledReporter)) {
+            LOG.warn(reporterClass + " is not subclass of " + TajoMetricsScheduledReporter.class.getCanonicalName());
+            continue;
+          }
+          TajoMetricsScheduledReporter reporter = (TajoMetricsScheduledReporter)reporterObject;
+          reporter.init(metricRegistry, groupName, hostAndPort, eachMetricsReporterProperties);
+          reporter.start();
+
+          metricsReporters.add(reporter);
+          LOG.info("Started metrics reporter " + reporter.getClass().getCanonicalName() + " for " + groupName);
+        } catch (ClassNotFoundException e) {
+          LOG.warn("No metrics reporter class[" + eachReporterName + "], required class= " + reporterClass);
+          continue;
+        } catch (Exception e) {
+          LOG.warn("Can't initiate metrics reporter class[" + eachReporterName + "]" + e.getMessage() , e);
+          continue;
+        }
+      }
+    }
+  }
+
+  private Map<String, String> findMetircsProperties(Map<String, String> allReporterProperties, String findKey) {
+    Map<String, String> metricsProperties = new HashMap<String, String>();
+
+    for (Map.Entry<String, String> entry: allReporterProperties.entrySet()) {
+      String eachKey = entry.getKey();
+      if (eachKey.indexOf(findKey) == 0) {
+        metricsProperties.put(eachKey, entry.getValue());
+      }
+    }
+    return metricsProperties;
+  }
+
+  class MetricsReloadListener implements ConfigurationListener {
+    @Override
+    public synchronized void configurationChanged(ConfigurationEvent event) {
+      if (!event.isBeforeUpdate()) {
+        stopAndClearReporter();
+        start();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/GangliaReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/GangliaReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/GangliaReporter.java
new file mode 100644
index 0000000..b9acf0e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/GangliaReporter.java
@@ -0,0 +1,258 @@
+/**
+ * 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.util.metrics.reporter;
+
+import com.codahale.metrics.*;
+import info.ganglia.gmetric4j.gmetric.GMetric;
+import info.ganglia.gmetric4j.gmetric.GMetricSlope;
+import info.ganglia.gmetric4j.gmetric.GMetricType;
+import info.ganglia.gmetric4j.gmetric.GangliaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.SortedMap;
+
+import static com.codahale.metrics.MetricRegistry.name;
+
+public class GangliaReporter extends TajoMetricsScheduledReporter {
+  private static final Logger LOG = LoggerFactory.getLogger(GangliaReporter.class);
+  public static final String REPORTER_NAME = "ganglia";
+
+  private GMetric ganglia;
+  private String prefix;
+  private int tMax = 60;
+  private int dMax = 0;
+
+  @Override
+  protected String getReporterName() {
+    return REPORTER_NAME;
+  }
+
+  @Override
+  protected void afterInit() {
+    String server = metricsProperties.get(metricsPropertyKey + "server");
+    String port = metricsProperties.get(metricsPropertyKey + "port");
+
+    if(server == null || server.isEmpty()) {
+      LOG.warn("No " + metricsPropertyKey + "server property in tajo-metrics.properties");
+      return;
+    }
+
+    if(port == null || port.isEmpty()) {
+      LOG.warn("No " + metricsPropertyKey + "port property in tajo-metrics.properties");
+      return;
+    }
+
+    try {
+      ganglia = new GMetric(server, Integer.parseInt(port), GMetric.UDPAddressingMode.MULTICAST, 1);
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    }
+  }
+
+  public void setPrefix(String prefix) {
+    this.prefix = prefix;
+  }
+
+  public void settMax(int tMax) {
+    this.tMax = tMax;
+  }
+
+  public void setdMax(int dMax) {
+    this.dMax = dMax;
+  }
+
+  @Override
+  public void report(SortedMap<String, Gauge> gauges,
+                     SortedMap<String, Counter> counters,
+                     SortedMap<String, Histogram> histograms,
+                     SortedMap<String, Meter> meters,
+                     SortedMap<String, Timer> timers) {
+    for (Map.Entry<String, Gauge> entry : gauges.entrySet()) {
+      reportGauge(entry.getKey(), entry.getValue());
+    }
+
+    for (Map.Entry<String, Counter> entry : counters.entrySet()) {
+      reportCounter(entry.getKey(), entry.getValue());
+    }
+
+    for (Map.Entry<String, Histogram> entry : histograms.entrySet()) {
+      reportHistogram(entry.getKey(), entry.getValue());
+    }
+
+    for (Map.Entry<String, Meter> entry : meters.entrySet()) {
+      reportMeter(entry.getKey(), entry.getValue());
+    }
+
+    for (Map.Entry<String, Timer> entry : timers.entrySet()) {
+      reportTimer(entry.getKey(), entry.getValue());
+    }
+  }
+
+  private void reportTimer(String name, Timer timer) {
+    final String group = group(name);
+    try {
+      final Snapshot snapshot = timer.getSnapshot();
+
+      announce(prefix(name, "max"), group, convertDuration(snapshot.getMax()), getDurationUnit());
+      announce(prefix(name, "mean"), group, convertDuration(snapshot.getMean()), getDurationUnit());
+      announce(prefix(name, "min"), group, convertDuration(snapshot.getMin()), getDurationUnit());
+      announce(prefix(name, "stddev"), group, convertDuration(snapshot.getStdDev()), getDurationUnit());
+
+      announce(prefix(name, "p50"), group, convertDuration(snapshot.getMedian()), getDurationUnit());
+      announce(prefix(name, "p75"),
+          group,
+          convertDuration(snapshot.get75thPercentile()),
+          getDurationUnit());
+      announce(prefix(name, "p95"),
+          group,
+          convertDuration(snapshot.get95thPercentile()),
+          getDurationUnit());
+      announce(prefix(name, "p98"),
+          group,
+          convertDuration(snapshot.get98thPercentile()),
+          getDurationUnit());
+      announce(prefix(name, "p99"),
+          group,
+          convertDuration(snapshot.get99thPercentile()),
+          getDurationUnit());
+      announce(prefix(name, "p999"),
+          group,
+          convertDuration(snapshot.get999thPercentile()),
+          getDurationUnit());
+
+      reportMetered(name, timer, group, "calls");
+    } catch (GangliaException e) {
+      LOG.warn("Unable to report timer {}", name, e);
+    }
+  }
+
+  private void reportMeter(String name, Meter meter) {
+    final String group = group(name);
+    try {
+      reportMetered(name, meter, group, "events");
+    } catch (GangliaException e) {
+      LOG.warn("Unable to report meter {}", name, e);
+    }
+  }
+
+  private void reportMetered(String name, Metered meter, String group, String eventName) throws GangliaException {
+    final String unit = eventName + '/' + getRateUnit();
+    announce(prefix(name, "count"), group, meter.getCount(), eventName);
+    announce(prefix(name, "m1_rate"), group, convertRate(meter.getOneMinuteRate()), unit);
+    announce(prefix(name, "m5_rate"), group, convertRate(meter.getFiveMinuteRate()), unit);
+    announce(prefix(name, "m15_rate"), group, convertRate(meter.getFifteenMinuteRate()), unit);
+    announce(prefix(name, "mean_rate"), group, convertRate(meter.getMeanRate()), unit);
+  }
+
+  private void reportHistogram(String name, Histogram histogram) {
+    final String group = group(name);
+    try {
+      final Snapshot snapshot = histogram.getSnapshot();
+
+      announce(prefix(name, "count"), group, histogram.getCount(), "");
+      announce(prefix(name, "max"), group, snapshot.getMax(), "");
+      announce(prefix(name, "mean"), group, snapshot.getMean(), "");
+      announce(prefix(name, "min"), group, snapshot.getMin(), "");
+      announce(prefix(name, "stddev"), group, snapshot.getStdDev(), "");
+      announce(prefix(name, "p50"), group, snapshot.getMedian(), "");
+      announce(prefix(name, "p75"), group, snapshot.get75thPercentile(), "");
+      announce(prefix(name, "p95"), group, snapshot.get95thPercentile(), "");
+      announce(prefix(name, "p98"), group, snapshot.get98thPercentile(), "");
+      announce(prefix(name, "p99"), group, snapshot.get99thPercentile(), "");
+      announce(prefix(name, "p999"), group, snapshot.get999thPercentile(), "");
+    } catch (GangliaException e) {
+      LOG.warn("Unable to report histogram {}", name, e);
+    }
+  }
+
+  private void reportCounter(String name, Counter counter) {
+    final String group = group(name);
+    try {
+      announce(prefix(name, "count"), group, counter.getCount(), "");
+    } catch (GangliaException e) {
+      LOG.warn("Unable to report counter {}", name, e);
+    }
+  }
+
+  private void reportGauge(String name, Gauge gauge) {
+    final String group = group(name);
+    final Object obj = gauge.getValue();
+
+    try {
+      ganglia.announce(name(prefix, name), String.valueOf(obj), detectType(obj), "",
+          GMetricSlope.BOTH, tMax, dMax, group);
+    } catch (GangliaException e) {
+      LOG.warn("Unable to report gauge {}", name, e);
+    }
+  }
+
+  private void announce(String name, String group, double value, String units) throws GangliaException {
+    ganglia.announce(name,
+        Double.toString(value),
+        GMetricType.DOUBLE,
+        units,
+        GMetricSlope.BOTH,
+        tMax,
+        dMax,
+        group);
+  }
+
+  private void announce(String name, String group, long value, String units) throws GangliaException {
+    final String v = Long.toString(value);
+    ganglia.announce(name,
+        v,
+        GMetricType.DOUBLE,
+        units,
+        GMetricSlope.BOTH,
+        tMax,
+        dMax,
+        group);
+  }
+
+  private GMetricType detectType(Object o) {
+    if (o instanceof Float) {
+      return GMetricType.FLOAT;
+    } else if (o instanceof Double) {
+      return GMetricType.DOUBLE;
+    } else if (o instanceof Byte) {
+      return GMetricType.INT8;
+    } else if (o instanceof Short) {
+      return GMetricType.INT16;
+    } else if (o instanceof Integer) {
+      return GMetricType.INT32;
+    } else if (o instanceof Long) {
+      return GMetricType.DOUBLE;
+    }
+    return GMetricType.STRING;
+  }
+
+  private String group(String name) {
+    String[] tokens = name.split("\\.");
+    if(tokens.length < 3) {
+      return "";
+    }
+    return tokens[0] + "." + tokens[1];
+  }
+
+  private String prefix(String name, String n) {
+    return name(prefix, name, n);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleReporter.java
new file mode 100644
index 0000000..80b77f1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleReporter.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.util.metrics.reporter;
+
+import com.codahale.metrics.*;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.concurrent.TimeUnit;
+
+public class MetricsConsoleReporter extends TajoMetricsReporter {
+  @Override
+  public void report(SortedMap<String, Gauge> gauges,
+                     SortedMap<String, Counter> counters,
+                     SortedMap<String, Histogram> histograms,
+                     SortedMap<String, Meter> meters,
+                     SortedMap<String, Timer> timers) {
+    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    final String dateTime = dateFormat.format(new Date());
+    double rateFactor = TimeUnit.SECONDS.toSeconds(1);
+
+    if (!gauges.isEmpty()) {
+      Map<String, Map<String, Gauge>> gaugeGroups = findMetricsItemGroup(gauges);
+
+      for(Map.Entry<String, Map<String, Gauge>> eachGroup: gaugeGroups.entrySet()) {
+        System.out.println(gaugeGroupToString(dateTime, null, rateFactor, eachGroup.getKey(), eachGroup.getValue()));
+      }
+    }
+
+    if (!counters.isEmpty()) {
+      Map<String, Map<String, Counter>> counterGroups = findMetricsItemGroup(counters);
+
+      for(Map.Entry<String, Map<String, Counter>> eachGroup: counterGroups.entrySet()) {
+        System.out.println(counterGroupToString(dateTime, null, rateFactor, eachGroup.getKey(), eachGroup.getValue()));
+      }
+    }
+
+    if (!histograms.isEmpty()) {
+      Map<String, Map<String, Histogram>> histogramGroups = findMetricsItemGroup(histograms);
+
+      for(Map.Entry<String, Map<String, Histogram>> eachGroup: histogramGroups.entrySet()) {
+        System.out.println(histogramGroupToString(dateTime, null, rateFactor, eachGroup.getKey(), eachGroup.getValue()));
+      }
+    }
+
+    if (!meters.isEmpty()) {
+      Map<String, Map<String, Meter>> meterGroups = findMetricsItemGroup(meters);
+
+      for(Map.Entry<String, Map<String, Meter>> eachGroup: meterGroups.entrySet()) {
+        System.out.println(meterGroupToString(dateTime, null, rateFactor, eachGroup.getKey(), eachGroup.getValue()));
+      }
+    }
+
+    if (!timers.isEmpty()) {
+      Map<String, Map<String, Timer>> timerGroups = findMetricsItemGroup(timers);
+
+      for(Map.Entry<String, Map<String, Timer>> eachGroup: timerGroups.entrySet()) {
+        System.out.println(timerGroupToString(dateTime, null, rateFactor, eachGroup.getKey(), eachGroup.getValue()));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleScheduledReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleScheduledReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleScheduledReporter.java
new file mode 100644
index 0000000..286ef8d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsConsoleScheduledReporter.java
@@ -0,0 +1,32 @@
+/**
+ * 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.util.metrics.reporter;
+
+public class MetricsConsoleScheduledReporter extends MetricsStreamScheduledReporter {
+  public static final String REPORTER_NAME = "console";
+  @Override
+  protected String getReporterName() {
+    return REPORTER_NAME;
+  }
+
+  @Override
+  protected void afterInit() {
+    setOutput(System.out);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsFileScheduledReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsFileScheduledReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsFileScheduledReporter.java
new file mode 100644
index 0000000..9e895b8
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsFileScheduledReporter.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.util.metrics.reporter;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+
+public class MetricsFileScheduledReporter extends MetricsStreamScheduledReporter {
+  private static final Log LOG = LogFactory.getLog(MetricsFileScheduledReporter.class);
+  public static final String REPORTER_NAME = "file";
+
+  protected String getReporterName() {
+    return REPORTER_NAME;
+  }
+
+  @Override
+  protected void afterInit() {
+    String fileName = metricsProperties.get(metricsPropertyKey + "filename");
+    if(fileName == null) {
+      LOG.warn("No " + metricsPropertyKey + "filename property in tajo-metrics.properties");
+      return;
+    }
+    try {
+      File file = new File(fileName);
+      File parentFile = file.getParentFile();
+      if(parentFile != null && !parentFile.exists()) {
+        if(!parentFile.mkdirs()) {
+          LOG.warn("Can't create dir for tajo metrics:" + parentFile.getAbsolutePath());
+        }
+      }
+      this.setOutput(new FileOutputStream(fileName, true));
+      this.setDateFormat(null);
+    } catch (FileNotFoundException e) {
+      LOG.warn("Can't open metrics file:" + fileName);
+      this.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsStreamScheduledReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsStreamScheduledReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsStreamScheduledReporter.java
new file mode 100644
index 0000000..4fbefd7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/MetricsStreamScheduledReporter.java
@@ -0,0 +1,179 @@
+/**
+ * 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.util.metrics.reporter;
+
+import com.codahale.metrics.*;
+import com.codahale.metrics.Timer;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+public abstract class MetricsStreamScheduledReporter extends TajoMetricsScheduledReporter {
+  private static final Log LOG = LogFactory.getLog(MetricsStreamScheduledReporter.class);
+
+  protected OutputStream output;
+  protected Locale locale;
+  protected Clock clock;
+  protected TimeZone timeZone;
+  protected MetricFilter filter;
+  protected DateFormat dateFormat;
+
+  private final byte[] NEW_LINE = "\n".getBytes();
+
+  public MetricsStreamScheduledReporter() {
+    dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    clock = Clock.defaultClock();
+  }
+
+  public void setOutput(OutputStream output) {
+    this.output = output;
+  }
+
+  public void setLocale(Locale locale) {
+    this.locale = locale;
+  }
+
+  public void setClock(Clock clock) {
+    this.clock = clock;
+  }
+
+  public void setTimeZone(TimeZone timeZone) {
+    this.dateFormat.setTimeZone(timeZone);
+    this.timeZone = timeZone;
+  }
+
+  public void setDateFormat(DateFormat dateFormat) {
+    this.dateFormat = dateFormat;
+  }
+
+  @Override
+  public void report(SortedMap<String, Gauge> gauges,
+                     SortedMap<String, Counter> counters,
+                     SortedMap<String, Histogram> histograms,
+                     SortedMap<String, Meter> meters,
+                     SortedMap<String, Timer> timers) {
+    final String dateTime = dateFormat == null ? "" + clock.getTime() : dateFormat.format(new Date(clock.getTime()));
+
+    if (!gauges.isEmpty()) {
+      Map<String, Map<String, Gauge>> gaugeGroups = findMetricsItemGroup(gauges);
+
+      for(Map.Entry<String, Map<String, Gauge>> eachGroup: gaugeGroups.entrySet()) {
+        printGaugeGroup(dateTime, eachGroup.getKey(), eachGroup.getValue());
+      }
+    }
+
+    if (!counters.isEmpty()) {
+      Map<String, Map<String, Counter>> counterGroups = findMetricsItemGroup(counters);
+
+      for(Map.Entry<String, Map<String, Counter>> eachGroup: counterGroups.entrySet()) {
+        printCounterGroup(dateTime, eachGroup.getKey(), eachGroup.getValue());
+      }
+    }
+
+    if (!histograms.isEmpty()) {
+      Map<String, Map<String, Histogram>> histogramGroups = findMetricsItemGroup(histograms);
+
+      for(Map.Entry<String, Map<String, Histogram>> eachGroup: histogramGroups.entrySet()) {
+        printHistogramGroup(dateTime, eachGroup.getKey(), eachGroup.getValue());
+      }
+    }
+
+    if (!meters.isEmpty()) {
+      Map<String, Map<String, Meter>> meterGroups = findMetricsItemGroup(meters);
+
+      for(Map.Entry<String, Map<String, Meter>> eachGroup: meterGroups.entrySet()) {
+        printMeterGroup(dateTime, eachGroup.getKey(), eachGroup.getValue());
+      }
+    }
+
+    if (!timers.isEmpty()) {
+      Map<String, Map<String, Timer>> timerGroups = findMetricsItemGroup(timers);
+
+      for(Map.Entry<String, Map<String, Timer>> eachGroup: timerGroups.entrySet()) {
+        printTimerGroup(dateTime, eachGroup.getKey(), eachGroup.getValue());
+      }
+    }
+    try {
+      output.flush();
+    } catch (IOException e) {
+    }
+  }
+
+  private void printMeterGroup(String dateTime, String groupName, Map<String, Meter> meters) {
+    try {
+      output.write(meterGroupToString(dateTime, hostAndPort, rateFactor, groupName, meters).getBytes());
+      output.write(NEW_LINE);
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    }
+  }
+
+  private void printCounterGroup(String dateTime, String groupName, Map<String, Counter> counters) {
+    try {
+      output.write(counterGroupToString(dateTime, hostAndPort, rateFactor, groupName, counters).getBytes());
+      output.write(NEW_LINE);
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    }
+  }
+
+  private void printGaugeGroup(String dateTime, String groupName, Map<String, Gauge> gauges) {
+    try {
+      output.write(gaugeGroupToString(dateTime, hostAndPort, rateFactor, groupName, gauges).getBytes());
+      output.write(NEW_LINE);
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    }
+  }
+
+  private void printHistogramGroup(String dateTime, String groupName, Map<String, Histogram> histograms) {
+    try {
+      output.write(histogramGroupToString(dateTime, hostAndPort, rateFactor, groupName, histograms).getBytes());
+      output.write(NEW_LINE);
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    }
+  }
+
+  private void printTimerGroup(String dateTime, String groupName, Map<String, Timer> timers) {
+    try {
+      output.write(timerGroupToString(dateTime, hostAndPort, rateFactor, groupName, timers).getBytes());
+      output.write(NEW_LINE);
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void close() {
+    if(output != null) {
+      try {
+        output.close();
+      } catch (IOException e) {
+      }
+    }
+
+    super.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/NullReporter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/NullReporter.java b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/NullReporter.java
new file mode 100644
index 0000000..9dc1755
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/metrics/reporter/NullReporter.java
@@ -0,0 +1,31 @@
+/**
+ * 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.util.metrics.reporter;
+
+import com.codahale.metrics.*;
+
+import java.util.SortedMap;
+
+public class NullReporter extends TajoMetricsReporter {
+  @Override
+  public void report(SortedMap<String, Gauge> gauges, SortedMap<String, Counter> counters,
+                     SortedMap<String, Histogram> histograms, SortedMap<String, Meter> meters,
+                     SortedMap<String, Timer> timers) {
+  }
+}


[12/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all.js
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all.js b/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all.js
new file mode 100644
index 0000000..2e35a49
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/static/js/jquery.jsPlumb-1.3.16-all.js
@@ -0,0 +1,10561 @@
+/*
+ * jsPlumb
+ * 
+ * Title:jsPlumb 1.3.16
+ * 
+ * Provides a way to visually connect elements on an HTML page, using either SVG, Canvas
+ * elements, or VML.  
+ * 
+ * This file contains the util functions
+ *
+ * Copyright (c) 2010 - 2012 Simon Porritt (http://jsplumb.org)
+ * 
+ * http://jsplumb.org
+ * http://github.com/sporritt/jsplumb
+ * http://code.google.com/p/jsplumb
+ * 
+ * Dual licensed under the MIT and GPL2 licenses.
+ */
+jsPlumbUtil = {
+    isArray : function(a) {
+        return Object.prototype.toString.call(a) === "[object Array]";
+    },
+    isString : function(s) {
+        return typeof s === "string";
+    },
+    isBoolean: function(s) {
+        return typeof s === "boolean";
+    },
+    isObject : function(o) {
+        return Object.prototype.toString.call(o) === "[object Object]";
+    },
+    isDate : function(o) {
+        return Object.prototype.toString.call(o) === "[object Date]";
+    },
+    isFunction: function(o) {
+        return Object.prototype.toString.call(o) === "[object Function]";
+    },
+    clone : function(a) {
+        if (this.isString(a)) return new String(a);
+        else if (this.isBoolean(a)) return new Boolean(a);
+        else if (this.isDate(a)) return new Date(a.getTime());
+        else if (this.isFunction(a)) return a;
+        else if (this.isArray(a)) {
+            var b = [];
+            for (var i = 0; i < a.length; i++)
+                b.push(this.clone(a[i]));
+            return b;
+        }
+        else if (this.isObject(a)) {
+            var b = {};
+            for (var i in a)
+                b[i] = this.clone(a[i]);
+            return b;
+        }
+        else return a;
+    },
+    merge : function(a, b) {
+        var c = this.clone(a);
+        for (var i in b) {
+            if (c[i] == null || this.isString(b[i]) || this.isBoolean(b[i]))
+                c[i] = b[i];
+            else {
+                if (this.isArray(b[i]) && this.isArray(c[i])) {
+                    var ar = [];
+                    ar.push.apply(ar, c[i]);
+                    ar.push.apply(ar, b[i]);
+                    c[i] = ar;
+                }
+                else if(this.isObject(c[i]) && this.isObject(b[i])) {
+                    for (var j in b[i])
+                        c[i][j] = b[i][j];
+                }
+            }
+        }
+        return c;
+    },
+    convertStyle : function(s, ignoreAlpha) {
+        // TODO: jsPlumb should support a separate 'opacity' style member.
+        if ("transparent" === s) return s;
+        var o = s,
+            pad = function(n) { return n.length == 1 ? "0" + n : n; },
+            hex = function(k) { return pad(Number(k).toString(16)); },
+            pattern = /(rgb[a]?\()(.*)(\))/;
+        if (s.match(pattern)) {
+            var parts = s.match(pattern)[2].split(",");
+            o = "#" + hex(parts[0]) + hex(parts[1]) + hex(parts[2]);
+            if (!ignoreAlpha && parts.length == 4)
+                o = o + hex(parts[3]);
+        }
+        return o;
+    },
+    gradient : function(p1, p2) {
+        p1 = jsPlumbUtil.isArray(p1) ? p1 : [p1.x, p1.y];
+        p2 = jsPlumbUtil.isArray(p2) ? p2 : [p2.x, p2.y];
+        return (p2[1] - p1[1]) / (p2[0] - p1[0]);
+    },
+    normal : function(p1, p2) {
+        return -1 / jsPlumbUtil.gradient(p1,p2);
+    },
+    lineLength : function(p1, p2) {
+        p1 = jsPlumbUtil.isArray(p1) ? p1 : [p1.x, p1.y];
+        p2 = jsPlumbUtil.isArray(p2) ? p2 : [p2.x, p2.y];
+        return Math.sqrt(Math.pow(p2[1] - p1[1], 2) + Math.pow(p2[0] - p1[0], 2));
+    },
+    segment : function(p1, p2) {
+        p1 = jsPlumbUtil.isArray(p1) ? p1 : [p1.x, p1.y];
+        p2 = jsPlumbUtil.isArray(p2) ? p2 : [p2.x, p2.y];
+        if (p2[0] > p1[0]) {
+            return (p2[1] > p1[1]) ? 2 : 1;
+        }
+        else {
+            return (p2[1] > p1[1]) ? 3 : 4;
+        }
+    },
+    intersects : function(r1, r2) {
+        var x1 = r1.x, x2 = r1.x + r1.w, y1 = r1.y, y2 = r1.y + r1.h,
+            a1 = r2.x, a2 = r2.x + r2.w, b1 = r2.y, b2 = r2.y + r2.h;
+
+        return  ( (x1 <= a1 && a1 <= x2) && (y1 <= b1 && b1 <= y2) ) ||
+            ( (x1 <= a2 && a2 <= x2) && (y1 <= b1 && b1 <= y2) ) ||
+            ( (x1 <= a1 && a1 <= x2) && (y1 <= b2 && b2 <= y2) ) ||
+            ( (x1 <= a2 && a1 <= x2) && (y1 <= b2 && b2 <= y2) ) ||
+
+            ( (a1 <= x1 && x1 <= a2) && (b1 <= y1 && y1 <= b2) ) ||
+            ( (a1 <= x2 && x2 <= a2) && (b1 <= y1 && y1 <= b2) ) ||
+            ( (a1 <= x1 && x1 <= a2) && (b1 <= y2 && y2 <= b2) ) ||
+            ( (a1 <= x2 && x1 <= a2) && (b1 <= y2 && y2 <= b2) );
+    },
+    segmentMultipliers : [null, [1, -1], [1, 1], [-1, 1], [-1, -1] ],
+    inverseSegmentMultipliers : [null, [-1, -1], [-1, 1], [1, 1], [1, -1] ],
+    pointOnLine : function(fromPoint, toPoint, distance) {
+        var m = jsPlumbUtil.gradient(fromPoint, toPoint),
+            s = jsPlumbUtil.segment(fromPoint, toPoint),
+            segmentMultiplier = distance > 0 ? jsPlumbUtil.segmentMultipliers[s] : jsPlumbUtil.inverseSegmentMultipliers[s],
+            theta = Math.atan(m),
+            y = Math.abs(distance * Math.sin(theta)) * segmentMultiplier[1],
+            x =  Math.abs(distance * Math.cos(theta)) * segmentMultiplier[0];
+        return { x:fromPoint.x + x, y:fromPoint.y + y };
+    },
+    /**
+     * calculates a perpendicular to the line fromPoint->toPoint, that passes through toPoint and is 'length' long.
+     * @param fromPoint
+     * @param toPoint
+     * @param length
+     */
+    perpendicularLineTo : function(fromPoint, toPoint, length) {
+        var m = jsPlumbUtil.gradient(fromPoint, toPoint),
+            theta2 = Math.atan(-1 / m),
+            y =  length / 2 * Math.sin(theta2),
+            x =  length / 2 * Math.cos(theta2);
+        return [{x:toPoint.x + x, y:toPoint.y + y}, {x:toPoint.x - x, y:toPoint.y - y}];
+    },
+    findWithFunction : function(a, f) {
+        if (a)
+            for (var i = 0; i < a.length; i++) if (f(a[i])) return i;
+        return -1;
+    },
+    indexOf : function(l, v) {
+        return jsPlumbUtil.findWithFunction(l, function(_v) { return _v == v; });
+    },
+    removeWithFunction : function(a, f) {
+        var idx = jsPlumbUtil.findWithFunction(a, f);
+        if (idx > -1) a.splice(idx, 1);
+        return idx != -1;
+    },
+    remove : function(l, v) {
+        var idx = jsPlumbUtil.indexOf(l, v);
+        if (idx > -1) l.splice(idx, 1);
+        return idx != -1;
+    },
+    // TODO support insert index
+    addWithFunction : function(list, item, hashFunction) {
+        if (jsPlumbUtil.findWithFunction(list, hashFunction) == -1) list.push(item);
+    },
+    addToList : function(map, key, value) {
+        var l = map[key];
+        if (l == null) {
+            l = [], map[key] = l;
+        }
+        l.push(value);
+        return l;
+    },
+    /**
+     * EventGenerator
+     * Superclass for objects that generate events - jsPlumb extends this, as does jsPlumbUIComponent, which all the UI elements extend.
+     */
+    EventGenerator : function() {
+        var _listeners = {}, self = this;
+
+        // this is a list of events that should re-throw any errors that occur during their dispatch. as of 1.3.0 this is private to
+        // jsPlumb, but it seems feasible that people might want to manipulate this list.  the thinking is that we don't want event
+        // listeners to bring down jsPlumb - or do we.  i can't make up my mind about this, but i know i want to hear about it if the "ready"
+        // event fails, because then my page has most likely not initialised.  so i have this halfway-house solution.  it will be interesting
+        // to hear what other people think.
+        var eventsToDieOn = [ "ready" ];
+
+        /*
+         * Binds a listener to an event.  
+         * 
+         * Parameters:
+         * 	event		-	name of the event to bind to.
+         * 	listener	-	function to execute.
+         */
+        this.bind = function(event, listener) {
+            jsPlumbUtil.addToList(_listeners, event, listener);
+            return self;
+        };
+        /*
+         * Fires an update for the given event.
+         * 
+         * Parameters:
+         * 	event				-	event to fire
+         * 	value				-	value to pass to the event listener(s).
+         *  originalEvent	 	- 	the original event from the browser
+         */
+        this.fire = function(event, value, originalEvent) {
+            if (_listeners[event]) {
+                for ( var i = 0; i < _listeners[event].length; i++) {
+                    // doing it this way rather than catching and then possibly re-throwing means that an error propagated by this
+                    // method will have the whole call stack available in the debugger.
+                    if (jsPlumbUtil.findWithFunction(eventsToDieOn, function(e) { return e === event}) != -1)
+                        _listeners[event][i](value, originalEvent);
+                    else {
+                        // for events we don't want to die on, catch and log.
+                        try {
+                            _listeners[event][i](value, originalEvent);
+                        } catch (e) {
+                            jsPlumbUtil.log("jsPlumb: fire failed for event " + event + " : " + e);
+                        }
+                    }
+                }
+            }
+            return self;
+        };
+        /*
+         * Clears either all listeners, or listeners for some specific event.
+         * 
+         * Parameters:
+         * 	event	-	optional. constrains the clear to just listeners for this event.
+         */
+        this.unbind = function(event) {
+            if (event)
+                delete _listeners[event];
+            else {
+                _listeners = {};
+            }
+            return self;
+        };
+
+        this.getListener = function(forEvent) {
+            return _listeners[forEvent];
+        };
+    },
+    logEnabled : true,
+    log : function() {
+        if (jsPlumbUtil.logEnabled && typeof console != "undefined") {
+            try {
+                var msg = arguments[arguments.length - 1];
+                console.log(msg);
+            }
+            catch (e) {}
+        }
+    },
+    group : function(g) { if (jsPlumbUtil.logEnabled && typeof console != "undefined") console.group(g); },
+    groupEnd : function(g) { if (jsPlumbUtil.logEnabled && typeof console != "undefined") console.groupEnd(g); },
+    time : function(t) { if (jsPlumbUtil.logEnabled && typeof console != "undefined") console.time(t); },
+    timeEnd : function(t) { if (jsPlumbUtil.logEnabled && typeof console != "undefined") console.timeEnd(t); }
+};/*
+ * jsPlumb
+ * 
+ * Title:jsPlumb 1.3.16
+ * 
+ * Provides a way to visually connect elements on an HTML page, using either SVG, Canvas
+ * elements, or VML.  
+ * 
+ * This file contains the base functionality for DOM type adapters. 
+ *
+ * Copyright (c) 2010 - 2012 Simon Porritt (http://jsplumb.org)
+ * 
+ * http://jsplumb.org
+ * http://github.com/sporritt/jsplumb
+ * http://code.google.com/p/jsplumb
+ * 
+ * Dual licensed under the MIT and GPL2 licenses.
+ */
+;(function() {
+
+    var canvasAvailable = !!document.createElement('canvas').getContext,
+        svgAvailable = !!window.SVGAngle || document.implementation.hasFeature("http://www.w3.org/TR/SVG11/feature#BasicStructure", "1.1"),
+    // http://stackoverflow.com/questions/654112/how-do-you-detect-support-for-vml-or-svg-in-a-browser
+        vmlAvailable = function() {
+            if (vmlAvailable.vml == undefined) {
+                var a = document.body.appendChild(document.createElement('div'));
+                a.innerHTML = '<v:shape id="vml_flag1" adj="1" />';
+                var b = a.firstChild;
+                b.style.behavior = "url(#default#VML)";
+                vmlAvailable.vml = b ? typeof b.adj == "object": true;
+                a.parentNode.removeChild(a);
+            }
+            return vmlAvailable.vml;
+        };
+
+    /**
+     Manages dragging for some instance of jsPlumb.
+     */
+    var DragManager = function(_currentInstance) {
+        var _draggables = {}, _dlist = [], _delements = {}, _elementsWithEndpoints = {};
+
+        /**
+         register some element as draggable.  right now the drag init stuff is done elsewhere, and it is
+         possible that will continue to be the case.
+         */
+        this.register = function(el) {
+            var jpcl = jsPlumb.CurrentLibrary;
+            el = jpcl.getElementObject(el);
+            var id = _currentInstance.getId(el),
+                domEl = jpcl.getDOMElement(el),
+                parentOffset = jpcl.getOffset(el);
+
+            if (!_draggables[id]) {
+                _draggables[id] = el;
+                _dlist.push(el);
+                _delements[id] = {};
+            }
+
+            // look for child elements that have endpoints and register them against this draggable.
+            var _oneLevel = function(p, startOffset) {
+                if (p) {
+                    for (var i = 0; i < p.childNodes.length; i++) {
+                        if (p.childNodes[i].nodeType != 3) {
+                            var cEl = jpcl.getElementObject(p.childNodes[i]),
+                                cid = _currentInstance.getId(cEl, null, true);
+                            if (cid && _elementsWithEndpoints[cid] && _elementsWithEndpoints[cid] > 0) {
+                                var cOff = jpcl.getOffset(cEl);
+                                _delements[id][cid] = {
+                                    id:cid,
+                                    offset:{
+                                        left:cOff.left - parentOffset.left,
+                                        top:cOff.top - parentOffset.top
+                                    }
+                                };
+                            }
+                            _oneLevel(p.childNodes[i]);
+                        }
+                    }
+                }
+            };
+
+            _oneLevel(domEl);
+        };
+
+        // refresh the offsets for child elements of this element. 
+        this.updateOffsets = function(elId) {
+            var jpcl = jsPlumb.CurrentLibrary,
+                el = jpcl.getElementObject(elId),
+                id = _currentInstance.getId(el),
+                children = _delements[id],
+                parentOffset = jpcl.getOffset(el);
+
+            if (children) {
+                for (var i in children) {
+                    var cel = jpcl.getElementObject(i),
+                        cOff = jpcl.getOffset(cel);
+
+                    _delements[id][i] = {
+                        id:i,
+                        offset:{
+                            left:cOff.left - parentOffset.left,
+                            top:cOff.top - parentOffset.top
+                        }
+                    };
+                }
+            }
+        };
+
+        /**
+         notification that an endpoint was added to the given el.  we go up from that el's parent
+         node, looking for a parent that has been registered as a draggable. if we find one, we add this
+         el to that parent's list of elements to update on drag (if it is not there already)
+         */
+        this.endpointAdded = function(el) {
+            var jpcl = jsPlumb.CurrentLibrary, b = document.body, id = _currentInstance.getId(el), c = jpcl.getDOMElement(el),
+                p = c.parentNode, done = p == b;
+
+            _elementsWithEndpoints[id] = _elementsWithEndpoints[id] ? _elementsWithEndpoints[id] + 1 : 1;
+
+            while (p != b) {
+                var pid = _currentInstance.getId(p, null, true);
+                if (pid && _draggables[pid]) {
+                    var idx = -1, pEl = jpcl.getElementObject(p), pLoc = jpcl.getOffset(pEl);
+
+                    if (_delements[pid][id] == null) {
+                        var cLoc = jsPlumb.CurrentLibrary.getOffset(el);
+                        _delements[pid][id] = {
+                            id:id,
+                            offset:{
+                                left:cLoc.left - pLoc.left,
+                                top:cLoc.top - pLoc.top
+                            }
+                        };
+                    }
+                    break;
+                }
+                p = p.parentNode;
+            }
+        };
+
+        this.endpointDeleted = function(endpoint) {
+            if (_elementsWithEndpoints[endpoint.elementId]) {
+                _elementsWithEndpoints[endpoint.elementId]--;
+                if (_elementsWithEndpoints[endpoint.elementId] <= 0) {
+                    for (var i in _delements) {
+                        delete _delements[i][endpoint.elementId];
+                    }
+                }
+            }
+        };
+
+        this.getElementsForDraggable = function(id) {
+            return _delements[id];
+        };
+
+        this.reset = function() {
+            _draggables = {};
+            _dlist = [];
+            _delements = {};
+            _elementsWithEndpoints = {};
+        };
+
+    };
+
+    // for those browsers that dont have it.  they still don't have it! but at least they won't crash.
+    if (!window.console)
+        window.console = { time:function(){}, timeEnd:function(){}, group:function(){}, groupEnd:function(){}, log:function(){} };
+
+    window.jsPlumbAdapter = {
+
+        headless:false,
+
+        appendToRoot : function(node) {
+            document.body.appendChild(node);
+        },
+        getRenderModes : function() {
+            return [ "canvas", "svg", "vml" ]
+        },
+        isRenderModeAvailable : function(m) {
+            return {
+                "canvas":canvasAvailable,
+                "svg":svgAvailable,
+                "vml":vmlAvailable()
+            }[m];
+        },
+        getDragManager : function(_jsPlumb) {
+            return new DragManager(_jsPlumb);
+        },
+        setRenderMode : function(mode) {
+            var renderMode;
+
+            if (mode) {
+                mode = mode.toLowerCase();
+
+                var canvasAvailable = this.isRenderModeAvailable("canvas"),
+                    svgAvailable = this.isRenderModeAvailable("svg"),
+                    vmlAvailable = this.isRenderModeAvailable("vml");
+
+                //if (mode !== jsPlumb.CANVAS && mode !== jsPlumb.SVG && mode !== jsPlumb.VML) throw new Error("render mode must be one of jsPlumb.CANVAS, jsPlumb.SVG or jsPlumb.VML");
+                // now test we actually have the capability to do this.						
+                if (mode === "svg") {
+                    if (svgAvailable) renderMode = "svg"
+                    else if (canvasAvailable) renderMode = "canvas"
+                    else if (vmlAvailable) renderMode = "vml"
+                }
+                else if (mode === "canvas" && canvasAvailable) renderMode = "canvas";
+                else if (vmlAvailable) renderMode = "vml";
+            }
+
+            return renderMode;
+        }
+    };
+
+})();/*
+ * jsPlumb
+ * 
+ * Title:jsPlumb 1.3.16
+ * 
+ * Provides a way to visually connect elements on an HTML page, using either SVG, Canvas
+ * elements, or VML.  
+ * 
+ * This file contains the jsPlumb core code.
+ *
+ * Copyright (c) 2010 - 2012 Simon Porritt (simon.porritt@gmail.com)
+ * 
+ * http://jsplumb.org
+ * http://github.com/sporritt/jsplumb
+ * http://code.google.com/p/jsplumb
+ * 
+ * Dual licensed under the MIT and GPL2 licenses.
+ */
+
+;(function() {
+
+    /**
+     * Class:jsPlumb
+     * The jsPlumb engine, registered as a static object in the window.  This object contains all of the methods you will use to
+     * create and maintain Connections and Endpoints.
+     */
+
+    var _findWithFunction = jsPlumbUtil.findWithFunction,
+        _indexOf = jsPlumbUtil.indexOf,
+        _removeWithFunction = jsPlumbUtil.removeWithFunction,
+        _remove = jsPlumbUtil.remove,
+    // TODO support insert index
+        _addWithFunction = jsPlumbUtil.addWithFunction,
+        _addToList = jsPlumbUtil.addToList,
+        /**
+         an isArray function that even works across iframes...see here:
+
+         http://tobyho.com/2011/01/28/checking-types-in-javascript/
+
+         i was originally using "a.constructor == Array" as a test.
+         */
+            _isArray = jsPlumbUtil.isArray,
+        _isString = jsPlumbUtil.isString,
+        _isObject = jsPlumbUtil.isObject;
+
+    var _connectionBeingDragged = null,
+        _getAttribute = function(el, attName) { return jsPlumb.CurrentLibrary.getAttribute(_getElementObject(el), attName); },
+        _setAttribute = function(el, attName, attValue) { jsPlumb.CurrentLibrary.setAttribute(_getElementObject(el), attName, attValue); },
+        _addClass = function(el, clazz) { jsPlumb.CurrentLibrary.addClass(_getElementObject(el), clazz); },
+        _hasClass = function(el, clazz) { return jsPlumb.CurrentLibrary.hasClass(_getElementObject(el), clazz); },
+        _removeClass = function(el, clazz) { jsPlumb.CurrentLibrary.removeClass(_getElementObject(el), clazz); },
+        _getElementObject = function(el) { return jsPlumb.CurrentLibrary.getElementObject(el); },
+        _getOffset = function(el, _instance) {
+            var o = jsPlumb.CurrentLibrary.getOffset(_getElementObject(el));
+            if (_instance != null) {
+                var z = _instance.getZoom();
+                return {left:o.left / z, top:o.top / z };
+            }
+            else
+                return o;
+        },
+        _getSize = function(el) {
+            return jsPlumb.CurrentLibrary.getSize(_getElementObject(el));
+        },
+        _log = jsPlumbUtil.log,
+        _group = jsPlumbUtil.group,
+        _groupEnd = jsPlumbUtil.groupEnd,
+        _time = jsPlumbUtil.time,
+        _timeEnd = jsPlumbUtil.timeEnd,
+
+        /**
+         * creates a timestamp, using milliseconds since 1970, but as a string.
+         */
+            _timestamp = function() { return "" + (new Date()).getTime(); },
+
+    /*
+     * Class:jsPlumbUIComponent
+     * Abstract superclass for UI components Endpoint and Connection.  Provides the abstraction of paintStyle/hoverPaintStyle,
+     * and also extends jsPlumbUtil.EventGenerator to provide the bind and fire methods.
+     */
+        jsPlumbUIComponent = function(params) {
+            var self = this,
+                a = arguments,
+                _hover = false,
+                parameters = params.parameters || {},
+                idPrefix = self.idPrefix,
+                id = idPrefix + (new Date()).getTime(),
+                paintStyle = null,
+                hoverPaintStyle = null;
+
+            self._jsPlumb = params["_jsPlumb"];
+            self.getId = function() { return id; };
+            self.tooltip = params.tooltip;
+            self.hoverClass = params.hoverClass || self._jsPlumb.Defaults.HoverClass || jsPlumb.Defaults.HoverClass;
+
+            // all components can generate events
+            jsPlumbUtil.EventGenerator.apply(this);
+            // all components get this clone function.
+            // TODO issue 116 showed a problem with this - it seems 'a' that is in
+            // the clone function's scope is shared by all invocations of it, the classic
+            // JS closure problem.  for now, jsPlumb does a version of this inline where 
+            // it used to call clone.  but it would be nice to find some time to look
+            // further at this.
+            this.clone = function() {
+                var o = new Object();
+                self.constructor.apply(o, a);
+                return o;
+            };
+
+            this.getParameter = function(name) { return parameters[name]; },
+                this.getParameters = function() {
+                    return parameters;
+                },
+                this.setParameter = function(name, value) { parameters[name] = value; },
+                this.setParameters = function(p) { parameters = p; },
+                this.overlayPlacements = [];
+
+            // user can supply a beforeDetach callback, which will be executed before a detach
+            // is performed; returning false prevents the detach.
+            var beforeDetach = params.beforeDetach;
+            this.isDetachAllowed = function(connection) {
+                var r = self._jsPlumb.checkCondition("beforeDetach", connection );
+                if (beforeDetach) {
+                    try {
+                        r = beforeDetach(connection);
+                    }
+                    catch (e) { _log("jsPlumb: beforeDetach callback failed", e); }
+                }
+                return r;
+            };
+
+            // user can supply a beforeDrop callback, which will be executed before a dropped
+            // connection is confirmed. user can return false to reject connection.
+            var beforeDrop = params.beforeDrop;
+            this.isDropAllowed = function(sourceId, targetId, scope, connection, dropEndpoint) {
+                var r = self._jsPlumb.checkCondition("beforeDrop", {
+                    sourceId:sourceId,
+                    targetId:targetId,
+                    scope:scope,
+                    connection:connection,
+                    dropEndpoint:dropEndpoint
+                });
+                if (beforeDrop) {
+                    try {
+                        r = beforeDrop({
+                            sourceId:sourceId,
+                            targetId:targetId,
+                            scope:scope,
+                            connection:connection,
+                            dropEndpoint:dropEndpoint
+                        });
+                    }
+                    catch (e) { _log("jsPlumb: beforeDrop callback failed", e); }
+                }
+                return r;
+            };
+
+            // helper method to update the hover style whenever it, or paintStyle, changes.
+            // we use paintStyle as the foundation and merge hoverPaintStyle over the
+            // top.
+            var _updateHoverStyle = function() {
+                if (paintStyle && hoverPaintStyle) {
+                    var mergedHoverStyle = {};
+                    jsPlumb.extend(mergedHoverStyle, paintStyle);
+                    jsPlumb.extend(mergedHoverStyle, hoverPaintStyle);
+                    delete self["hoverPaintStyle"];
+                    // we want the fillStyle of paintStyle to override a gradient, if possible.
+                    if (mergedHoverStyle.gradient && paintStyle.fillStyle)
+                        delete mergedHoverStyle["gradient"];
+                    hoverPaintStyle = mergedHoverStyle;
+                }
+            };
+
+            /*
+             * Sets the paint style and then repaints the element.
+             * 
+             * Parameters:
+             * 	style - Style to use.
+             */
+            this.setPaintStyle = function(style, doNotRepaint) {
+                paintStyle = style;
+                self.paintStyleInUse = paintStyle;
+                _updateHoverStyle();
+                if (!doNotRepaint) self.repaint();
+            };
+
+            /**
+             * Gets the component's paint style.
+             *
+             * Returns:
+             * the component's paint style. if there is no hoverPaintStyle set then this will be the paint style used all the time, otherwise this is the style used when the mouse is not hovering.
+             */
+            this.getPaintStyle = function() {
+                return paintStyle;
+            };
+
+            /*
+             * Sets the paint style to use when the mouse is hovering over the element. This is null by default.
+             * The hover paint style is applied as extensions to the paintStyle; it does not entirely replace
+             * it.  This is because people will most likely want to change just one thing when hovering, say the
+             * color for example, but leave the rest of the appearance the same.
+             * 
+             * Parameters:
+             * 	style - Style to use when the mouse is hovering.
+             *  doNotRepaint - if true, the component will not be repainted.  useful when setting things up initially.
+             */
+            this.setHoverPaintStyle = function(style, doNotRepaint) {
+                hoverPaintStyle = style;
+                _updateHoverStyle();
+                if (!doNotRepaint) self.repaint();
+            };
+
+            /**
+             * Gets the component's hover paint style.
+             *
+             * Returns:
+             * the component's hover paint style. may be null.
+             */
+            this.getHoverPaintStyle = function() {
+                return hoverPaintStyle;
+            };
+
+            /*
+             * sets/unsets the hover state of this element.
+             * 
+             * Parameters:
+             * 	hover - hover state boolean
+             * 	ignoreAttachedElements - if true, does not notify any attached elements of the change in hover state.  used mostly to avoid infinite loops.
+             */
+            this.setHover = function(hover, ignoreAttachedElements, timestamp) {
+                // while dragging, we ignore these events.  this keeps the UI from flashing and
+                // swishing and whatevering.
+                if (!self._jsPlumb.currentlyDragging && !self._jsPlumb.isHoverSuspended()) {
+
+                    _hover = hover;
+                    if (self.hoverClass != null && self.canvas != null) {
+                        if (hover)
+                            jpcl.addClass(self.canvas, self.hoverClass);
+                        else
+                            jpcl.removeClass(self.canvas, self.hoverClass);
+                    }
+                    if (hoverPaintStyle != null) {
+                        self.paintStyleInUse = hover ? hoverPaintStyle : paintStyle;
+                        timestamp = timestamp || _timestamp();
+                        self.repaint({timestamp:timestamp, recalc:false});
+                    }
+                    // get the list of other affected elements, if supported by this component.
+                    // for a connection, its the endpoints.  for an endpoint, its the connections! surprise.
+                    if (self.getAttachedElements && !ignoreAttachedElements)
+                        _updateAttachedElements(hover, _timestamp(), self);
+                }
+            };
+
+            this.isHover = function() { return _hover; };
+
+            var zIndex = null;
+            this.setZIndex = function(v) { zIndex = v; };
+            this.getZIndex = function() { return zIndex; };
+
+            var jpcl = jsPlumb.CurrentLibrary,
+                events = [ "click", "dblclick", "mouseenter", "mouseout", "mousemove", "mousedown", "mouseup", "contextmenu" ],
+                eventFilters = { "mouseout":"mouseexit" },
+                bindOne = function(o, c, evt) {
+                    var filteredEvent = eventFilters[evt] || evt;
+                    jpcl.bind(o, evt, function(ee) {
+                        c.fire(filteredEvent, c, ee);
+                    });
+                },
+                unbindOne = function(o, evt) {
+                    var filteredEvent = eventFilters[evt] || evt;
+                    jpcl.unbind(o, evt);
+                };
+
+            this.attachListeners = function(o, c) {
+                for (var i = 0; i < events.length; i++) {
+                    bindOne(o, c, events[i]);
+                }
+            };
+
+            var _updateAttachedElements = function(state, timestamp, sourceElement) {
+                var affectedElements = self.getAttachedElements();		// implemented in subclasses
+                if (affectedElements) {
+                    for (var i = 0; i < affectedElements.length; i++) {
+                        if (!sourceElement || sourceElement != affectedElements[i])
+                            affectedElements[i].setHover(state, true, timestamp);			// tell the attached elements not to inform their own attached elements.
+                    }
+                }
+            };
+
+            this.reattachListenersForElement = function(o) {
+                if (arguments.length > 1) {
+                    for (var i = 0; i < events.length; i++)
+                        unbindOne(o, events[i]);
+                    for (var i = 1; i < arguments.length; i++)
+                        self.attachListeners(o, arguments[i]);
+                }
+            };
+
+            /*
+             * TYPES
+             */
+            var _types = [],
+                _splitType = function(t) { return t == null ? null : t.split(" ")},
+                _applyTypes = function(doNotRepaint) {
+                    if (self.getDefaultType) {
+                        var td = self.getTypeDescriptor();
+
+                        var o = jsPlumbUtil.merge({}, self.getDefaultType());
+                        for (var i = 0; i < _types.length; i++)
+                            o = jsPlumbUtil.merge(o, self._jsPlumb.getType(_types[i], td));
+
+                        self.applyType(o);
+                        if (!doNotRepaint) self.repaint();
+                    }
+                };
+
+            self.setType = function(typeId, doNotRepaint) {
+                _types = _splitType(typeId) || [];
+                _applyTypes(doNotRepaint);
+            };
+
+            /*
+             * Function : getType
+             * Gets the 'types' of this component.
+             */
+            self.getType = function() {
+                return _types;
+            };
+
+            self.hasType = function(typeId) {
+                return jsPlumbUtil.indexOf(_types, typeId) != -1;
+            };
+
+            self.addType = function(typeId, doNotRepaint) {
+                var t = _splitType(typeId), _cont = false;
+                if (t != null) {
+                    for (var i = 0; i < t.length; i++) {
+                        if (!self.hasType(t[i])) {
+                            _types.push(t[i]);
+                            _cont = true;
+                        }
+                    }
+                    if (_cont) _applyTypes(doNotRepaint);
+                }
+            };
+
+            self.removeType = function(typeId, doNotRepaint) {
+                var t = _splitType(typeId), _cont = false, _one = function(tt) {
+                    var idx = jsPlumbUtil.indexOf(_types, tt);
+                    if (idx != -1) {
+                        _types.splice(idx, 1);
+                        return true;
+                    }
+                    return false;
+                };
+
+                if (t != null) {
+                    for (var i = 0; i < t.length; i++) {
+                        _cont = _one(t[i]) || _cont;
+                    }
+                    if (_cont) _applyTypes(doNotRepaint);
+                }
+            };
+
+            self.toggleType = function(typeId, doNotRepaint) {
+                var t = _splitType(typeId);
+                if (t != null) {
+                    for (var i = 0; i < t.length; i++) {
+                        var idx = jsPlumbUtil.indexOf(_types, t[i]);
+                        if (idx != -1)
+                            _types.splice(idx, 1);
+                        else
+                            _types.push(t[i]);
+                    }
+
+                    _applyTypes(doNotRepaint);
+                }
+            };
+
+            this.applyType = function(t) {
+                self.setPaintStyle(t.paintStyle);
+                self.setHoverPaintStyle(t.hoverPaintStyle);
+                if (t.parameters){
+                    for (var i in t.parameters)
+                        self.setParameter(i, t.parameters[i]);
+                }
+            };
+
+        },
+
+        overlayCapableJsPlumbUIComponent = function(params) {
+            jsPlumbUIComponent.apply(this, arguments);
+            var self = this;
+            this.overlays = [];
+
+            var processOverlay = function(o) {
+                    var _newOverlay = null;
+                    if (_isArray(o)) {	// this is for the shorthand ["Arrow", { width:50 }] syntax
+                        // there's also a three arg version:
+                        // ["Arrow", { width:50 }, {location:0.7}] 
+                        // which merges the 3rd arg into the 2nd.
+                        var type = o[0],
+                        // make a copy of the object so as not to mess up anyone else's reference...
+                            p = jsPlumb.extend({component:self, _jsPlumb:self._jsPlumb}, o[1]);
+                        if (o.length == 3) jsPlumb.extend(p, o[2]);
+                        _newOverlay = new jsPlumb.Overlays[self._jsPlumb.getRenderMode()][type](p);
+                        if (p.events) {
+                            for (var evt in p.events) {
+                                _newOverlay.bind(evt, p.events[evt]);
+                            }
+                        }
+                    } else if (o.constructor == String) {
+                        _newOverlay = new jsPlumb.Overlays[self._jsPlumb.getRenderMode()][o]({component:self, _jsPlumb:self._jsPlumb});
+                    } else {
+                        _newOverlay = o;
+                    }
+
+                    self.overlays.push(_newOverlay);
+                },
+                calculateOverlaysToAdd = function(params) {
+                    var defaultKeys = self.defaultOverlayKeys || [],
+                        o = params.overlays,
+                        checkKey = function(k) {
+                            return self._jsPlumb.Defaults[k] || jsPlumb.Defaults[k] || [];
+                        };
+
+                    if (!o) o = [];
+
+                    for (var i = 0; i < defaultKeys.length; i++)
+                        o.unshift.apply(o, checkKey(defaultKeys[i]));
+
+                    return o;
+                }
+
+            var _overlays = calculateOverlaysToAdd(params);//params.overlays || self._jsPlumb.Defaults.Overlays;
+            if (_overlays) {
+                for (var i = 0; i < _overlays.length; i++) {
+                    processOverlay(_overlays[i]);
+                }
+            }
+
+            // overlay finder helper method
+            var _getOverlayIndex = function(id) {
+                var idx = -1;
+                for (var i = 0; i < self.overlays.length; i++) {
+                    if (id === self.overlays[i].id) {
+                        idx = i;
+                        break;
+                    }
+                }
+                return idx;
+            };
+
+            this.addOverlay = function(overlay, doNotRepaint) {
+                processOverlay(overlay);
+                if (!doNotRepaint) self.repaint();
+            };
+
+            this.getOverlay = function(id) {
+                var idx = _getOverlayIndex(id);
+                return idx >= 0 ? self.overlays[idx] : null;
+            };
+
+            this.getOverlays = function() {
+                return self.overlays;
+            };
+
+            this.hideOverlay = function(id) {
+                var o = self.getOverlay(id);
+                if (o) o.hide();
+            };
+
+            this.hideOverlays = function() {
+                for (var i = 0; i < self.overlays.length; i++)
+                    self.overlays[i].hide();
+            };
+
+            this.showOverlay = function(id) {
+                var o = self.getOverlay(id);
+                if (o) o.show();
+            };
+
+            this.showOverlays = function() {
+                for (var i = 0; i < self.overlays.length; i++)
+                    self.overlays[i].show();
+            };
+
+            this.removeAllOverlays = function() {
+                for (var i = 0; i < self.overlays.length; i++) {
+                    if (self.overlays[i].cleanup) self.overlays[i].cleanup();
+                }
+
+                self.overlays.splice(0, self.overlays.length);
+                self.repaint();
+            };
+
+            this.removeOverlay = function(overlayId) {
+                var idx = _getOverlayIndex(overlayId);
+                if (idx != -1) {
+                    var o = self.overlays[idx];
+                    if (o.cleanup) o.cleanup();
+                    self.overlays.splice(idx, 1);
+                }
+            };
+
+            this.removeOverlays = function() {
+                for (var i = 0; i < arguments.length; i++)
+                    self.removeOverlay(arguments[i]);
+            };
+
+            // this is a shortcut helper method to let people add a label as
+            // overlay.			
+            var _internalLabelOverlayId = "__label",
+                _makeLabelOverlay = function(params) {
+
+                    var _params = {
+                            cssClass:params.cssClass,
+                            labelStyle : this.labelStyle,
+                            id:_internalLabelOverlayId,
+                            component:self,
+                            _jsPlumb:self._jsPlumb
+                        },
+                        mergedParams = jsPlumb.extend(_params, params);
+
+                    return new jsPlumb.Overlays[self._jsPlumb.getRenderMode()].Label( mergedParams );
+                };
+            if (params.label) {
+                var loc = params.labelLocation || self.defaultLabelLocation || 0.5,
+                    labelStyle = params.labelStyle || self._jsPlumb.Defaults.LabelStyle || jsPlumb.Defaults.LabelStyle;
+                this.overlays.push(_makeLabelOverlay({
+                    label:params.label,
+                    location:loc,
+                    labelStyle:labelStyle
+                }));
+            }
+
+
+            this.setLabel = function(l) {
+                var lo = self.getOverlay(_internalLabelOverlayId);
+                if (!lo) {
+                    var params = l.constructor == String || l.constructor == Function ? { label:l } : l;
+                    lo = _makeLabelOverlay(params);
+                    this.overlays.push(lo);
+                }
+                else {
+                    if (l.constructor == String || l.constructor == Function) lo.setLabel(l);
+                    else {
+                        if (l.label) lo.setLabel(l.label);
+                        if (l.location) lo.setLocation(l.location);
+                    }
+                }
+
+                if (!self._jsPlumb.isSuspendDrawing())
+                    self.repaint();
+            };
+
+
+            this.getLabel = function() {
+                var lo = self.getOverlay(_internalLabelOverlayId);
+                return lo != null ? lo.getLabel() : null;
+            };
+
+
+            this.getLabelOverlay = function() {
+                return self.getOverlay(_internalLabelOverlayId);
+            };
+
+            var superAt = this.applyType;
+            this.applyType = function(t) {
+                superAt(t);
+                self.removeAllOverlays();
+                if (t.overlays) {
+                    for (var i = 0; i < t.overlays.length; i++)
+                        self.addOverlay(t.overlays[i], true);
+                }
+            };
+        },
+
+        _bindListeners = function(obj, _self, _hoverFunction) {
+            obj.bind("click", function(ep, e) { _self.fire("click", _self, e); });
+            obj.bind("dblclick", function(ep, e) { _self.fire("dblclick", _self, e); });
+            obj.bind("contextmenu", function(ep, e) { _self.fire("contextmenu", _self, e); });
+            obj.bind("mouseenter", function(ep, e) {
+                if (!_self.isHover()) {
+                    _hoverFunction(true);
+                    _self.fire("mouseenter", _self, e);
+                }
+            });
+            obj.bind("mouseexit", function(ep, e) {
+                if (_self.isHover()) {
+                    _hoverFunction(false);
+                    _self.fire("mouseexit", _self, e);
+                }
+            });
+        };
+
+    var _jsPlumbInstanceIndex = 0,
+        getInstanceIndex = function() {
+            var i = _jsPlumbInstanceIndex + 1;
+            _jsPlumbInstanceIndex++;
+            return i;
+        };
+
+    var jsPlumbInstance = function(_defaults) {
+
+        /*
+         * Property: Defaults 
+         * 
+         * These are the default settings for jsPlumb.  They are what will be used if you do not supply specific pieces of information 
+         * to the various API calls. A convenient way to implement your own look and feel can be to override these defaults 
+         * by including a script somewhere after the jsPlumb include, but before you make any calls to jsPlumb.
+         * 
+         * Properties:
+         * 	-	*Anchor*				    The default anchor to use for all connections (both source and target). Default is "BottomCenter".
+         * 	-	*Anchors*				    The default anchors to use ([source, target]) for all connections. Defaults are ["BottomCenter", "BottomCenter"].
+         *  -   *ConnectionsDetachable*		Whether or not connections are detachable by default (using the mouse). Defaults to true.
+         *  -   *ConnectionOverlays*		The default overlay definitions for Connections. Defaults to an empty list.
+         * 	-	*Connector*				The default connector definition to use for all connections.  Default is "Bezier".
+         * 	-   *ConnectorZIndex*       Optional value for the z-index of Connections that are not in the hover state. If you set this, jsPlumb will set the z-index of all created Connections to be this value, and the z-index of any Connections in the hover state to be this value plus one. This brings hovered connections up on top of others, which is a nice effect in busy UIs.
+         *  -   *Container*				Optional selector or element id that instructs jsPlumb to append elements it creates to a specific element.
+         * 	-	*DragOptions*			The default drag options to pass in to connect, makeTarget and addEndpoint calls. Default is empty.
+         * 	-	*DropOptions*			The default drop options to pass in to connect, makeTarget and addEndpoint calls. Default is empty.
+         * 	-	*Endpoint*				The default endpoint definition to use for all connections (both source and target).  Default is "Dot".
+         *  -   *EndpointOverlays*		The default overlay definitions for Endpoints. Defaults to an empty list.
+         * 	-	*Endpoints*				The default endpoint definitions ([ source, target ]) to use for all connections.  Defaults are ["Dot", "Dot"].
+         * 	-	*EndpointStyle*			The default style definition to use for all endpoints. Default is fillStyle:"#456".
+         * 	-	*EndpointStyles*		The default style definitions ([ source, target ]) to use for all endpoints.  Defaults are empty.
+         * 	-	*EndpointHoverStyle*	The default hover style definition to use for all endpoints. Default is null.
+         * 	-	*EndpointHoverStyles*	The default hover style definitions ([ source, target ]) to use for all endpoints. Defaults are null.
+         * 	-	*HoverPaintStyle*		The default hover style definition to use for all connections. Defaults are null.
+         * 	-	*LabelStyle*			The default style to use for label overlays on connections.
+         * 	-	*LogEnabled*			Whether or not the jsPlumb log is enabled. defaults to false.
+         * 	-	*Overlays*				The default overlay definitions (for both Connections and Endpoint). Defaults to an empty list.
+         * 	-	*MaxConnections*		The default maximum number of connections for an Endpoint.  Defaults to 1.		 
+         * 	-	*PaintStyle*			The default paint style for a connection. Default is line width of 8 pixels, with color "#456".
+         * 	-	*ReattachConnections*	Whether or not to reattach Connections that a user has detached with the mouse and then dropped. Default is false.
+         * 	-	*RenderMode*			What mode to use to paint with.  If you're on IE<9, you don't really get to choose this.  You'll just get VML.  Otherwise, the jsPlumb default is to use SVG.
+         * 	-	*Scope*				The default "scope" to use for connections. Scope lets you assign connections to different categories. 
+         */
+        this.Defaults = {
+            Anchor : "BottomCenter",
+            Anchors : [ null, null ],
+            ConnectionsDetachable : true,
+            ConnectionOverlays : [ ],
+            Connector : "Bezier",
+            ConnectorZIndex : null,
+            Container : null,
+            DragOptions : { },
+            DropOptions : { },
+            Endpoint : "Dot",
+            EndpointOverlays : [ ],
+            Endpoints : [ null, null ],
+            EndpointStyle : { fillStyle : "#456" },
+            EndpointStyles : [ null, null ],
+            EndpointHoverStyle : null,
+            EndpointHoverStyles : [ null, null ],
+            HoverPaintStyle : null,
+            LabelStyle : { color : "black" },
+            LogEnabled : false,
+            Overlays : [ ],
+            MaxConnections : 1,
+            PaintStyle : { lineWidth : 8, strokeStyle : "#456" },
+            ReattachConnections:false,
+            RenderMode : "svg",
+            Scope : "jsPlumb_DefaultScope"
+        };
+        if (_defaults) jsPlumb.extend(this.Defaults, _defaults);
+
+        this.logEnabled = this.Defaults.LogEnabled;
+
+        var _connectionTypes = { }, _endpointTypes = {};
+        this.registerConnectionType = function(id, type) {
+            _connectionTypes[id] = jsPlumb.extend({}, type);
+        };
+        this.registerConnectionTypes = function(types) {
+            for (var i in types)
+                _connectionTypes[i] = jsPlumb.extend({}, types[i]);
+        };
+        this.registerEndpointType = function(id, type) {
+            _endpointTypes[id] = jsPlumb.extend({}, type);
+        };
+        this.registerEndpointTypes = function(types) {
+            for (var i in types)
+                _endpointTypes[i] = jsPlumb.extend({}, types[i]);
+        };
+        this.getType = function(id, typeDescriptor) {
+            return typeDescriptor ===  "connection" ? _connectionTypes[id] : _endpointTypes[id];
+        };
+
+        jsPlumbUtil.EventGenerator.apply(this);
+        var _currentInstance = this,
+            _instanceIndex = getInstanceIndex(),
+            _bb = _currentInstance.bind,
+            _initialDefaults = {},
+            _zoom = 1;
+
+        this.setZoom = function(z, repaintEverything) {
+            _zoom = z;
+            if (repaintEverything) _currentInstance.repaintEverything();
+        };
+        this.getZoom = function() { return _zoom; };
+
+        for (var i in this.Defaults)
+            _initialDefaults[i] = this.Defaults[i];
+
+        this.bind = function(event, fn) {
+            if ("ready" === event && initialized) fn();
+            else _bb.apply(_currentInstance,[event, fn]);
+        };
+
+        _currentInstance.importDefaults = function(d) {
+            for (var i in d) {
+                _currentInstance.Defaults[i] = d[i];
+            }
+        };
+
+        _currentInstance.restoreDefaults = function() {
+            _currentInstance.Defaults = jsPlumb.extend({}, _initialDefaults);
+        };
+
+        var log = null,
+            resizeTimer = null,
+            initialized = false,
+            connectionsByScope = {},
+            /**
+             * map of element id -> endpoint lists. an element can have an arbitrary
+             * number of endpoints on it, and not all of them have to be connected
+             * to anything.
+             */
+                endpointsByElement = {},
+            endpointsByUUID = {},
+            offsets = {},
+            offsetTimestamps = {},
+            floatingConnections = {},
+            draggableStates = {},
+            canvasList = [],
+            sizes = [],
+        //listeners = {}, // a map: keys are event types, values are lists of listeners.
+            DEFAULT_SCOPE = this.Defaults.Scope,
+            renderMode = null,  // will be set in init()							
+
+            /**
+             * helper method to add an item to a list, creating the list if it does
+             * not yet exist.
+             */
+                _addToList = function(map, key, value) {
+                var l = map[key];
+                if (l == null) {
+                    l = [];
+                    map[key] = l;
+                }
+                l.push(value);
+                return l;
+            },
+
+            /**
+             * appends an element to some other element, which is calculated as follows:
+             *
+             * 1. if _currentInstance.Defaults.Container exists, use that element.
+             * 2. if the 'parent' parameter exists, use that.
+             * 3. otherwise just use the root element (for DOM usage, the document body).
+             *
+             */
+                _appendElement = function(el, parent) {
+                if (_currentInstance.Defaults.Container)
+                    jsPlumb.CurrentLibrary.appendElement(el, _currentInstance.Defaults.Container);
+                else if (!parent)
+                //document.body.appendChild(el);
+                    jsPlumbAdapter.appendToRoot(el);
+                else
+                    jsPlumb.CurrentLibrary.appendElement(el, parent);
+            },
+
+            _curIdStamp = 1,
+            _idstamp = function() { return "" + _curIdStamp++; },
+
+            /**
+             * YUI, for some reason, put the result of a Y.all call into an object that contains
+             * a '_nodes' array, instead of handing back an array-like object like the other
+             * libraries do.
+             */
+                _convertYUICollection = function(c) {
+                return c._nodes ? c._nodes : c;
+            },
+
+            /**
+             * Draws an endpoint and its connections. this is the main entry point into drawing connections as well
+             * as endpoints, since jsPlumb is endpoint-centric under the hood.
+             *
+             * @param element element to draw (of type library specific element object)
+             * @param ui UI object from current library's event system. optional.
+             * @param timestamp timestamp for this paint cycle. used to speed things up a little by cutting down the amount of offset calculations we do.
+             */
+                _draw = function(element, ui, timestamp) {
+
+                // TOD is it correct to filter by headless at this top level? how would a headless adapter ever repaint?
+                if (!jsPlumbAdapter.headless && !_suspendDrawing) {
+                    var id = _getAttribute(element, "id"),
+                        repaintEls = _currentInstance.dragManager.getElementsForDraggable(id);
+
+                    if (timestamp == null) timestamp = _timestamp();
+
+                    _currentInstance.anchorManager.redraw(id, ui, timestamp);
+
+                    if (repaintEls) {
+                        for (var i in repaintEls) {
+                            _currentInstance.anchorManager.redraw(repaintEls[i].id, ui, timestamp, repaintEls[i].offset);
+                        }
+                    }
+                }
+            },
+
+            /**
+             * executes the given function against the given element if the first
+             * argument is an object, or the list of elements, if the first argument
+             * is a list. the function passed in takes (element, elementId) as
+             * arguments.
+             */
+                _elementProxy = function(element, fn) {
+                var retVal = null;
+                if (_isArray(element)) {
+                    retVal = [];
+                    for ( var i = 0; i < element.length; i++) {
+                        var el = _getElementObject(element[i]), id = _getAttribute(el, "id");
+                        retVal.push(fn(el, id)); // append return values to what we will return
+                    }
+                } else {
+                    var el = _getElementObject(element), id = _getAttribute(el, "id");
+                    retVal = fn(el, id);
+                }
+                return retVal;
+            },
+
+            /**
+             * gets an Endpoint by uuid.
+             */
+                _getEndpoint = function(uuid) { return endpointsByUUID[uuid]; },
+
+            /**
+             * inits a draggable if it's not already initialised.
+             */
+                _initDraggableIfNecessary = function(element, isDraggable, dragOptions) {
+                // TODO move to DragManager?
+                if (!jsPlumbAdapter.headless) {
+                    var draggable = isDraggable == null ? false : isDraggable,
+                        jpcl = jsPlumb.CurrentLibrary;
+                    if (draggable) {
+                        if (jpcl.isDragSupported(element) && !jpcl.isAlreadyDraggable(element)) {
+                            var options = dragOptions || _currentInstance.Defaults.DragOptions || jsPlumb.Defaults.DragOptions;
+                            options = jsPlumb.extend( {}, options); // make a copy.
+                            var dragEvent = jpcl.dragEvents["drag"],
+                                stopEvent = jpcl.dragEvents["stop"],
+                                startEvent = jpcl.dragEvents["start"];
+
+                            options[startEvent] = _wrap(options[startEvent], function() {
+                                _currentInstance.setHoverSuspended(true);
+                            });
+
+                            options[dragEvent] = _wrap(options[dragEvent], function() {
+                                var ui = jpcl.getUIPosition(arguments, _currentInstance.getZoom());
+                                _draw(element, ui);
+                                _addClass(element, "jsPlumb_dragged");
+                            });
+                            options[stopEvent] = _wrap(options[stopEvent], function() {
+                                var ui = jpcl.getUIPosition(arguments, _currentInstance.getZoom());
+                                _draw(element, ui);
+                                _removeClass(element, "jsPlumb_dragged");
+                                _currentInstance.setHoverSuspended(false);
+                            });
+                            var elId = _getId(element); // need ID
+                            draggableStates[elId] = true;
+                            var draggable = draggableStates[elId];
+                            options.disabled = draggable == null ? false : !draggable;
+                            jpcl.initDraggable(element, options, false);
+                            _currentInstance.dragManager.register(element);
+                        }
+                    }
+                }
+            },
+
+        /*
+         * prepares a final params object that can be passed to _newConnection, taking into account defaults, events, etc.
+         */
+            _prepareConnectionParams = function(params, referenceParams) {
+                var _p = jsPlumb.extend( {
+                    sourceIsNew:true,
+                    targetIsNew:true
+                }, params);
+                if (referenceParams) jsPlumb.extend(_p, referenceParams);
+
+                // hotwire endpoints passed as source or target to sourceEndpoint/targetEndpoint, respectively.
+                if (_p.source && _p.source.endpoint) _p.sourceEndpoint = _p.source;
+                if (_p.source && _p.target.endpoint) _p.targetEndpoint = _p.target;
+
+                // test for endpoint uuids to connect
+                if (params.uuids) {
+                    _p.sourceEndpoint = _getEndpoint(params.uuids[0]);
+                    _p.targetEndpoint = _getEndpoint(params.uuids[1]);
+                }
+
+                // now ensure that if we do have Endpoints already, they're not full.
+                // source:
+                if (_p.sourceEndpoint && _p.sourceEndpoint.isFull()) {
+                    _log(_currentInstance, "could not add connection; source endpoint is full");
+                    return;
+                }
+
+                // target:
+                if (_p.targetEndpoint && _p.targetEndpoint.isFull()) {
+                    _log(_currentInstance, "could not add connection; target endpoint is full");
+                    return;
+                }
+
+                // at this point, if we have source or target Endpoints, they were not new and we should mark the
+                // flag to reflect that.  this is for use later with the deleteEndpointsOnDetach flag.
+                if (_p.sourceEndpoint) _p.sourceIsNew = false;
+                if (_p.targetEndpoint) _p.targetIsNew = false;
+
+                // if source endpoint mandates connection type and nothing specified in our params, use it.
+                if (!_p.type && _p.sourceEndpoint)
+                    _p.type = _p.sourceEndpoint.connectionType;
+
+                // copy in any connectorOverlays that were specified on the source endpoint.
+                // it doesnt copy target endpoint overlays.  i'm not sure if we want it to or not.
+                if (_p.sourceEndpoint && _p.sourceEndpoint.connectorOverlays) {
+                    _p.overlays = _p.overlays || [];
+                    for (var i = 0; i < _p.sourceEndpoint.connectorOverlays.length; i++) {
+                        _p.overlays.push(_p.sourceEndpoint.connectorOverlays[i]);
+                    }
+                }
+
+                // tooltip.  params.tooltip takes precedence, then sourceEndpoint.connectorTooltip.
+                _p.tooltip = params.tooltip;
+                if (!_p.tooltip && _p.sourceEndpoint && _p.sourceEndpoint.connectorTooltip)
+                    _p.tooltip = _p.sourceEndpoint.connectorTooltip;
+
+                // if there's a target specified (which of course there should be), and there is no
+                // target endpoint specified, and 'newConnection' was not set to true, then we check to
+                // see if a prior call to makeTarget has provided us with the specs for the target endpoint, and
+                // we use those if so.  additionally, if the makeTarget call was specified with 'uniqueEndpoint' set
+                // to true, then if that target endpoint has already been created, we re-use it.
+                if (_p.target && !_p.target.endpoint && !_p.targetEndpoint && !_p.newConnection) {
+                    var tid = _getId(_p.target),
+                        tep =_targetEndpointDefinitions[tid],
+                        existingUniqueEndpoint = _targetEndpoints[tid];
+
+                    if (tep) {
+                        // if target not enabled, return.
+                        if (!_targetsEnabled[tid]) return;
+
+                        // check for max connections??						
+                        var newEndpoint = existingUniqueEndpoint != null ? existingUniqueEndpoint : _currentInstance.addEndpoint(_p.target, tep);
+                        if (_targetEndpointsUnique[tid]) _targetEndpoints[tid] = newEndpoint;
+                        _p.targetEndpoint = newEndpoint;
+                        newEndpoint._makeTargetCreator = true;
+                        _p.targetIsNew = true;
+                    }
+                }
+
+                // same thing, but for source.
+                if (_p.source && !_p.source.endpoint && !_p.sourceEndpoint && !_p.newConnection) {
+                    var tid = _getId(_p.source),
+                        tep = _sourceEndpointDefinitions[tid],
+                        existingUniqueEndpoint = _sourceEndpoints[tid];
+
+                    if (tep) {
+                        // if source not enabled, return.					
+                        if (!_sourcesEnabled[tid]) return;
+
+                        var newEndpoint = existingUniqueEndpoint != null ? existingUniqueEndpoint : _currentInstance.addEndpoint(_p.source, tep);
+                        if (_sourceEndpointsUnique[tid]) _sourceEndpoints[tid] = newEndpoint;
+                        _p.sourceEndpoint = newEndpoint;
+                        _p.sourceIsNew = true;
+                    }
+                }
+
+                return _p;
+            },
+
+            _newConnection = function(params) {
+                var connectionFunc = _currentInstance.Defaults.ConnectionType || _currentInstance.getDefaultConnectionType(),
+                    endpointFunc = _currentInstance.Defaults.EndpointType || Endpoint,
+                    parent = jsPlumb.CurrentLibrary.getParent;
+
+                if (params.container)
+                    params["parent"] = params.container;
+                else {
+                    if (params.sourceEndpoint)
+                        params["parent"] = params.sourceEndpoint.parent;
+                    else if (params.source.constructor == endpointFunc)
+                        params["parent"] = params.source.parent;
+                    else params["parent"] = parent(params.source);
+                }
+
+                params["_jsPlumb"] = _currentInstance;
+                var con = new connectionFunc(params);
+                con.id = "con_" + _idstamp();
+                _eventFireProxy("click", "click", con);
+                _eventFireProxy("dblclick", "dblclick", con);
+                _eventFireProxy("contextmenu", "contextmenu", con);
+                return con;
+            },
+
+            /**
+             * adds the connection to the backing model, fires an event if necessary and then redraws
+             */
+                _finaliseConnection = function(jpc, params, originalEvent) {
+                params = params || {};
+                // add to list of connections (by scope).
+                if (!jpc.suspendedEndpoint)
+                    _addToList(connectionsByScope, jpc.scope, jpc);
+                // fire an event
+                if (!params.doNotFireConnectionEvent && params.fireEvent !== false) {
+
+                    var eventArgs = {
+                        connection:jpc,
+                        source : jpc.source, target : jpc.target,
+                        sourceId : jpc.sourceId, targetId : jpc.targetId,
+                        sourceEndpoint : jpc.endpoints[0], targetEndpoint : jpc.endpoints[1]
+                    };
+
+                    _currentInstance.fire("jsPlumbConnection", eventArgs, originalEvent);
+                    // this is from 1.3.11 onwards. "jsPlumbConnection" always felt so unnecessary, so
+                    // I've added this alias in 1.3.11, with a view to removing "jsPlumbConnection" completely in a future version. be aware, of course, you should only register listeners for one or the other of these events.
+                    _currentInstance.fire("connection", eventArgs, originalEvent);
+                }
+
+                // always inform the anchor manager
+                // except that if jpc has a suspended endpoint it's not true to say the
+                // connection is new; it has just (possibly) moved. the question is whether
+                // to make that call here or in the anchor manager.  i think perhaps here.
+                _currentInstance.anchorManager.newConnection(jpc);
+                // force a paint
+                _draw(jpc.source);
+            },
+
+            _eventFireProxy = function(event, proxyEvent, obj) {
+                obj.bind(event, function(originalObject, originalEvent) {
+                    _currentInstance.fire(proxyEvent, obj, originalEvent);
+                });
+            },
+
+            /**
+             * for the given endpoint params, returns an appropriate parent element for the UI elements that will be added.
+             * this function is used by _newEndpoint (directly below), and also in the makeSource function in jsPlumb.
+             *
+             *   the logic is to first look for a "container" member of params, and pass that back if found.  otherwise we
+             *   handoff to the 'getParent' function in the current library.
+             */
+                _getParentFromParams = function(params) {
+                if (params.container)
+                    return params.container;
+                else {
+                    var tag = jsPlumb.CurrentLibrary.getTagName(params.source),
+                        p = jsPlumb.CurrentLibrary.getParent(params.source);
+                    if (tag && tag.toLowerCase() === "td")
+                        return jsPlumb.CurrentLibrary.getParent(p);
+                    else return p;
+                }
+            },
+
+            /**
+             factory method to prepare a new endpoint.  this should always be used instead of creating Endpoints
+             manually, since this method attaches event listeners and an id.
+             */
+                _newEndpoint = function(params) {
+                var endpointFunc = _currentInstance.Defaults.EndpointType || Endpoint;
+                params.parent = _getParentFromParams(params);
+                params["_jsPlumb"] = _currentInstance;
+                var ep = new endpointFunc(params);
+                ep.id = "ep_" + _idstamp();
+                _eventFireProxy("click", "endpointClick", ep);
+                _eventFireProxy("dblclick", "endpointDblClick", ep);
+                _eventFireProxy("contextmenu", "contextmenu", ep);
+                if (!jsPlumbAdapter.headless)
+                    _currentInstance.dragManager.endpointAdded(params.source);
+                return ep;
+            },
+
+            /**
+             * performs the given function operation on all the connections found
+             * for the given element id; this means we find all the endpoints for
+             * the given element, and then for each endpoint find the connectors
+             * connected to it. then we pass each connection in to the given
+             * function.
+             */
+                _operation = function(elId, func, endpointFunc) {
+                var endpoints = endpointsByElement[elId];
+                if (endpoints && endpoints.length) {
+                    for ( var i = 0; i < endpoints.length; i++) {
+                        for ( var j = 0; j < endpoints[i].connections.length; j++) {
+                            var retVal = func(endpoints[i].connections[j]);
+                            // if the function passed in returns true, we exit.
+                            // most functions return false.
+                            if (retVal) return;
+                        }
+                        if (endpointFunc) endpointFunc(endpoints[i]);
+                    }
+                }
+            },
+            /**
+             * perform an operation on all elements.
+             */
+                _operationOnAll = function(func) {
+                for ( var elId in endpointsByElement) {
+                    _operation(elId, func);
+                }
+            },
+
+            /**
+             * helper to remove an element from the DOM.
+             */
+                _removeElement = function(element, parent) {
+                if (element != null && element.parentNode != null) {
+                    element.parentNode.removeChild(element);
+                }
+            },
+            /**
+             * helper to remove a list of elements from the DOM.
+             */
+                _removeElements = function(elements, parent) {
+                for ( var i = 0; i < elements.length; i++)
+                    _removeElement(elements[i], parent);
+            },
+            /**
+             * Sets whether or not the given element(s) should be draggable,
+             * regardless of what a particular plumb command may request.
+             *
+             * @param element
+             *            May be a string, a element objects, or a list of
+             *            strings/elements.
+             * @param draggable
+             *            Whether or not the given element(s) should be draggable.
+             */
+                _setDraggable = function(element, draggable) {
+                return _elementProxy(element, function(el, id) {
+                    draggableStates[id] = draggable;
+                    if (jsPlumb.CurrentLibrary.isDragSupported(el)) {
+                        jsPlumb.CurrentLibrary.setDraggable(el, draggable);
+                    }
+                });
+            },
+            /**
+             * private method to do the business of hiding/showing.
+             *
+             * @param el
+             *            either Id of the element in question or a library specific
+             *            object for the element.
+             * @param state
+             *            String specifying a value for the css 'display' property
+             *            ('block' or 'none').
+             */
+                _setVisible = function(el, state, alsoChangeEndpoints) {
+                state = state === "block";
+                var endpointFunc = null;
+                if (alsoChangeEndpoints) {
+                    if (state) endpointFunc = function(ep) {
+                        ep.setVisible(true, true, true);
+                    };
+                    else endpointFunc = function(ep) {
+                        ep.setVisible(false, true, true);
+                    };
+                }
+                var id = _getAttribute(el, "id");
+                _operation(id, function(jpc) {
+                    if (state && alsoChangeEndpoints) {
+                        // this test is necessary because this functionality is new, and i wanted to maintain backwards compatibility.
+                        // this block will only set a connection to be visible if the other endpoint in the connection is also visible.
+                        var oidx = jpc.sourceId === id ? 1 : 0;
+                        if (jpc.endpoints[oidx].isVisible()) jpc.setVisible(true);
+                    }
+                    else  // the default behaviour for show, and what always happens for hide, is to just set the visibility without getting clever.
+                        jpc.setVisible(state);
+                }, endpointFunc);
+            },
+            /**
+             * toggles the draggable state of the given element(s).
+             *
+             * @param el
+             *            either an id, or an element object, or a list of
+             *            ids/element objects.
+             */
+                _toggleDraggable = function(el) {
+                return _elementProxy(el, function(el, elId) {
+                    var state = draggableStates[elId] == null ? false : draggableStates[elId];
+                    state = !state;
+                    draggableStates[elId] = state;
+                    jsPlumb.CurrentLibrary.setDraggable(el, state);
+                    return state;
+                });
+            },
+            /**
+             * private method to do the business of toggling hiding/showing.
+             *
+             * @param elId
+             *            Id of the element in question
+             */
+                _toggleVisible = function(elId, changeEndpoints) {
+                var endpointFunc = null;
+                if (changeEndpoints) {
+                    endpointFunc = function(ep) {
+                        var state = ep.isVisible();
+                        ep.setVisible(!state);
+                    };
+                }
+                _operation(elId, function(jpc) {
+                    var state = jpc.isVisible();
+                    jpc.setVisible(!state);
+                }, endpointFunc);
+                // todo this should call _elementProxy, and pass in the
+                // _operation(elId, f) call as a function. cos _toggleDraggable does
+                // that.
+            },
+            /**
+             * updates the offset and size for a given element, and stores the
+             * values. if 'offset' is not null we use that (it would have been
+             * passed in from a drag call) because it's faster; but if it is null,
+             * or if 'recalc' is true in order to force a recalculation, we get the current values.
+             */
+                _updateOffset = function(params) {
+                var timestamp = params.timestamp, recalc = params.recalc, offset = params.offset, elId = params.elId;
+                if (_suspendDrawing && !timestamp) timestamp = _suspendedAt;
+                if (!recalc) {
+                    if (timestamp && timestamp === offsetTimestamps[elId])
+                        return offsets[elId];
+                }
+                if (recalc || !offset) { // if forced repaint or no offset available, we recalculate.
+                    // get the current size and offset, and store them
+                    var s = _getElementObject(elId);
+                    if (s != null) {
+                        sizes[elId] = _getSize(s);
+                        offsets[elId] = _getOffset(s, _currentInstance);
+                        offsetTimestamps[elId] = timestamp;
+                    }
+                } else {
+                    offsets[elId] = offset;
+                    if (sizes[elId] == null) {
+                        var s = _getElementObject(elId);
+                        if (s != null) sizes[elId] = _getSize(s);
+                    }
+                }
+
+                if(offsets[elId] && !offsets[elId].right) {
+                    offsets[elId].right = offsets[elId].left + sizes[elId][0];
+                    offsets[elId].bottom = offsets[elId].top + sizes[elId][1];
+                    offsets[elId].width = sizes[elId][0];
+                    offsets[elId].height = sizes[elId][1];
+                    offsets[elId].centerx = offsets[elId].left + (offsets[elId].width / 2);
+                    offsets[elId].centery = offsets[elId].top + (offsets[elId].height / 2);
+                }
+                return offsets[elId];
+            },
+
+        // TODO comparison performance
+            _getCachedData = function(elId) {
+                var o = offsets[elId];
+                if (!o) o = _updateOffset({elId:elId});
+                return {o:o, s:sizes[elId]};
+            },
+
+            /**
+             * gets an id for the given element, creating and setting one if
+             * necessary.  the id is of the form
+             *
+             *	jsPlumb_<instance index>_<index in instance>
+             *
+             * where "index in instance" is a monotonically increasing integer that starts at 0,
+             * for each instance.  this method is used not only to assign ids to elements that do not
+             * have them but also to connections and endpoints.
+             */
+                _getId = function(element, uuid, doNotCreateIfNotFound) {
+                var ele = _getElementObject(element);
+                var id = _getAttribute(ele, "id");
+                if (!id || id == "undefined") {
+                    // check if fixed uuid parameter is given
+                    if (arguments.length == 2 && arguments[1] != undefined)
+                        id = uuid;
+                    else if (arguments.length == 1 || (arguments.length == 3 && !arguments[2]))
+                        id = "jsPlumb_" + _instanceIndex + "_" + _idstamp();
+
+                    if (!doNotCreateIfNotFound) _setAttribute(ele, "id", id);
+                }
+                return id;
+            },
+
+            /**
+             * wraps one function with another, creating a placeholder for the
+             * wrapped function if it was null. this is used to wrap the various
+             * drag/drop event functions - to allow jsPlumb to be notified of
+             * important lifecycle events without imposing itself on the user's
+             * drag/drop functionality. TODO: determine whether or not we should
+             * support an error handler concept, if one of the functions fails.
+             *
+             * @param wrappedFunction original function to wrap; may be null.
+             * @param newFunction function to wrap the original with.
+             * @param returnOnThisValue Optional. Indicates that the wrappedFunction should
+             * not be executed if the newFunction returns a value matching 'returnOnThisValue'.
+             * note that this is a simple comparison and only works for primitives right now.
+             */
+                _wrap = function(wrappedFunction, newFunction, returnOnThisValue) {
+                wrappedFunction = wrappedFunction || function() { };
+                newFunction = newFunction || function() { };
+                return function() {
+                    var r = null;
+                    try {
+                        r = newFunction.apply(this, arguments);
+                    } catch (e) {
+                        _log(_currentInstance, "jsPlumb function failed : " + e);
+                    }
+                    if (returnOnThisValue == null || (r !== returnOnThisValue)) {
+                        try {
+                            wrappedFunction.apply(this, arguments);
+                        } catch (e) {
+                            _log(_currentInstance, "wrapped function failed : " + e);
+                        }
+                    }
+                    return r;
+                };
+            };
+
+        /*
+         * Property: connectorClass 
+         *   The CSS class to set on Connection elements. This value is a String and can have multiple classes; the entire String is appended as-is.
+         */
+        this.connectorClass = "_jsPlumb_connector";
+
+        /*
+         * Property: endpointClass 
+         *   The CSS class to set on Endpoint elements. This value is a String and can have multiple classes; the entire String is appended as-is.
+         */
+        this.endpointClass = "_jsPlumb_endpoint";
+
+        /*
+         * Property: overlayClass 
+         * The CSS class to set on an Overlay that is an HTML element. This value is a String and can have multiple classes; the entire String is appended as-is.
+         */
+        this.overlayClass = "_jsPlumb_overlay";
+
+        this.Anchors = {};
+
+        this.Connectors = {
+            "canvas":{},
+            "svg":{},
+            "vml":{}
+        };
+
+        this.Endpoints = {
+            "canvas":{},
+            "svg":{},
+            "vml":{}
+        };
+
+        this.Overlays = {
+            "canvas":{},
+            "svg":{},
+            "vml":{}
+        };
+
+// ************************ PLACEHOLDER DOC ENTRIES FOR NATURAL DOCS *****************************************
+
+        /*
+         * Function: importDefaults
+         * Imports all the given defaults into this instance of jsPlumb.		
+         */
+
+        /*
+         * Function: restoreDefaults
+         * Restores the default settings to "factory" values.
+         */
+
+        /*
+         * Function: bind
+         * Bind to an event on jsPlumb.  
+         * 
+         * Parameters:
+         * 	event - the event to bind.  Available events on jsPlumb are:
+         *         - *jsPlumbConnection* 			: 	notification that a new Connection was established.  jsPlumb passes the new Connection to the callback.
+         *         - *jsPlumbConnectionDetached* 	: 	notification that a Connection was detached.  jsPlumb passes the detached Connection to the callback.
+         *         - *click*						:	notification that a Connection was clicked.  jsPlumb passes the Connection that was clicked to the callback.
+         *         - *dblclick*						:	notification that a Connection was double clicked.  jsPlumb passes the Connection that was double clicked to the callback.
+         *         - *endpointClick*				:	notification that an Endpoint was clicked.  jsPlumb passes the Endpoint that was clicked to the callback.
+         *         - *endpointDblClick*				:	notification that an Endpoint was double clicked.  jsPlumb passes the Endpoint that was double clicked to the callback.
+         *         - *beforeDrop*					: 	notification that a Connection is about to be dropped. Returning false from this method cancels the drop. jsPlumb passes { sourceId, targetId, scope, connection, dropEndpoint } to your callback. For more information, refer to the jsPlumb documentation.
+         *         - *beforeDetach*					: 	notification that a Connection is about to be detached. Returning false from this method cancels the detach. jsPlumb passes the Connection to your callback. For more information, refer to the jsPlumb documentation.
+         *		   - *connectionDrag* 				:   notification that an existing Connection is being dragged. jsPlumb passes the Connection to your callback function.
+         *         - *connectionDragEnd*            :   notification that the drag of an existing Connection has ended.  jsPlumb passes the Connection to your callback function.
+         *         
+         *  callback - function to callback. This function will be passed the Connection/Endpoint that caused the event, and also the original event.    
+         */
+
+        /*
+         * Function: unbind
+         * Clears either all listeners, or listeners for some specific event.
+         * 
+         * Parameters:
+         * 	event	-	optional. constrains the clear to just listeners for this event.
+         */
+
+        /*
+         * Function: addClass
+         * Add class(es) to some element(s).
+         *
+         * Parameters:
+         * 	el			-	element id, dom element, or selector representing the element(s) to add class(es) to.
+         * 	clazz		-	string representing the class(es) to add. may contain multiple classes separated by spaces.
+         */
+
+        /*
+         * Function: removeClass
+         * Remove class from some selement(s).
+         *
+         * Parameters:
+         * 	el			-	element id, dom element, or selector representing the element(s) to remove a class from.
+         * 	clazz		-	string representing the class to remove. 
+         */
+
+        /*
+         * Function: hasClass
+         * Checks if an element has some class.
+         *
+         * Parameters:
+         * 	el			-	element id, dom element, or selector representing the element to test. If the selector matches multiple elements, we return the test for the first element in the selector only.
+         * 	clazz		-	string representing the class to test for. 
+         */
+
+// *************** END OF PLACEHOLDER DOC ENTRIES FOR NATURAL DOCS ***********************************************************		
+
+
+// --------------------------- jsPLumbInstance public API ---------------------------------------------------------
+
+        this.addClass = function(el, clazz) { return jsPlumb.CurrentLibrary.addClass(el, clazz); };
+        this.removeClass = function(el, clazz) { return jsPlumb.CurrentLibrary.removeClass(el, clazz); };
+        this.hasClass = function(el, clazz) { return jsPlumb.CurrentLibrary.hasClass(el, clazz); };
+
+        /*
+         Function: addEndpoint 
+
+         Adds an <Endpoint> to a given element or elements.
+
+         Parameters:
+
+         el - Element to add the endpoint to. Either an element id, a selector representing some element(s), or an array of either of these. 
+         params - Object containing Endpoint constructor arguments.  For more information, see <Endpoint>.
+         referenceParams - Object containing more Endpoint constructor arguments; it will be merged with params by jsPlumb.  You would use this if you had some 
+         shared parameters that you wanted to reuse when you added Endpoints to a number of elements. The allowed values in
+         this object are anything that 'params' can contain.  See <Endpoint>.
+
+         Returns: 
+         The newly created <Endpoint>, if el referred to a single element.  Otherwise, an array of newly created <Endpoint>s. 
+
+         See Also: 
+         <addEndpoints>
+         */
+        this.addEndpoint = function(el, params, referenceParams) {
+            referenceParams = referenceParams || {};
+            var p = jsPlumb.extend({}, referenceParams);
+            jsPlumb.extend(p, params);
+            p.endpoint = 

<TRUNCATED>

[14/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/resources/webapps/static/js/jquery.js
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/static/js/jquery.js b/tajo-core/src/main/resources/webapps/static/js/jquery.js
new file mode 100644
index 0000000..63174a0
--- /dev/null
+++ b/tajo-core/src/main/resources/webapps/static/js/jquery.js
@@ -0,0 +1,2 @@
+/*! jQuery v1.8.2 jquery.com | jquery.org/license */
+(function(a,b){function G(a){var b=F[a]={};return p.each(a.split(s),function(a,c){b[c]=!0}),b}function J(a,c,d){if(d===b&&a.nodeType===1){var e="data-"+c.replace(I,"-$1").toLowerCase();d=a.getAttribute(e);if(typeof d=="string"){try{d=d==="true"?!0:d==="false"?!1:d==="null"?null:+d+""===d?+d:H.test(d)?p.parseJSON(d):d}catch(f){}p.data(a,c,d)}else d=b}return d}function K(a){var b;for(b in a){if(b==="data"&&p.isEmptyObject(a[b]))continue;if(b!=="toJSON")return!1}return!0}function ba(){return!1}function bb(){return!0}function bh(a){return!a||!a.parentNode||a.parentNode.nodeType===11}function bi(a,b){do a=a[b];while(a&&a.nodeType!==1);return a}function bj(a,b,c){b=b||0;if(p.isFunction(b))return p.grep(a,function(a,d){var e=!!b.call(a,d,a);return e===c});if(b.nodeType)return p.grep(a,function(a,d){return a===b===c});if(typeof b=="string"){var d=p.grep(a,function(a){return a.nodeType===1});if(be.test(b))return p.filter(b,d,!c);b=p.filter(b,d)}return p.grep(a,function(a,d){return p.inArray(
 a,b)>=0===c})}function bk(a){var b=bl.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}function bC(a,b){return a.getElementsByTagName(b)[0]||a.appendChild(a.ownerDocument.createElement(b))}function bD(a,b){if(b.nodeType!==1||!p.hasData(a))return;var c,d,e,f=p._data(a),g=p._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;d<e;d++)p.event.add(b,c,h[c][d])}g.data&&(g.data=p.extend({},g.data))}function bE(a,b){var c;if(b.nodeType!==1)return;b.clearAttributes&&b.clearAttributes(),b.mergeAttributes&&b.mergeAttributes(a),c=b.nodeName.toLowerCase(),c==="object"?(b.parentNode&&(b.outerHTML=a.outerHTML),p.support.html5Clone&&a.innerHTML&&!p.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):c==="input"&&bv.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):c==="option"?b.selected=a.defaultSelected:c==="input"||c==="textarea"?b.defaultValue=a.defaultValue:c==="scri
 pt"&&b.text!==a.text&&(b.text=a.text),b.removeAttribute(p.expando)}function bF(a){return typeof a.getElementsByTagName!="undefined"?a.getElementsByTagName("*"):typeof a.querySelectorAll!="undefined"?a.querySelectorAll("*"):[]}function bG(a){bv.test(a.type)&&(a.defaultChecked=a.checked)}function bY(a,b){if(b in a)return b;var c=b.charAt(0).toUpperCase()+b.slice(1),d=b,e=bW.length;while(e--){b=bW[e]+c;if(b in a)return b}return d}function bZ(a,b){return a=b||a,p.css(a,"display")==="none"||!p.contains(a.ownerDocument,a)}function b$(a,b){var c,d,e=[],f=0,g=a.length;for(;f<g;f++){c=a[f];if(!c.style)continue;e[f]=p._data(c,"olddisplay"),b?(!e[f]&&c.style.display==="none"&&(c.style.display=""),c.style.display===""&&bZ(c)&&(e[f]=p._data(c,"olddisplay",cc(c.nodeName)))):(d=bH(c,"display"),!e[f]&&d!=="none"&&p._data(c,"olddisplay",d))}for(f=0;f<g;f++){c=a[f];if(!c.style)continue;if(!b||c.style.display==="none"||c.style.display==="")c.style.display=b?e[f]||"":"none"}return a}function b_(a,b,c){
 var d=bP.exec(b);return d?Math.max(0,d[1]-(c||0))+(d[2]||"px"):b}function ca(a,b,c,d){var e=c===(d?"border":"content")?4:b==="width"?1:0,f=0;for(;e<4;e+=2)c==="margin"&&(f+=p.css(a,c+bV[e],!0)),d?(c==="content"&&(f-=parseFloat(bH(a,"padding"+bV[e]))||0),c!=="margin"&&(f-=parseFloat(bH(a,"border"+bV[e]+"Width"))||0)):(f+=parseFloat(bH(a,"padding"+bV[e]))||0,c!=="padding"&&(f+=parseFloat(bH(a,"border"+bV[e]+"Width"))||0));return f}function cb(a,b,c){var d=b==="width"?a.offsetWidth:a.offsetHeight,e=!0,f=p.support.boxSizing&&p.css(a,"boxSizing")==="border-box";if(d<=0||d==null){d=bH(a,b);if(d<0||d==null)d=a.style[b];if(bQ.test(d))return d;e=f&&(p.support.boxSizingReliable||d===a.style[b]),d=parseFloat(d)||0}return d+ca(a,b,c||(f?"border":"content"),e)+"px"}function cc(a){if(bS[a])return bS[a];var b=p("<"+a+">").appendTo(e.body),c=b.css("display");b.remove();if(c==="none"||c===""){bI=e.body.appendChild(bI||p.extend(e.createElement("iframe"),{frameBorder:0,width:0,height:0}));if(!bJ||!bI.
 createElement)bJ=(bI.contentWindow||bI.contentDocument).document,bJ.write("<!doctype html><html><body>"),bJ.close();b=bJ.body.appendChild(bJ.createElement(a)),c=bH(b,"display"),e.body.removeChild(bI)}return bS[a]=c,c}function ci(a,b,c,d){var e;if(p.isArray(b))p.each(b,function(b,e){c||ce.test(a)?d(a,e):ci(a+"["+(typeof e=="object"?b:"")+"]",e,c,d)});else if(!c&&p.type(b)==="object")for(e in b)ci(a+"["+e+"]",b[e],c,d);else d(a,b)}function cz(a){return function(b,c){typeof b!="string"&&(c=b,b="*");var d,e,f,g=b.toLowerCase().split(s),h=0,i=g.length;if(p.isFunction(c))for(;h<i;h++)d=g[h],f=/^\+/.test(d),f&&(d=d.substr(1)||"*"),e=a[d]=a[d]||[],e[f?"unshift":"push"](c)}}function cA(a,c,d,e,f,g){f=f||c.dataTypes[0],g=g||{},g[f]=!0;var h,i=a[f],j=0,k=i?i.length:0,l=a===cv;for(;j<k&&(l||!h);j++)h=i[j](c,d,e),typeof h=="string"&&(!l||g[h]?h=b:(c.dataTypes.unshift(h),h=cA(a,c,d,e,h,g)));return(l||!h)&&!g["*"]&&(h=cA(a,c,d,e,"*",g)),h}function cB(a,c){var d,e,f=p.ajaxSettings.flatOptions||{};f
 or(d in c)c[d]!==b&&((f[d]?a:e||(e={}))[d]=c[d]);e&&p.extend(!0,a,e)}function cC(a,c,d){var e,f,g,h,i=a.contents,j=a.dataTypes,k=a.responseFields;for(f in k)f in d&&(c[k[f]]=d[f]);while(j[0]==="*")j.shift(),e===b&&(e=a.mimeType||c.getResponseHeader("content-type"));if(e)for(f in i)if(i[f]&&i[f].test(e)){j.unshift(f);break}if(j[0]in d)g=j[0];else{for(f in d){if(!j[0]||a.converters[f+" "+j[0]]){g=f;break}h||(h=f)}g=g||h}if(g)return g!==j[0]&&j.unshift(g),d[g]}function cD(a,b){var c,d,e,f,g=a.dataTypes.slice(),h=g[0],i={},j=0;a.dataFilter&&(b=a.dataFilter(b,a.dataType));if(g[1])for(c in a.converters)i[c.toLowerCase()]=a.converters[c];for(;e=g[++j];)if(e!=="*"){if(h!=="*"&&h!==e){c=i[h+" "+e]||i["* "+e];if(!c)for(d in i){f=d.split(" ");if(f[1]===e){c=i[h+" "+f[0]]||i["* "+f[0]];if(c){c===!0?c=i[d]:i[d]!==!0&&(e=f[0],g.splice(j--,0,e));break}}}if(c!==!0)if(c&&a["throws"])b=c(b);else try{b=c(b)}catch(k){return{state:"parsererror",error:c?k:"No conversion from "+h+" to "+e}}}h=e}return{sta
 te:"success",data:b}}function cL(){try{return new a.XMLHttpRequest}catch(b){}}function cM(){try{return new a.ActiveXObject("Microsoft.XMLHTTP")}catch(b){}}function cU(){return setTimeout(function(){cN=b},0),cN=p.now()}function cV(a,b){p.each(b,function(b,c){var d=(cT[b]||[]).concat(cT["*"]),e=0,f=d.length;for(;e<f;e++)if(d[e].call(a,b,c))return})}function cW(a,b,c){var d,e=0,f=0,g=cS.length,h=p.Deferred().always(function(){delete i.elem}),i=function(){var b=cN||cU(),c=Math.max(0,j.startTime+j.duration-b),d=1-(c/j.duration||0),e=0,f=j.tweens.length;for(;e<f;e++)j.tweens[e].run(d);return h.notifyWith(a,[j,d,c]),d<1&&f?c:(h.resolveWith(a,[j]),!1)},j=h.promise({elem:a,props:p.extend({},b),opts:p.extend(!0,{specialEasing:{}},c),originalProperties:b,originalOptions:c,startTime:cN||cU(),duration:c.duration,tweens:[],createTween:function(b,c,d){var e=p.Tween(a,j.opts,b,c,j.opts.specialEasing[b]||j.opts.easing);return j.tweens.push(e),e},stop:function(b){var c=0,d=b?j.tweens.length:0;for(;c<
 d;c++)j.tweens[c].run(1);return b?h.resolveWith(a,[j,b]):h.rejectWith(a,[j,b]),this}}),k=j.props;cX(k,j.opts.specialEasing);for(;e<g;e++){d=cS[e].call(j,a,k,j.opts);if(d)return d}return cV(j,k),p.isFunction(j.opts.start)&&j.opts.start.call(a,j),p.fx.timer(p.extend(i,{anim:j,queue:j.opts.queue,elem:a})),j.progress(j.opts.progress).done(j.opts.done,j.opts.complete).fail(j.opts.fail).always(j.opts.always)}function cX(a,b){var c,d,e,f,g;for(c in a){d=p.camelCase(c),e=b[d],f=a[c],p.isArray(f)&&(e=f[1],f=a[c]=f[0]),c!==d&&(a[d]=f,delete a[c]),g=p.cssHooks[d];if(g&&"expand"in g){f=g.expand(f),delete a[d];for(c in f)c in a||(a[c]=f[c],b[c]=e)}else b[d]=e}}function cY(a,b,c){var d,e,f,g,h,i,j,k,l=this,m=a.style,n={},o=[],q=a.nodeType&&bZ(a);c.queue||(j=p._queueHooks(a,"fx"),j.unqueued==null&&(j.unqueued=0,k=j.empty.fire,j.empty.fire=function(){j.unqueued||k()}),j.unqueued++,l.always(function(){l.always(function(){j.unqueued--,p.queue(a,"fx").length||j.empty.fire()})})),a.nodeType===1&&("heig
 ht"in b||"width"in b)&&(c.overflow=[m.overflow,m.overflowX,m.overflowY],p.css(a,"display")==="inline"&&p.css(a,"float")==="none"&&(!p.support.inlineBlockNeedsLayout||cc(a.nodeName)==="inline"?m.display="inline-block":m.zoom=1)),c.overflow&&(m.overflow="hidden",p.support.shrinkWrapBlocks||l.done(function(){m.overflow=c.overflow[0],m.overflowX=c.overflow[1],m.overflowY=c.overflow[2]}));for(d in b){f=b[d];if(cP.exec(f)){delete b[d];if(f===(q?"hide":"show"))continue;o.push(d)}}g=o.length;if(g){h=p._data(a,"fxshow")||p._data(a,"fxshow",{}),q?p(a).show():l.done(function(){p(a).hide()}),l.done(function(){var b;p.removeData(a,"fxshow",!0);for(b in n)p.style(a,b,n[b])});for(d=0;d<g;d++)e=o[d],i=l.createTween(e,q?h[e]:0),n[e]=h[e]||p.style(a,e),e in h||(h[e]=i.start,q&&(i.end=i.start,i.start=e==="width"||e==="height"?1:0))}}function cZ(a,b,c,d,e){return new cZ.prototype.init(a,b,c,d,e)}function c$(a,b){var c,d={height:a},e=0;b=b?1:0;for(;e<4;e+=2-b)c=bV[e],d["margin"+c]=d["padding"+c]=a;retur
 n b&&(d.opacity=d.width=a),d}function da(a){return p.isWindow(a)?a:a.nodeType===9?a.defaultView||a.parentWindow:!1}var c,d,e=a.document,f=a.location,g=a.navigator,h=a.jQuery,i=a.$,j=Array.prototype.push,k=Array.prototype.slice,l=Array.prototype.indexOf,m=Object.prototype.toString,n=Object.prototype.hasOwnProperty,o=String.prototype.trim,p=function(a,b){return new p.fn.init(a,b,c)},q=/[\-+]?(?:\d*\.|)\d+(?:[eE][\-+]?\d+|)/.source,r=/\S/,s=/\s+/,t=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,u=/^(?:[^#<]*(<[\w\W]+>)[^>]*$|#([\w\-]*)$)/,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^[\],:{}\s]*$/,x=/(?:^|:|,)(?:\s*\[)+/g,y=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,z=/"[^"\\\r\n]*"|true|false|null|-?(?:\d\d*\.|)\d+(?:[eE][\-+]?\d+|)/g,A=/^-ms-/,B=/-([\da-z])/gi,C=function(a,b){return(b+"").toUpperCase()},D=function(){e.addEventListener?(e.removeEventListener("DOMContentLoaded",D,!1),p.ready()):e.readyState==="complete"&&(e.detachEvent("onreadystatechange",D),p.ready())},E={};p.fn=p.prototype={constructor:p,init
 :function(a,c,d){var f,g,h,i;if(!a)return this;if(a.nodeType)return this.context=this[0]=a,this.length=1,this;if(typeof a=="string"){a.charAt(0)==="<"&&a.charAt(a.length-1)===">"&&a.length>=3?f=[null,a,null]:f=u.exec(a);if(f&&(f[1]||!c)){if(f[1])return c=c instanceof p?c[0]:c,i=c&&c.nodeType?c.ownerDocument||c:e,a=p.parseHTML(f[1],i,!0),v.test(f[1])&&p.isPlainObject(c)&&this.attr.call(a,c,!0),p.merge(this,a);g=e.getElementById(f[2]);if(g&&g.parentNode){if(g.id!==f[2])return d.find(a);this.length=1,this[0]=g}return this.context=e,this.selector=a,this}return!c||c.jquery?(c||d).find(a):this.constructor(c).find(a)}return p.isFunction(a)?d.ready(a):(a.selector!==b&&(this.selector=a.selector,this.context=a.context),p.makeArray(a,this))},selector:"",jquery:"1.8.2",length:0,size:function(){return this.length},toArray:function(){return k.call(this)},get:function(a){return a==null?this.toArray():a<0?this[this.length+a]:this[a]},pushStack:function(a,b,c){var d=p.merge(this.constructor(),a);ret
 urn d.prevObject=this,d.context=this.context,b==="find"?d.selector=this.selector+(this.selector?" ":"")+c:b&&(d.selector=this.selector+"."+b+"("+c+")"),d},each:function(a,b){return p.each(this,a,b)},ready:function(a){return p.ready.promise().done(a),this},eq:function(a){return a=+a,a===-1?this.slice(a):this.slice(a,a+1)},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},slice:function(){return this.pushStack(k.apply(this,arguments),"slice",k.call(arguments).join(","))},map:function(a){return this.pushStack(p.map(this,function(b,c){return a.call(b,c,b)}))},end:function(){return this.prevObject||this.constructor(null)},push:j,sort:[].sort,splice:[].splice},p.fn.init.prototype=p.fn,p.extend=p.fn.extend=function(){var a,c,d,e,f,g,h=arguments[0]||{},i=1,j=arguments.length,k=!1;typeof h=="boolean"&&(k=h,h=arguments[1]||{},i=2),typeof h!="object"&&!p.isFunction(h)&&(h={}),j===i&&(h=this,--i);for(;i<j;i++)if((a=arguments[i])!=null)for(c in a){d=h[c],e=a[c];if(h===e)con
 tinue;k&&e&&(p.isPlainObject(e)||(f=p.isArray(e)))?(f?(f=!1,g=d&&p.isArray(d)?d:[]):g=d&&p.isPlainObject(d)?d:{},h[c]=p.extend(k,g,e)):e!==b&&(h[c]=e)}return h},p.extend({noConflict:function(b){return a.$===p&&(a.$=i),b&&a.jQuery===p&&(a.jQuery=h),p},isReady:!1,readyWait:1,holdReady:function(a){a?p.readyWait++:p.ready(!0)},ready:function(a){if(a===!0?--p.readyWait:p.isReady)return;if(!e.body)return setTimeout(p.ready,1);p.isReady=!0;if(a!==!0&&--p.readyWait>0)return;d.resolveWith(e,[p]),p.fn.trigger&&p(e).trigger("ready").off("ready")},isFunction:function(a){return p.type(a)==="function"},isArray:Array.isArray||function(a){return p.type(a)==="array"},isWindow:function(a){return a!=null&&a==a.window},isNumeric:function(a){return!isNaN(parseFloat(a))&&isFinite(a)},type:function(a){return a==null?String(a):E[m.call(a)]||"object"},isPlainObject:function(a){if(!a||p.type(a)!=="object"||a.nodeType||p.isWindow(a))return!1;try{if(a.constructor&&!n.call(a,"constructor")&&!n.call(a.constructo
 r.prototype,"isPrototypeOf"))return!1}catch(c){return!1}var d;for(d in a);return d===b||n.call(a,d)},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},error:function(a){throw new Error(a)},parseHTML:function(a,b,c){var d;return!a||typeof a!="string"?null:(typeof b=="boolean"&&(c=b,b=0),b=b||e,(d=v.exec(a))?[b.createElement(d[1])]:(d=p.buildFragment([a],b,c?null:[]),p.merge([],(d.cacheable?p.clone(d.fragment):d.fragment).childNodes)))},parseJSON:function(b){if(!b||typeof b!="string")return null;b=p.trim(b);if(a.JSON&&a.JSON.parse)return a.JSON.parse(b);if(w.test(b.replace(y,"@").replace(z,"]").replace(x,"")))return(new Function("return "+b))();p.error("Invalid JSON: "+b)},parseXML:function(c){var d,e;if(!c||typeof c!="string")return null;try{a.DOMParser?(e=new DOMParser,d=e.parseFromString(c,"text/xml")):(d=new ActiveXObject("Microsoft.XMLDOM"),d.async="false",d.loadXML(c))}catch(f){d=b}return(!d||!d.documentElement||d.getElementsByTagName("parsererror").length)&&p.error(
 "Invalid XML: "+c),d},noop:function(){},globalEval:function(b){b&&r.test(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(A,"ms-").replace(B,C)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,c,d){var e,f=0,g=a.length,h=g===b||p.isFunction(a);if(d){if(h){for(e in a)if(c.apply(a[e],d)===!1)break}else for(;f<g;)if(c.apply(a[f++],d)===!1)break}else if(h){for(e in a)if(c.call(a[e],e,a[e])===!1)break}else for(;f<g;)if(c.call(a[f],f,a[f++])===!1)break;return a},trim:o&&!o.call(" ")?function(a){return a==null?"":o.call(a)}:function(a){return a==null?"":(a+"").replace(t,"")},makeArray:function(a,b){var c,d=b||[];return a!=null&&(c=p.type(a),a.length==null||c==="string"||c==="function"||c==="regexp"||p.isWindow(a)?j.call(d,a):p.merge(d,a)),d},inArray:function(a,b,c){var d;if(b){if(l)return l.call(b,a,c);d=b.length,c=c?c<0?Math.max(0,d+c):c:0;for(;c<d;c++)if(c in b&&b[c]===a)return c}return-1
 },merge:function(a,c){var d=c.length,e=a.length,f=0;if(typeof d=="number")for(;f<d;f++)a[e++]=c[f];else while(c[f]!==b)a[e++]=c[f++];return a.length=e,a},grep:function(a,b,c){var d,e=[],f=0,g=a.length;c=!!c;for(;f<g;f++)d=!!b(a[f],f),c!==d&&e.push(a[f]);return e},map:function(a,c,d){var e,f,g=[],h=0,i=a.length,j=a instanceof p||i!==b&&typeof i=="number"&&(i>0&&a[0]&&a[i-1]||i===0||p.isArray(a));if(j)for(;h<i;h++)e=c(a[h],h,d),e!=null&&(g[g.length]=e);else for(f in a)e=c(a[f],f,d),e!=null&&(g[g.length]=e);return g.concat.apply([],g)},guid:1,proxy:function(a,c){var d,e,f;return typeof c=="string"&&(d=a[c],c=a,a=d),p.isFunction(a)?(e=k.call(arguments,2),f=function(){return a.apply(c,e.concat(k.call(arguments)))},f.guid=a.guid=a.guid||p.guid++,f):b},access:function(a,c,d,e,f,g,h){var i,j=d==null,k=0,l=a.length;if(d&&typeof d=="object"){for(k in d)p.access(a,c,k,d[k],1,g,e);f=1}else if(e!==b){i=h===b&&p.isFunction(e),j&&(i?(i=c,c=function(a,b,c){return i.call(p(a),c)}):(c.call(a,e),c=nul
 l));if(c)for(;k<l;k++)c(a[k],d,i?e.call(a[k],k,c(a[k],d)):e,h);f=1}return f?a:j?c.call(a):l?c(a[0],d):g},now:function(){return(new Date).getTime()}}),p.ready.promise=function(b){if(!d){d=p.Deferred();if(e.readyState==="complete")setTimeout(p.ready,1);else if(e.addEventListener)e.addEventListener("DOMContentLoaded",D,!1),a.addEventListener("load",p.ready,!1);else{e.attachEvent("onreadystatechange",D),a.attachEvent("onload",p.ready);var c=!1;try{c=a.frameElement==null&&e.documentElement}catch(f){}c&&c.doScroll&&function g(){if(!p.isReady){try{c.doScroll("left")}catch(a){return setTimeout(g,50)}p.ready()}}()}}return d.promise(b)},p.each("Boolean Number String Function Array Date RegExp Object".split(" "),function(a,b){E["[object "+b+"]"]=b.toLowerCase()}),c=p(e);var F={};p.Callbacks=function(a){a=typeof a=="string"?F[a]||G(a):p.extend({},a);var c,d,e,f,g,h,i=[],j=!a.once&&[],k=function(b){c=a.memory&&b,d=!0,h=f||0,f=0,g=i.length,e=!0;for(;i&&h<g;h++)if(i[h].apply(b[0],b[1])===!1&&a.sto
 pOnFalse){c=!1;break}e=!1,i&&(j?j.length&&k(j.shift()):c?i=[]:l.disable())},l={add:function(){if(i){var b=i.length;(function d(b){p.each(b,function(b,c){var e=p.type(c);e==="function"&&(!a.unique||!l.has(c))?i.push(c):c&&c.length&&e!=="string"&&d(c)})})(arguments),e?g=i.length:c&&(f=b,k(c))}return this},remove:function(){return i&&p.each(arguments,function(a,b){var c;while((c=p.inArray(b,i,c))>-1)i.splice(c,1),e&&(c<=g&&g--,c<=h&&h--)}),this},has:function(a){return p.inArray(a,i)>-1},empty:function(){return i=[],this},disable:function(){return i=j=c=b,this},disabled:function(){return!i},lock:function(){return j=b,c||l.disable(),this},locked:function(){return!j},fireWith:function(a,b){return b=b||[],b=[a,b.slice?b.slice():b],i&&(!d||j)&&(e?j.push(b):k(b)),this},fire:function(){return l.fireWith(this,arguments),this},fired:function(){return!!d}};return l},p.extend({Deferred:function(a){var b=[["resolve","done",p.Callbacks("once memory"),"resolved"],["reject","fail",p.Callbacks("once m
 emory"),"rejected"],["notify","progress",p.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return p.Deferred(function(c){p.each(b,function(b,d){var f=d[0],g=a[b];e[d[1]](p.isFunction(g)?function(){var a=g.apply(this,arguments);a&&p.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f+"With"](this===e?c:this,[a])}:c[f])}),a=null}).promise()},promise:function(a){return a!=null?p.extend(a,d):d}},e={};return d.pipe=d.then,p.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[a^1][2].disable,b[2][2].lock),e[f[0]]=g.fire,e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=k.call(arguments),d=c.length,e=d!==1||a&&p.isFunction(a.promise)?d:0,f=e===1?a:p.Deferred(),g=function(a,b,c){return function(d){b[a]=this,c[a]=arguments.length>1?k.call(arguments):d,c===h?f.notifyWith(b,c):--e||f.
 resolveWith(b,c)}},h,i,j;if(d>1){h=new Array(d),i=new Array(d),j=new Array(d);for(;b<d;b++)c[b]&&p.isFunction(c[b].promise)?c[b].promise().done(g(b,j,c)).fail(f.reject).progress(g(b,i,h)):--e}return e||f.resolveWith(j,c),f.promise()}}),p.support=function(){var b,c,d,f,g,h,i,j,k,l,m,n=e.createElement("div");n.setAttribute("className","t"),n.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",c=n.getElementsByTagName("*"),d=n.getElementsByTagName("a")[0],d.style.cssText="top:1px;float:left;opacity:.5";if(!c||!c.length)return{};f=e.createElement("select"),g=f.appendChild(e.createElement("option")),h=n.getElementsByTagName("input")[0],b={leadingWhitespace:n.firstChild.nodeType===3,tbody:!n.getElementsByTagName("tbody").length,htmlSerialize:!!n.getElementsByTagName("link").length,style:/top/.test(d.getAttribute("style")),hrefNormalized:d.getAttribute("href")==="/a",opacity:/^0.5/.test(d.style.opacity),cssFloat:!!d.style.cssFloat,checkOn:h.value==="on",optSelect
 ed:g.selected,getSetAttribute:n.className!=="t",enctype:!!e.createElement("form").enctype,html5Clone:e.createElement("nav").cloneNode(!0).outerHTML!=="<:nav></:nav>",boxModel:e.compatMode==="CSS1Compat",submitBubbles:!0,changeBubbles:!0,focusinBubbles:!1,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},h.checked=!0,b.noCloneChecked=h.cloneNode(!0).checked,f.disabled=!0,b.optDisabled=!g.disabled;try{delete n.test}catch(o){b.deleteExpando=!1}!n.addEventListener&&n.attachEvent&&n.fireEvent&&(n.attachEvent("onclick",m=function(){b.noCloneEvent=!1}),n.cloneNode(!0).fireEvent("onclick"),n.detachEvent("onclick",m)),h=e.createElement("input"),h.value="t",h.setAttribute("type","radio"),b.radioValue=h.value==="t",h.setAttribute("checked","checked"),h.setAttribute("name","t"),n.appendChild(h),i=e.createDocumentFragment(),i.appendChild(n.lastChild),b.checkClone=i.cloneNode(!0).cloneNode(!0).lastChild.che
 cked,b.appendChecked=h.checked,i.removeChild(h),i.appendChild(n);if(n.attachEvent)for(k in{submit:!0,change:!0,focusin:!0})j="on"+k,l=j in n,l||(n.setAttribute(j,"return;"),l=typeof n[j]=="function"),b[k+"Bubbles"]=l;return p(function(){var c,d,f,g,h="padding:0;margin:0;border:0;display:block;overflow:hidden;",i=e.getElementsByTagName("body")[0];if(!i)return;c=e.createElement("div"),c.style.cssText="visibility:hidden;border:0;width:0;height:0;position:static;top:0;margin-top:1px",i.insertBefore(c,i.firstChild),d=e.createElement("div"),c.appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",f=d.getElementsByTagName("td"),f[0].style.cssText="padding:0;margin:0;border:0;display:none",l=f[0].offsetHeight===0,f[0].style.display="",f[1].style.display="none",b.reliableHiddenOffsets=l&&f[0].offsetHeight===0,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-to
 p:1%;position:absolute;top:1%;",b.boxSizing=d.offsetWidth===4,b.doesNotIncludeMarginInBodyOffset=i.offsetTop!==1,a.getComputedStyle&&(b.pixelPosition=(a.getComputedStyle(d,null)||{}).top!=="1%",b.boxSizingReliable=(a.getComputedStyle(d,null)||{width:"4px"}).width==="4px",g=e.createElement("div"),g.style.cssText=d.style.cssText=h,g.style.marginRight=g.style.width="0",d.style.width="1px",d.appendChild(g),b.reliableMarginRight=!parseFloat((a.getComputedStyle(g,null)||{}).marginRight)),typeof d.style.zoom!="undefined"&&(d.innerHTML="",d.style.cssText=h+"width:1px;padding:1px;display:inline;zoom:1",b.inlineBlockNeedsLayout=d.offsetWidth===3,d.style.display="block",d.style.overflow="visible",d.innerHTML="<div></div>",d.firstChild.style.width="5px",b.shrinkWrapBlocks=d.offsetWidth!==3,c.style.zoom=1),i.removeChild(c),c=d=f=g=null}),i.removeChild(n),c=d=f=g=h=i=n=null,b}();var H=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,I=/([A-Z])/g;p.extend({cache:{},deletedIds:[],uuid:0,expando:"jQuery"+(p.fn.jquery
 +Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(a){return a=a.nodeType?p.cache[a[p.expando]]:a[p.expando],!!a&&!K(a)},data:function(a,c,d,e){if(!p.acceptData(a))return;var f,g,h=p.expando,i=typeof c=="string",j=a.nodeType,k=j?p.cache:a,l=j?a[h]:a[h]&&h;if((!l||!k[l]||!e&&!k[l].data)&&i&&d===b)return;l||(j?a[h]=l=p.deletedIds.pop()||p.guid++:l=h),k[l]||(k[l]={},j||(k[l].toJSON=p.noop));if(typeof c=="object"||typeof c=="function")e?k[l]=p.extend(k[l],c):k[l].data=p.extend(k[l].data,c);return f=k[l],e||(f.data||(f.data={}),f=f.data),d!==b&&(f[p.camelCase(c)]=d),i?(g=f[c],g==null&&(g=f[p.camelCase(c)])):g=f,g},removeData:function(a,b,c){if(!p.acceptData(a))return;var d,e,f,g=a.nodeType,h=g?p.cache:a,i=g?a[p.expando]:p.expando;if(!h[i])return;if(b){d=c?h[i]:h[i].data;if(d){p.isArray(b)||(b in d?b=[b]:(b=p.camelCase(b),b in d?b=[b]:b=b.split(" ")));for(e=0,f=b.length;e<f;e++)delete d[b[e]];if(!(c?K:p.isEmpt
 yObject)(d))return}}if(!c){delete h[i].data;if(!K(h[i]))return}g?p.cleanData([a],!0):p.support.deleteExpando||h!=h.window?delete h[i]:h[i]=null},_data:function(a,b,c){return p.data(a,b,c,!0)},acceptData:function(a){var b=a.nodeName&&p.noData[a.nodeName.toLowerCase()];return!b||b!==!0&&a.getAttribute("classid")===b}}),p.fn.extend({data:function(a,c){var d,e,f,g,h,i=this[0],j=0,k=null;if(a===b){if(this.length){k=p.data(i);if(i.nodeType===1&&!p._data(i,"parsedAttrs")){f=i.attributes;for(h=f.length;j<h;j++)g=f[j].name,g.indexOf("data-")||(g=p.camelCase(g.substring(5)),J(i,g,k[g]));p._data(i,"parsedAttrs",!0)}}return k}return typeof a=="object"?this.each(function(){p.data(this,a)}):(d=a.split(".",2),d[1]=d[1]?"."+d[1]:"",e=d[1]+"!",p.access(this,function(c){if(c===b)return k=this.triggerHandler("getData"+e,[d[0]]),k===b&&i&&(k=p.data(i,a),k=J(i,a,k)),k===b&&d[1]?this.data(d[0]):k;d[1]=c,this.each(function(){var b=p(this);b.triggerHandler("setData"+e,d),p.data(this,a,c),b.triggerHandler("
 changeData"+e,d)})},null,c,arguments.length>1,null,!1))},removeData:function(a){return this.each(function(){p.removeData(this,a)})}}),p.extend({queue:function(a,b,c){var d;if(a)return b=(b||"fx")+"queue",d=p._data(a,b),c&&(!d||p.isArray(c)?d=p._data(a,b,p.makeArray(c)):d.push(c)),d||[]},dequeue:function(a,b){b=b||"fx";var c=p.queue(a,b),d=c.length,e=c.shift(),f=p._queueHooks(a,b),g=function(){p.dequeue(a,b)};e==="inprogress"&&(e=c.shift(),d--),e&&(b==="fx"&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return p._data(a,c)||p._data(a,c,{empty:p.Callbacks("once memory").add(function(){p.removeData(a,b+"queue",!0),p.removeData(a,c,!0)})})}}),p.fn.extend({queue:function(a,c){var d=2;return typeof a!="string"&&(c=a,a="fx",d--),arguments.length<d?p.queue(this[0],a):c===b?this:this.each(function(){var b=p.queue(this,a,c);p._queueHooks(this,a),a==="fx"&&b[0]!=="inprogress"&&p.dequeue(this,a)})},dequeue:function(a){
 return this.each(function(){p.dequeue(this,a)})},delay:function(a,b){return a=p.fx?p.fx.speeds[a]||a:a,b=b||"fx",this.queue(b,function(b,c){var d=setTimeout(b,a);c.stop=function(){clearTimeout(d)}})},clearQueue:function(a){return this.queue(a||"fx",[])},promise:function(a,c){var d,e=1,f=p.Deferred(),g=this,h=this.length,i=function(){--e||f.resolveWith(g,[g])};typeof a!="string"&&(c=a,a=b),a=a||"fx";while(h--)d=p._data(g[h],a+"queueHooks"),d&&d.empty&&(e++,d.empty.add(i));return i(),f.promise(c)}});var L,M,N,O=/[\t\r\n]/g,P=/\r/g,Q=/^(?:button|input)$/i,R=/^(?:button|input|object|select|textarea)$/i,S=/^a(?:rea|)$/i,T=/^(?:autofocus|autoplay|async|checked|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped|selected)$/i,U=p.support.getSetAttribute;p.fn.extend({attr:function(a,b){return p.access(this,p.attr,a,b,arguments.length>1)},removeAttr:function(a){return this.each(function(){p.removeAttr(this,a)})},prop:function(a,b){return p.access(this,p.prop,a,b,argumen
 ts.length>1)},removeProp:function(a){return a=p.propFix[a]||a,this.each(function(){try{this[a]=b,delete this[a]}catch(c){}})},addClass:function(a){var b,c,d,e,f,g,h;if(p.isFunction(a))return this.each(function(b){p(this).addClass(a.call(this,b,this.className))});if(a&&typeof a=="string"){b=a.split(s);for(c=0,d=this.length;c<d;c++){e=this[c];if(e.nodeType===1)if(!e.className&&b.length===1)e.className=a;else{f=" "+e.className+" ";for(g=0,h=b.length;g<h;g++)f.indexOf(" "+b[g]+" ")<0&&(f+=b[g]+" ");e.className=p.trim(f)}}}return this},removeClass:function(a){var c,d,e,f,g,h,i;if(p.isFunction(a))return this.each(function(b){p(this).removeClass(a.call(this,b,this.className))});if(a&&typeof a=="string"||a===b){c=(a||"").split(s);for(h=0,i=this.length;h<i;h++){e=this[h];if(e.nodeType===1&&e.className){d=(" "+e.className+" ").replace(O," ");for(f=0,g=c.length;f<g;f++)while(d.indexOf(" "+c[f]+" ")>=0)d=d.replace(" "+c[f]+" "," ");e.className=a?p.trim(d):""}}}return this},toggleClass:function(
 a,b){var c=typeof a,d=typeof b=="boolean";return p.isFunction(a)?this.each(function(c){p(this).toggleClass(a.call(this,c,this.className,b),b)}):this.each(function(){if(c==="string"){var e,f=0,g=p(this),h=b,i=a.split(s);while(e=i[f++])h=d?h:!g.hasClass(e),g[h?"addClass":"removeClass"](e)}else if(c==="undefined"||c==="boolean")this.className&&p._data(this,"__className__",this.className),this.className=this.className||a===!1?"":p._data(this,"__className__")||""})},hasClass:function(a){var b=" "+a+" ",c=0,d=this.length;for(;c<d;c++)if(this[c].nodeType===1&&(" "+this[c].className+" ").replace(O," ").indexOf(b)>=0)return!0;return!1},val:function(a){var c,d,e,f=this[0];if(!arguments.length){if(f)return c=p.valHooks[f.type]||p.valHooks[f.nodeName.toLowerCase()],c&&"get"in c&&(d=c.get(f,"value"))!==b?d:(d=f.value,typeof d=="string"?d.replace(P,""):d==null?"":d);return}return e=p.isFunction(a),this.each(function(d){var f,g=p(this);if(this.nodeType!==1)return;e?f=a.call(this,d,g.val()):f=a,f==
 null?f="":typeof f=="number"?f+="":p.isArray(f)&&(f=p.map(f,function(a){return a==null?"":a+""})),c=p.valHooks[this.type]||p.valHooks[this.nodeName.toLowerCase()];if(!c||!("set"in c)||c.set(this,f,"value")===b)this.value=f})}}),p.extend({valHooks:{option:{get:function(a){var b=a.attributes.value;return!b||b.specified?a.value:a.text}},select:{get:function(a){var b,c,d,e,f=a.selectedIndex,g=[],h=a.options,i=a.type==="select-one";if(f<0)return null;c=i?f:0,d=i?f+1:h.length;for(;c<d;c++){e=h[c];if(e.selected&&(p.support.optDisabled?!e.disabled:e.getAttribute("disabled")===null)&&(!e.parentNode.disabled||!p.nodeName(e.parentNode,"optgroup"))){b=p(e).val();if(i)return b;g.push(b)}}return i&&!g.length&&h.length?p(h[f]).val():g},set:function(a,b){var c=p.makeArray(b);return p(a).find("option").each(function(){this.selected=p.inArray(p(this).val(),c)>=0}),c.length||(a.selectedIndex=-1),c}}},attrFn:{},attr:function(a,c,d,e){var f,g,h,i=a.nodeType;if(!a||i===3||i===8||i===2)return;if(e&&p.isFu
 nction(p.fn[c]))return p(a)[c](d);if(typeof a.getAttribute=="undefined")return p.prop(a,c,d);h=i!==1||!p.isXMLDoc(a),h&&(c=c.toLowerCase(),g=p.attrHooks[c]||(T.test(c)?M:L));if(d!==b){if(d===null){p.removeAttr(a,c);return}return g&&"set"in g&&h&&(f=g.set(a,d,c))!==b?f:(a.setAttribute(c,d+""),d)}return g&&"get"in g&&h&&(f=g.get(a,c))!==null?f:(f=a.getAttribute(c),f===null?b:f)},removeAttr:function(a,b){var c,d,e,f,g=0;if(b&&a.nodeType===1){d=b.split(s);for(;g<d.length;g++)e=d[g],e&&(c=p.propFix[e]||e,f=T.test(e),f||p.attr(a,e,""),a.removeAttribute(U?e:c),f&&c in a&&(a[c]=!1))}},attrHooks:{type:{set:function(a,b){if(Q.test(a.nodeName)&&a.parentNode)p.error("type property can't be changed");else if(!p.support.radioValue&&b==="radio"&&p.nodeName(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}},value:{get:function(a,b){return L&&p.nodeName(a,"button")?L.get(a,b):b in a?a.value:null},set:function(a,b,c){if(L&&p.nodeName(a,"button"))return L.set(a,b,c);a.value=
 b}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(a,c,d){var e,f,g,h=a.nodeType;if(!a||h===3||h===8||h===2)return;return g=h!==1||!p.isXMLDoc(a),g&&(c=p.propFix[c]||c,f=p.propHooks[c]),d!==b?f&&"set"in f&&(e=f.set(a,d,c))!==b?e:a[c]=d:f&&"get"in f&&(e=f.get(a,c))!==null?e:a[c]},propHooks:{tabIndex:{get:function(a){var c=a.getAttributeNode("tabindex");return c&&c.specified?parseInt(c.value,10):R.test(a.nodeName)||S.test(a.nodeName)&&a.href?0:b}}}}),M={get:function(a,c){var d,e=p.prop(a,c);return e===!0||typeof e!="boolean"&&(d=a.getAttributeNode(c))&&d.nodeValue!==!1?c.toLowerCase():b},set:function(a,b,c){var d;return b===!1?p.removeAttr(a,c):(d=p.propFix[c]||c,d in a&&(a[d]=!0),a.setAttribute(c,c.toLowerCase())),c}},U||(N={name:!0,id:!0,coords:!0
 },L=p.valHooks.button={get:function(a,c){var d;return d=a.getAttributeNode(c),d&&(N[c]?d.value!=="":d.specified)?d.value:b},set:function(a,b,c){var d=a.getAttributeNode(c);return d||(d=e.createAttribute(c),a.setAttributeNode(d)),d.value=b+""}},p.each(["width","height"],function(a,b){p.attrHooks[b]=p.extend(p.attrHooks[b],{set:function(a,c){if(c==="")return a.setAttribute(b,"auto"),c}})}),p.attrHooks.contenteditable={get:L.get,set:function(a,b,c){b===""&&(b="false"),L.set(a,b,c)}}),p.support.hrefNormalized||p.each(["href","src","width","height"],function(a,c){p.attrHooks[c]=p.extend(p.attrHooks[c],{get:function(a){var d=a.getAttribute(c,2);return d===null?b:d}})}),p.support.style||(p.attrHooks.style={get:function(a){return a.style.cssText.toLowerCase()||b},set:function(a,b){return a.style.cssText=b+""}}),p.support.optSelected||(p.propHooks.selected=p.extend(p.propHooks.selected,{get:function(a){var b=a.parentNode;return b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex),nu
 ll}})),p.support.enctype||(p.propFix.enctype="encoding"),p.support.checkOn||p.each(["radio","checkbox"],function(){p.valHooks[this]={get:function(a){return a.getAttribute("value")===null?"on":a.value}}}),p.each(["radio","checkbox"],function(){p.valHooks[this]=p.extend(p.valHooks[this],{set:function(a,b){if(p.isArray(b))return a.checked=p.inArray(p(a).val(),b)>=0}})});var V=/^(?:textarea|input|select)$/i,W=/^([^\.]*|)(?:\.(.+)|)$/,X=/(?:^|\s)hover(\.\S+|)\b/,Y=/^key/,Z=/^(?:mouse|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=function(a){return p.event.special.hover?a:a.replace(X,"mouseenter$1 mouseleave$1")};p.event={add:function(a,c,d,e,f){var g,h,i,j,k,l,m,n,o,q,r;if(a.nodeType===3||a.nodeType===8||!c||!d||!(g=p._data(a)))return;d.handler&&(o=d,d=o.handler,f=o.selector),d.guid||(d.guid=p.guid++),i=g.events,i||(g.events=i={}),h=g.handle,h||(g.handle=h=function(a){return typeof p!="undefined"&&(!a||p.event.triggered!==a.type)?p.event.dispatch.apply(h.elem,arguments):b},h.
 elem=a),c=p.trim(_(c)).split(" ");for(j=0;j<c.length;j++){k=W.exec(c[j])||[],l=k[1],m=(k[2]||"").split(".").sort(),r=p.event.special[l]||{},l=(f?r.delegateType:r.bindType)||l,r=p.event.special[l]||{},n=p.extend({type:l,origType:k[1],data:e,handler:d,guid:d.guid,selector:f,needsContext:f&&p.expr.match.needsContext.test(f),namespace:m.join(".")},o),q=i[l];if(!q){q=i[l]=[],q.delegateCount=0;if(!r.setup||r.setup.call(a,e,m,h)===!1)a.addEventListener?a.addEventListener(l,h,!1):a.attachEvent&&a.attachEvent("on"+l,h)}r.add&&(r.add.call(a,n),n.handler.guid||(n.handler.guid=d.guid)),f?q.splice(q.delegateCount++,0,n):q.push(n),p.event.global[l]=!0}a=null},global:{},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,q,r=p.hasData(a)&&p._data(a);if(!r||!(m=r.events))return;b=p.trim(_(b||"")).split(" ");for(f=0;f<b.length;f++){g=W.exec(b[f])||[],h=i=g[1],j=g[2];if(!h){for(h in m)p.event.remove(a,h+b[f],c,d,!0);continue}n=p.event.special[h]||{},h=(d?n.delegateType:n.bindType)||h,o=m[h]||[],k=o.le
 ngth,j=j?new RegExp("(^|\\.)"+j.split(".").sort().join("\\.(?:.*\\.|)")+"(\\.|$)"):null;for(l=0;l<o.length;l++)q=o[l],(e||i===q.origType)&&(!c||c.guid===q.guid)&&(!j||j.test(q.namespace))&&(!d||d===q.selector||d==="**"&&q.selector)&&(o.splice(l--,1),q.selector&&o.delegateCount--,n.remove&&n.remove.call(a,q));o.length===0&&k!==o.length&&((!n.teardown||n.teardown.call(a,j,r.handle)===!1)&&p.removeEvent(a,h,r.handle),delete m[h])}p.isEmptyObject(m)&&(delete r.handle,p.removeData(a,"events",!0))},customEvent:{getData:!0,setData:!0,changeData:!0},trigger:function(c,d,f,g){if(!f||f.nodeType!==3&&f.nodeType!==8){var h,i,j,k,l,m,n,o,q,r,s=c.type||c,t=[];if($.test(s+p.event.triggered))return;s.indexOf("!")>=0&&(s=s.slice(0,-1),i=!0),s.indexOf(".")>=0&&(t=s.split("."),s=t.shift(),t.sort());if((!f||p.event.customEvent[s])&&!p.event.global[s])return;c=typeof c=="object"?c[p.expando]?c:new p.Event(s,c):new p.Event(s),c.type=s,c.isTrigger=!0,c.exclusive=i,c.namespace=t.join("."),c.namespace_re=c.
 namespace?new RegExp("(^|\\.)"+t.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,m=s.indexOf(":")<0?"on"+s:"";if(!f){h=p.cache;for(j in h)h[j].events&&h[j].events[s]&&p.event.trigger(c,d,h[j].handle.elem,!0);return}c.result=b,c.target||(c.target=f),d=d!=null?p.makeArray(d):[],d.unshift(c),n=p.event.special[s]||{};if(n.trigger&&n.trigger.apply(f,d)===!1)return;q=[[f,n.bindType||s]];if(!g&&!n.noBubble&&!p.isWindow(f)){r=n.delegateType||s,k=$.test(r+s)?f:f.parentNode;for(l=f;k;k=k.parentNode)q.push([k,r]),l=k;l===(f.ownerDocument||e)&&q.push([l.defaultView||l.parentWindow||a,r])}for(j=0;j<q.length&&!c.isPropagationStopped();j++)k=q[j][0],c.type=q[j][1],o=(p._data(k,"events")||{})[c.type]&&p._data(k,"handle"),o&&o.apply(k,d),o=m&&k[m],o&&p.acceptData(k)&&o.apply&&o.apply(k,d)===!1&&c.preventDefault();return c.type=s,!g&&!c.isDefaultPrevented()&&(!n._default||n._default.apply(f.ownerDocument,d)===!1)&&(s!=="click"||!p.nodeName(f,"a"))&&p.acceptData(f)&&m&&f[s]&&(s!=="focus"&&s!=="blur"||c.target.o
 ffsetWidth!==0)&&!p.isWindow(f)&&(l=f[m],l&&(f[m]=null),p.event.triggered=s,f[s](),p.event.triggered=b,l&&(f[m]=l)),c.result}return},dispatch:function(c){c=p.event.fix(c||a.event);var d,e,f,g,h,i,j,l,m,n,o=(p._data(this,"events")||{})[c.type]||[],q=o.delegateCount,r=k.call(arguments),s=!c.exclusive&&!c.namespace,t=p.event.special[c.type]||{},u=[];r[0]=c,c.delegateTarget=this;if(t.preDispatch&&t.preDispatch.call(this,c)===!1)return;if(q&&(!c.button||c.type!=="click"))for(f=c.target;f!=this;f=f.parentNode||this)if(f.disabled!==!0||c.type!=="click"){h={},j=[];for(d=0;d<q;d++)l=o[d],m=l.selector,h[m]===b&&(h[m]=l.needsContext?p(m,this).index(f)>=0:p.find(m,this,null,[f]).length),h[m]&&j.push(l);j.length&&u.push({elem:f,matches:j})}o.length>q&&u.push({elem:this,matches:o.slice(q)});for(d=0;d<u.length&&!c.isPropagationStopped();d++){i=u[d],c.currentTarget=i.elem;for(e=0;e<i.matches.length&&!c.isImmediatePropagationStopped();e++){l=i.matches[e];if(s||!c.namespace&&!l.namespace||c.namespace
 _re&&c.namespace_re.test(l.namespace))c.data=l.data,c.handleObj=l,g=((p.event.special[l.origType]||{}).handle||l.handler).apply(i.elem,r),g!==b&&(c.result=g,g===!1&&(c.preventDefault(),c.stopPropagation()))}}return t.postDispatch&&t.postDispatch.call(this,c),c.result},props:"attrChange attrName relatedNode srcElement altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(a,b){return a.which==null&&(a.which=b.charCode!=null?b.charCode:b.keyCode),a}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(a,c){var d,f,g,h=c.button,i=c.fromElement;return a.pageX==null&&c.clientX!=null&&(d=a.target.ownerDocument||e,f=d.documentElement,g=d.body,a.pageX=c.clientX+(f&&f.scrollLeft||g&&g.scrollLeft||0)-(f&&f.clientLeft||g&&g.clientLeft||0),a.pageY=c
 .clientY+(f&&f.scrollTop||g&&g.scrollTop||0)-(f&&f.clientTop||g&&g.clientTop||0)),!a.relatedTarget&&i&&(a.relatedTarget=i===a.target?c.toElement:i),!a.which&&h!==b&&(a.which=h&1?1:h&2?3:h&4?2:0),a}},fix:function(a){if(a[p.expando])return a;var b,c,d=a,f=p.event.fixHooks[a.type]||{},g=f.props?this.props.concat(f.props):this.props;a=p.Event(d);for(b=g.length;b;)c=g[--b],a[c]=d[c];return a.target||(a.target=d.srcElement||e),a.target.nodeType===3&&(a.target=a.target.parentNode),a.metaKey=!!a.metaKey,f.filter?f.filter(a,d):a},special:{load:{noBubble:!0},focus:{delegateType:"focusin"},blur:{delegateType:"focusout"},beforeunload:{setup:function(a,b,c){p.isWindow(this)&&(this.onbeforeunload=c)},teardown:function(a,b){this.onbeforeunload===b&&(this.onbeforeunload=null)}}},simulate:function(a,b,c,d){var e=p.extend(new p.Event,c,{type:a,isSimulated:!0,originalEvent:{}});d?p.event.trigger(e,null,b):p.event.dispatch.call(b,e),e.isDefaultPrevented()&&c.preventDefault()}},p.event.handle=p.event.di
 spatch,p.removeEvent=e.removeEventListener?function(a,b,c){a.removeEventListener&&a.removeEventListener(b,c,!1)}:function(a,b,c){var d="on"+b;a.detachEvent&&(typeof a[d]=="undefined"&&(a[d]=null),a.detachEvent(d,c))},p.Event=function(a,b){if(this instanceof p.Event)a&&a.type?(this.originalEvent=a,this.type=a.type,this.isDefaultPrevented=a.defaultPrevented||a.returnValue===!1||a.getPreventDefault&&a.getPreventDefault()?bb:ba):this.type=a,b&&p.extend(this,b),this.timeStamp=a&&a.timeStamp||p.now(),this[p.expando]=!0;else return new p.Event(a,b)},p.Event.prototype={preventDefault:function(){this.isDefaultPrevented=bb;var a=this.originalEvent;if(!a)return;a.preventDefault?a.preventDefault():a.returnValue=!1},stopPropagation:function(){this.isPropagationStopped=bb;var a=this.originalEvent;if(!a)return;a.stopPropagation&&a.stopPropagation(),a.cancelBubble=!0},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=bb,this.stopPropagation()},isDefaultPrevented:ba,isPropagatio
 nStopped:ba,isImmediatePropagationStopped:ba},p.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(a,b){p.event.special[a]={delegateType:b,bindType:b,handle:function(a){var c,d=this,e=a.relatedTarget,f=a.handleObj,g=f.selector;if(!e||e!==d&&!p.contains(d,e))a.type=f.origType,c=f.handler.apply(this,arguments),a.type=b;return c}}}),p.support.submitBubbles||(p.event.special.submit={setup:function(){if(p.nodeName(this,"form"))return!1;p.event.add(this,"click._submit keypress._submit",function(a){var c=a.target,d=p.nodeName(c,"input")||p.nodeName(c,"button")?c.form:b;d&&!p._data(d,"_submit_attached")&&(p.event.add(d,"submit._submit",function(a){a._submit_bubble=!0}),p._data(d,"_submit_attached",!0))})},postDispatch:function(a){a._submit_bubble&&(delete a._submit_bubble,this.parentNode&&!a.isTrigger&&p.event.simulate("submit",this.parentNode,a,!0))},teardown:function(){if(p.nodeName(this,"form"))return!1;p.event.remove(this,"._submit")}}),p.support.changeBubbles||(p.event.specia
 l.change={setup:function(){if(V.test(this.nodeName)){if(this.type==="checkbox"||this.type==="radio")p.event.add(this,"propertychange._change",function(a){a.originalEvent.propertyName==="checked"&&(this._just_changed=!0)}),p.event.add(this,"click._change",function(a){this._just_changed&&!a.isTrigger&&(this._just_changed=!1),p.event.simulate("change",this,a,!0)});return!1}p.event.add(this,"beforeactivate._change",function(a){var b=a.target;V.test(b.nodeName)&&!p._data(b,"_change_attached")&&(p.event.add(b,"change._change",function(a){this.parentNode&&!a.isSimulated&&!a.isTrigger&&p.event.simulate("change",this.parentNode,a,!0)}),p._data(b,"_change_attached",!0))})},handle:function(a){var b=a.target;if(this!==b||a.isSimulated||a.isTrigger||b.type!=="radio"&&b.type!=="checkbox")return a.handleObj.handler.apply(this,arguments)},teardown:function(){return p.event.remove(this,"._change"),!V.test(this.nodeName)}}),p.support.focusinBubbles||p.each({focus:"focusin",blur:"focusout"},function(a
 ,b){var c=0,d=function(a){p.event.simulate(b,a.target,p.event.fix(a),!0)};p.event.special[b]={setup:function(){c++===0&&e.addEventListener(a,d,!0)},teardown:function(){--c===0&&e.removeEventListener(a,d,!0)}}}),p.fn.extend({on:function(a,c,d,e,f){var g,h;if(typeof a=="object"){typeof c!="string"&&(d=d||c,c=b);for(h in a)this.on(h,c,d,a[h],f);return this}d==null&&e==null?(e=c,d=c=b):e==null&&(typeof c=="string"?(e=d,d=b):(e=d,d=c,c=b));if(e===!1)e=ba;else if(!e)return this;return f===1&&(g=e,e=function(a){return p().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=p.guid++)),this.each(function(){p.event.add(this,a,e,d,c)})},one:function(a,b,c,d){return this.on(a,b,c,d,1)},off:function(a,c,d){var e,f;if(a&&a.preventDefault&&a.handleObj)return e=a.handleObj,p(a.delegateTarget).off(e.namespace?e.origType+"."+e.namespace:e.origType,e.selector,e.handler),this;if(typeof a=="object"){for(f in a)this.off(f,c,a[f]);return this}if(c===!1||typeof c=="function")d=c,c=b;return d===!1&&(d=ba
 ),this.each(function(){p.event.remove(this,a,d,c)})},bind:function(a,b,c){return this.on(a,null,b,c)},unbind:function(a,b){return this.off(a,null,b)},live:function(a,b,c){return p(this.context).on(a,this.selector,b,c),this},die:function(a,b){return p(this.context).off(a,this.selector||"**",b),this},delegate:function(a,b,c,d){return this.on(b,a,c,d)},undelegate:function(a,b,c){return arguments.length===1?this.off(a,"**"):this.off(b,a||"**",c)},trigger:function(a,b){return this.each(function(){p.event.trigger(a,b,this)})},triggerHandler:function(a,b){if(this[0])return p.event.trigger(a,b,this[0],!0)},toggle:function(a){var b=arguments,c=a.guid||p.guid++,d=0,e=function(c){var e=(p._data(this,"lastToggle"+a.guid)||0)%d;return p._data(this,"lastToggle"+a.guid,e+1),c.preventDefault(),b[e].apply(this,arguments)||!1};e.guid=c;while(d<b.length)b[d++].guid=c;return this.click(e)},hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)}}),p.each("blur focus focusin focusout load resize 
 scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(a,b){p.fn[b]=function(a,c){return c==null&&(c=a,a=null),arguments.length>0?this.on(b,null,a,c):this.trigger(b)},Y.test(b)&&(p.event.fixHooks[b]=p.event.keyHooks),Z.test(b)&&(p.event.fixHooks[b]=p.event.mouseHooks)}),function(a,b){function bc(a,b,c,d){c=c||[],b=b||r;var e,f,i,j,k=b.nodeType;if(!a||typeof a!="string")return c;if(k!==1&&k!==9)return[];i=g(b);if(!i&&!d)if(e=P.exec(a))if(j=e[1]){if(k===9){f=b.getElementById(j);if(!f||!f.parentNode)return c;if(f.id===j)return c.push(f),c}else if(b.ownerDocument&&(f=b.ownerDocument.getElementById(j))&&h(b,f)&&f.id===j)return c.push(f),c}else{if(e[2])return w.apply(c,x.call(b.getElementsByTagName(a),0)),c;if((j=e[3])&&_&&b.getElementsByClassName)return w.apply(c,x.call(b.getElementsByClassName(j),0)),c}return bp(a.replace(L,"$1"),b,c,d,i)}function bd(a){return fu
 nction(b){var c=b.nodeName.toLowerCase();return c==="input"&&b.type===a}}function be(a){return function(b){var c=b.nodeName.toLowerCase();return(c==="input"||c==="button")&&b.type===a}}function bf(a){return z(function(b){return b=+b,z(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function bg(a,b,c){if(a===b)return c;var d=a.nextSibling;while(d){if(d===b)return-1;d=d.nextSibling}return 1}function bh(a,b){var c,d,f,g,h,i,j,k=C[o][a];if(k)return b?0:k.slice(0);h=a,i=[],j=e.preFilter;while(h){if(!c||(d=M.exec(h)))d&&(h=h.slice(d[0].length)),i.push(f=[]);c=!1;if(d=N.exec(h))f.push(c=new q(d.shift())),h=h.slice(c.length),c.type=d[0].replace(L," ");for(g in e.filter)(d=W[g].exec(h))&&(!j[g]||(d=j[g](d,r,!0)))&&(f.push(c=new q(d.shift())),h=h.slice(c.length),c.type=g,c.matches=d);if(!c)break}return b?h.length:h?bc.error(a):C(a,i).slice(0)}function bi(a,b,d){var e=b.dir,f=d&&b.dir==="parentNode",g=u++;return b.first?function(b,c,d){while(b=b[e
 ])if(f||b.nodeType===1)return a(b,c,d)}:function(b,d,h){if(!h){var i,j=t+" "+g+" ",k=j+c;while(b=b[e])if(f||b.nodeType===1){if((i=b[o])===k)return b.sizset;if(typeof i=="string"&&i.indexOf(j)===0){if(b.sizset)return b}else{b[o]=k;if(a(b,d,h))return b.sizset=!0,b;b.sizset=!1}}}else while(b=b[e])if(f||b.nodeType===1)if(a(b,d,h))return b}}function bj(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function bk(a,b,c,d,e){var f,g=[],h=0,i=a.length,j=b!=null;for(;h<i;h++)if(f=a[h])if(!c||c(f,d,e))g.push(f),j&&b.push(h);return g}function bl(a,b,c,d,e,f){return d&&!d[o]&&(d=bl(d)),e&&!e[o]&&(e=bl(e,f)),z(function(f,g,h,i){if(f&&e)return;var j,k,l,m=[],n=[],o=g.length,p=f||bo(b||"*",h.nodeType?[h]:h,[],f),q=a&&(f||!b)?bk(p,m,a,h,i):p,r=c?e||(f?a:o||d)?[]:g:q;c&&c(q,r,h,i);if(d){l=bk(r,n),d(l,[],h,i),j=l.length;while(j--)if(k=l[j])r[n[j]]=!(q[n[j]]=k)}if(f){j=a&&r.length;while(j--)if(k=r[j])f[m[j]]=!(g[m[j]]=k)}else r=bk(r===g?r.splice(o,r
 .length):r),e?e(null,g,r,i):w.apply(g,r)})}function bm(a){var b,c,d,f=a.length,g=e.relative[a[0].type],h=g||e.relative[" "],i=g?1:0,j=bi(function(a){return a===b},h,!0),k=bi(function(a){return y.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==l)||((b=c).nodeType?j(a,c,d):k(a,c,d))}];for(;i<f;i++)if(c=e.relative[a[i].type])m=[bi(bj(m),c)];else{c=e.filter[a[i].type].apply(null,a[i].matches);if(c[o]){d=++i;for(;d<f;d++)if(e.relative[a[d].type])break;return bl(i>1&&bj(m),i>1&&a.slice(0,i-1).join("").replace(L,"$1"),c,i<d&&bm(a.slice(i,d)),d<f&&bm(a=a.slice(d)),d<f&&a.join(""))}m.push(c)}return bj(m)}function bn(a,b){var d=b.length>0,f=a.length>0,g=function(h,i,j,k,m){var n,o,p,q=[],s=0,u="0",x=h&&[],y=m!=null,z=l,A=h||f&&e.find.TAG("*",m&&i.parentNode||i),B=t+=z==null?1:Math.E;y&&(l=i!==r&&i,c=g.el);for(;(n=A[u])!=null;u++){if(f&&n){for(o=0;p=a[o];o++)if(p(n,i,j)){k.push(n);break}y&&(t=B,c=++g.el)}d&&((n=!p&&n)&&s--,h&&x.push(n))}s+=u;if(d&&u!==s){for(o=0;p=b[o];o++)p(x,q,i,j);
 if(h){if(s>0)while(u--)!x[u]&&!q[u]&&(q[u]=v.call(k));q=bk(q)}w.apply(k,q),y&&!h&&q.length>0&&s+b.length>1&&bc.uniqueSort(k)}return y&&(t=B,l=z),x};return g.el=0,d?z(g):g}function bo(a,b,c,d){var e=0,f=b.length;for(;e<f;e++)bc(a,b[e],c,d);return c}function bp(a,b,c,d,f){var g,h,j,k,l,m=bh(a),n=m.length;if(!d&&m.length===1){h=m[0]=m[0].slice(0);if(h.length>2&&(j=h[0]).type==="ID"&&b.nodeType===9&&!f&&e.relative[h[1].type]){b=e.find.ID(j.matches[0].replace(V,""),b,f)[0];if(!b)return c;a=a.slice(h.shift().length)}for(g=W.POS.test(a)?-1:h.length-1;g>=0;g--){j=h[g];if(e.relative[k=j.type])break;if(l=e.find[k])if(d=l(j.matches[0].replace(V,""),R.test(h[0].type)&&b.parentNode||b,f)){h.splice(g,1),a=d.length&&h.join("");if(!a)return w.apply(c,x.call(d,0)),c;break}}}return i(a,m)(d,b,f,c,R.test(a)),c}function bq(){}var c,d,e,f,g,h,i,j,k,l,m=!0,n="undefined",o=("sizcache"+Math.random()).replace(".",""),q=String,r=a.document,s=r.documentElement,t=0,u=0,v=[].pop,w=[].push,x=[].slice,y=[].indexO
 f||function(a){var b=0,c=this.length;for(;b<c;b++)if(this[b]===a)return b;return-1},z=function(a,b){return a[o]=b==null||b,a},A=function(){var a={},b=[];return z(function(c,d){return b.push(c)>e.cacheLength&&delete a[b.shift()],a[c]=d},a)},B=A(),C=A(),D=A(),E="[\\x20\\t\\r\\n\\f]",F="(?:\\\\.|[-\\w]|[^\\x00-\\xa0])+",G=F.replace("w","w#"),H="([*^$|!~]?=)",I="\\["+E+"*("+F+")"+E+"*(?:"+H+E+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+G+")|)|)"+E+"*\\]",J=":("+F+")(?:\\((?:(['\"])((?:\\\\.|[^\\\\])*?)\\2|([^()[\\]]*|(?:(?:"+I+")|[^:]|\\\\.)*|.*))\\)|)",K=":(even|odd|eq|gt|lt|nth|first|last)(?:\\("+E+"*((?:-\\d)?\\d*)"+E+"*\\)|)(?=[^-]|$)",L=new RegExp("^"+E+"+|((?:^|[^\\\\])(?:\\\\.)*)"+E+"+$","g"),M=new RegExp("^"+E+"*,"+E+"*"),N=new RegExp("^"+E+"*([\\x20\\t\\r\\n\\f>+~])"+E+"*"),O=new RegExp(J),P=/^(?:#([\w\-]+)|(\w+)|\.([\w\-]+))$/,Q=/^:not/,R=/[\x20\t\r\n\f]*[+~]/,S=/:not\($/,T=/h\d/i,U=/input|select|textarea|button/i,V=/\\(?!\\)/g,W={ID:new RegExp("^#("+F+")"),CLASS:new RegExp("^\\.(
 "+F+")"),NAME:new RegExp("^\\[name=['\"]?("+F+")['\"]?\\]"),TAG:new RegExp("^("+F.replace("w","w*")+")"),ATTR:new RegExp("^"+I),PSEUDO:new RegExp("^"+J),POS:new RegExp(K,"i"),CHILD:new RegExp("^:(only|nth|first|last)-child(?:\\("+E+"*(even|odd|(([+-]|)(\\d*)n|)"+E+"*(?:([+-]|)"+E+"*(\\d+)|))"+E+"*\\)|)","i"),needsContext:new RegExp("^"+E+"*[>+~]|"+K,"i")},X=function(a){var b=r.createElement("div");try{return a(b)}catch(c){return!1}finally{b=null}},Y=X(function(a){return a.appendChild(r.createComment("")),!a.getElementsByTagName("*").length}),Z=X(function(a){return a.innerHTML="<a href='#'></a>",a.firstChild&&typeof a.firstChild.getAttribute!==n&&a.firstChild.getAttribute("href")==="#"}),$=X(function(a){a.innerHTML="<select></select>";var b=typeof a.lastChild.getAttribute("multiple");return b!=="boolean"&&b!=="string"}),_=X(function(a){return a.innerHTML="<div class='hidden e'></div><div class='hidden'></div>",!a.getElementsByClassName||!a.getElementsByClassName("e").length?!1:(a.las
 tChild.className="e",a.getElementsByClassName("e").length===2)}),ba=X(function(a){a.id=o+0,a.innerHTML="<a name='"+o+"'></a><div name='"+o+"'></div>",s.insertBefore(a,s.firstChild);var b=r.getElementsByName&&r.getElementsByName(o).length===2+r.getElementsByName(o+0).length;return d=!r.getElementById(o),s.removeChild(a),b});try{x.call(s.childNodes,0)[0].nodeType}catch(bb){x=function(a){var b,c=[];for(;b=this[a];a++)c.push(b);return c}}bc.matches=function(a,b){return bc(a,null,null,b)},bc.matchesSelector=function(a,b){return bc(b,null,null,[a]).length>0},f=bc.getText=function(a){var b,c="",d=0,e=a.nodeType;if(e){if(e===1||e===9||e===11){if(typeof a.textContent=="string")return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=f(a)}else if(e===3||e===4)return a.nodeValue}else for(;b=a[d];d++)c+=f(b);return c},g=bc.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?b.nodeName!=="HTML":!1},h=bc.contains=s.contains?function(a,b){var c=a.nodeType===9?a.documentEle
 ment:a,d=b&&b.parentNode;return a===d||!!(d&&d.nodeType===1&&c.contains&&c.contains(d))}:s.compareDocumentPosition?function(a,b){return b&&!!(a.compareDocumentPosition(b)&16)}:function(a,b){while(b=b.parentNode)if(b===a)return!0;return!1},bc.attr=function(a,b){var c,d=g(a);return d||(b=b.toLowerCase()),(c=e.attrHandle[b])?c(a):d||$?a.getAttribute(b):(c=a.getAttributeNode(b),c?typeof a[b]=="boolean"?a[b]?b:null:c.specified?c.value:null:null)},e=bc.selectors={cacheLength:50,createPseudo:z,match:W,attrHandle:Z?{}:{href:function(a){return a.getAttribute("href",2)},type:function(a){return a.getAttribute("type")}},find:{ID:d?function(a,b,c){if(typeof b.getElementById!==n&&!c){var d=b.getElementById(a);return d&&d.parentNode?[d]:[]}}:function(a,c,d){if(typeof c.getElementById!==n&&!d){var e=c.getElementById(a);return e?e.id===a||typeof e.getAttributeNode!==n&&e.getAttributeNode("id").value===a?[e]:b:[]}},TAG:Y?function(a,b){if(typeof b.getElementsByTagName!==n)return b.getElementsByTagName
 (a)}:function(a,b){var c=b.getElementsByTagName(a);if(a==="*"){var d,e=[],f=0;for(;d=c[f];f++)d.nodeType===1&&e.push(d);return e}return c},NAME:ba&&function(a,b){if(typeof b.getElementsByName!==n)return b.getElementsByName(name)},CLASS:_&&function(a,b,c){if(typeof b.getElementsByClassName!==n&&!c)return b.getElementsByClassName(a)}},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(V,""),a[3]=(a[4]||a[5]||"").replace(V,""),a[2]==="~="&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),a[1]==="nth"?(a[2]||bc.error(a[0]),a[3]=+(a[3]?a[4]+(a[5]||1):2*(a[2]==="even"||a[2]==="odd")),a[4]=+(a[6]+a[7]||a[2]==="odd")):a[2]&&bc.error(a[0]),a},PSEUDO:function(a){var b,c;if(W.CHILD.test(a[0]))return null;if(a[3])a[2]=a[3];else if(b=a[4])O.test(b)&&(c=bh(b,!0))&&(c=b.indexOf(")",b.length-c)-b.length)&&(b=b.slice(0,c),a[0]=a[0].sli
 ce(0,c)),a[2]=b;return a.slice(0,3)}},filter:{ID:d?function(a){return a=a.replace(V,""),function(b){return b.getAttribute("id")===a}}:function(a){return a=a.replace(V,""),function(b){var c=typeof b.getAttributeNode!==n&&b.getAttributeNode("id");return c&&c.value===a}},TAG:function(a){return a==="*"?function(){return!0}:(a=a.replace(V,"").toLowerCase(),function(b){return b.nodeName&&b.nodeName.toLowerCase()===a})},CLASS:function(a){var b=B[o][a];return b||(b=B(a,new RegExp("(^|"+E+")"+a+"("+E+"|$)"))),function(a){return b.test(a.className||typeof a.getAttribute!==n&&a.getAttribute("class")||"")}},ATTR:function(a,b,c){return function(d,e){var f=bc.attr(d,a);return f==null?b==="!=":b?(f+="",b==="="?f===c:b==="!="?f!==c:b==="^="?c&&f.indexOf(c)===0:b==="*="?c&&f.indexOf(c)>-1:b==="$="?c&&f.substr(f.length-c.length)===c:b==="~="?(" "+f+" ").indexOf(c)>-1:b==="|="?f===c||f.substr(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d){return a==="nth"?function(a){var b,e,f=a.parentNode;if(
 c===1&&d===0)return!0;if(f){e=0;for(b=f.firstChild;b;b=b.nextSibling)if(b.nodeType===1){e++;if(a===b)break}}return e-=d,e===c||e%c===0&&e/c>=0}:function(b){var c=b;switch(a){case"only":case"first":while(c=c.previousSibling)if(c.nodeType===1)return!1;if(a==="first")return!0;c=b;case"last":while(c=c.nextSibling)if(c.nodeType===1)return!1;return!0}}},PSEUDO:function(a,b){var c,d=e.pseudos[a]||e.setFilters[a.toLowerCase()]||bc.error("unsupported pseudo: "+a);return d[o]?d(b):d.length>1?(c=[a,a,"",b],e.setFilters.hasOwnProperty(a.toLowerCase())?z(function(a,c){var e,f=d(a,b),g=f.length;while(g--)e=y.call(a,f[g]),a[e]=!(c[e]=f[g])}):function(a){return d(a,0,c)}):d}},pseudos:{not:z(function(a){var b=[],c=[],d=i(a.replace(L,"$1"));return d[o]?z(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)if(f=g[h])a[h]=!(b[h]=f)}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:z(function(a){return function(b){return bc(a,b).length>0}}),contains:z(function(a){return function(b)
 {return(b.textContent||b.innerText||f(b)).indexOf(a)>-1}}),enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&!!a.checked||b==="option"&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},parent:function(a){return!e.pseudos.empty(a)},empty:function(a){var b;a=a.firstChild;while(a){if(a.nodeName>"@"||(b=a.nodeType)===3||b===4)return!1;a=a.nextSibling}return!0},header:function(a){return T.test(a.nodeName)},text:function(a){var b,c;return a.nodeName.toLowerCase()==="input"&&(b=a.type)==="text"&&((c=a.getAttribute("type"))==null||c.toLowerCase()===b)},radio:bd("radio"),checkbox:bd("checkbox"),file:bd("file"),password:bd("password"),image:bd("image"),submit:be("submit"),reset:be("reset"),button:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&a.type==="button"||b==="button"},input:function(a){return U.test(a.nodeName)}
 ,focus:function(a){var b=a.ownerDocument;return a===b.activeElement&&(!b.hasFocus||b.hasFocus())&&(!!a.type||!!a.href)},active:function(a){return a===a.ownerDocument.activeElement},first:bf(function(a,b,c){return[0]}),last:bf(function(a,b,c){return[b-1]}),eq:bf(function(a,b,c){return[c<0?c+b:c]}),even:bf(function(a,b,c){for(var d=0;d<b;d+=2)a.push(d);return a}),odd:bf(function(a,b,c){for(var d=1;d<b;d+=2)a.push(d);return a}),lt:bf(function(a,b,c){for(var d=c<0?c+b:c;--d>=0;)a.push(d);return a}),gt:bf(function(a,b,c){for(var d=c<0?c+b:c;++d<b;)a.push(d);return a})}},j=s.compareDocumentPosition?function(a,b){return a===b?(k=!0,0):(!a.compareDocumentPosition||!b.compareDocumentPosition?a.compareDocumentPosition:a.compareDocumentPosition(b)&4)?-1:1}:function(a,b){if(a===b)return k=!0,0;if(a.sourceIndex&&b.sourceIndex)return a.sourceIndex-b.sourceIndex;var c,d,e=[],f=[],g=a.parentNode,h=b.parentNode,i=g;if(g===h)return bg(a,b);if(!g)return-1;if(!h)return 1;while(i)e.unshift(i),i=i.parent
 Node;i=h;while(i)f.unshift(i),i=i.parentNode;c=e.length,d=f.length;for(var j=0;j<c&&j<d;j++)if(e[j]!==f[j])return bg(e[j],f[j]);return j===c?bg(a,f[j],-1):bg(e[j],b,1)},[0,0].sort(j),m=!k,bc.uniqueSort=function(a){var b,c=1;k=m,a.sort(j);if(k)for(;b=a[c];c++)b===a[c-1]&&a.splice(c--,1);return a},bc.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},i=bc.compile=function(a,b){var c,d=[],e=[],f=D[o][a];if(!f){b||(b=bh(a)),c=b.length;while(c--)f=bm(b[c]),f[o]?d.push(f):e.push(f);f=D(a,bn(e,d))}return f},r.querySelectorAll&&function(){var a,b=bp,c=/'|\\/g,d=/\=[\x20\t\r\n\f]*([^'"\]]*)[\x20\t\r\n\f]*\]/g,e=[":focus"],f=[":active",":focus"],h=s.matchesSelector||s.mozMatchesSelector||s.webkitMatchesSelector||s.oMatchesSelector||s.msMatchesSelector;X(function(a){a.innerHTML="<select><option selected=''></option></select>",a.querySelectorAll("[selected]").length||e.push("\\["+E+"*(?:checked|disabled|ismap|multiple|readonly|selected|value)"),a.querySelectorAll(":
 checked").length||e.push(":checked")}),X(function(a){a.innerHTML="<p test=''></p>",a.querySelectorAll("[test^='']").length&&e.push("[*^$]="+E+"*(?:\"\"|'')"),a.innerHTML="<input type='hidden'/>",a.querySelectorAll(":enabled").length||e.push(":enabled",":disabled")}),e=new RegExp(e.join("|")),bp=function(a,d,f,g,h){if(!g&&!h&&(!e||!e.test(a))){var i,j,k=!0,l=o,m=d,n=d.nodeType===9&&a;if(d.nodeType===1&&d.nodeName.toLowerCase()!=="object"){i=bh(a),(k=d.getAttribute("id"))?l=k.replace(c,"\\$&"):d.setAttribute("id",l),l="[id='"+l+"'] ",j=i.length;while(j--)i[j]=l+i[j].join("");m=R.test(a)&&d.parentNode||d,n=i.join(",")}if(n)try{return w.apply(f,x.call(m.querySelectorAll(n),0)),f}catch(p){}finally{k||d.removeAttribute("id")}}return b(a,d,f,g,h)},h&&(X(function(b){a=h.call(b,"div");try{h.call(b,"[test!='']:sizzle"),f.push("!=",J)}catch(c){}}),f=new RegExp(f.join("|")),bc.matchesSelector=function(b,c){c=c.replace(d,"='$1']");if(!g(b)&&!f.test(c)&&(!e||!e.test(c)))try{var i=h.call(b,c);if(i
 ||a||b.document&&b.document.nodeType!==11)return i}catch(j){}return bc(c,null,null,[b]).length>0})}(),e.pseudos.nth=e.pseudos.eq,e.filters=bq.prototype=e.pseudos,e.setFilters=new bq,bc.attr=p.attr,p.find=bc,p.expr=bc.selectors,p.expr[":"]=p.expr.pseudos,p.unique=bc.uniqueSort,p.text=bc.getText,p.isXMLDoc=bc.isXML,p.contains=bc.contains}(a);var bc=/Until$/,bd=/^(?:parents|prev(?:Until|All))/,be=/^.[^:#\[\.,]*$/,bf=p.expr.match.needsContext,bg={children:!0,contents:!0,next:!0,prev:!0};p.fn.extend({find:function(a){var b,c,d,e,f,g,h=this;if(typeof a!="string")return p(a).filter(function(){for(b=0,c=h.length;b<c;b++)if(p.contains(h[b],this))return!0});g=this.pushStack("","find",a);for(b=0,c=this.length;b<c;b++){d=g.length,p.find(a,this[b],g);if(b>0)for(e=d;e<g.length;e++)for(f=0;f<d;f++)if(g[f]===g[e]){g.splice(e--,1);break}}return g},has:function(a){var b,c=p(a,this),d=c.length;return this.filter(function(){for(b=0;b<d;b++)if(p.contains(this,c[b]))return!0})},not:function(a){return thi
 s.pushStack(bj(this,a,!1),"not",a)},filter:function(a){return this.pushStack(bj(this,a,!0),"filter",a)},is:function(a){return!!a&&(typeof a=="string"?bf.test(a)?p(a,this.context).index(this[0])>=0:p.filter(a,this).length>0:this.filter(a).length>0)},closest:function(a,b){var c,d=0,e=this.length,f=[],g=bf.test(a)||typeof a!="string"?p(a,b||this.context):0;for(;d<e;d++){c=this[d];while(c&&c.ownerDocument&&c!==b&&c.nodeType!==11){if(g?g.index(c)>-1:p.find.matchesSelector(c,a)){f.push(c);break}c=c.parentNode}}return f=f.length>1?p.unique(f):f,this.pushStack(f,"closest",a)},index:function(a){return a?typeof a=="string"?p.inArray(this[0],p(a)):p.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.prevAll().length:-1},add:function(a,b){var c=typeof a=="string"?p(a,b):p.makeArray(a&&a.nodeType?[a]:a),d=p.merge(this.get(),c);return this.pushStack(bh(c[0])||bh(d[0])?d:p.unique(d))},addBack:function(a){return this.add(a==null?this.prevObject:this.prevObject.filter(a))}}),p.fn.andSelf
 =p.fn.addBack,p.each({parent:function(a){var b=a.parentNode;return b&&b.nodeType!==11?b:null},parents:function(a){return p.dir(a,"parentNode")},parentsUntil:function(a,b,c){return p.dir(a,"parentNode",c)},next:function(a){return bi(a,"nextSibling")},prev:function(a){return bi(a,"previousSibling")},nextAll:function(a){return p.dir(a,"nextSibling")},prevAll:function(a){return p.dir(a,"previousSibling")},nextUntil:function(a,b,c){return p.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return p.dir(a,"previousSibling",c)},siblings:function(a){return p.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return p.sibling(a.firstChild)},contents:function(a){return p.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:p.merge([],a.childNodes)}},function(a,b){p.fn[a]=function(c,d){var e=p.map(this,b,c);return bc.test(a)||(d=c),d&&typeof d=="string"&&(e=p.filter(d,e)),e=this.length>1&&!bg[a]?p.unique(e):e,this.length>1&&bd.test(a)&&(e=e.reverse()),this.pushStack(e,a
 ,k.call(arguments).join(","))}}),p.extend({filter:function(a,b,c){return c&&(a=":not("+a+")"),b.length===1?p.find.matchesSelector(b[0],a)?[b[0]]:[]:p.find.matches(a,b)},dir:function(a,c,d){var e=[],f=a[c];while(f&&f.nodeType!==9&&(d===b||f.nodeType!==1||!p(f).is(d)))f.nodeType===1&&e.push(f),f=f[c];return e},sibling:function(a,b){var c=[];for(;a;a=a.nextSibling)a.nodeType===1&&a!==b&&c.push(a);return c}});var bl="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",bm=/ jQuery\d+="(?:null|\d+)"/g,bn=/^\s+/,bo=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bp=/<([\w:]+)/,bq=/<tbody/i,br=/<|&#?\w+;/,bs=/<(?:script|style|link)/i,bt=/<(?:script|object|embed|option|style)/i,bu=new RegExp("<(?:"+bl+")[\\s/>]","i"),bv=/^(?:checkbox|radio)$/,bw=/checked\s*(?:[^=]|=\s*.checked.)/i,bx=/\/(java|ecma)script/i,by=/^\s*<!(?:\[CDATA\[|\-\-)|[\]\-]{2}>\s*$/g,bz={option:[
 1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],area:[1,"<map>","</map>"],_default:[0,"",""]},bA=bk(e),bB=bA.appendChild(e.createElement("div"));bz.optgroup=bz.option,bz.tbody=bz.tfoot=bz.colgroup=bz.caption=bz.thead,bz.th=bz.td,p.support.htmlSerialize||(bz._default=[1,"X<div>","</div>"]),p.fn.extend({text:function(a){return p.access(this,function(a){return a===b?p.text(this):this.empty().append((this[0]&&this[0].ownerDocument||e).createTextNode(a))},null,a,arguments.length)},wrapAll:function(a){if(p.isFunction(a))return this.each(function(b){p(this).wrapAll(a.call(this,b))});if(this[0]){var b=p(a,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstChild&&a.firstChild.nodeType===1)a=a.firs
 tChild;return a}).append(this)}return this},wrapInner:function(a){return p.isFunction(a)?this.each(function(b){p(this).wrapInner(a.call(this,b))}):this.each(function(){var b=p(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=p.isFunction(a);return this.each(function(c){p(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(){return this.parent().each(function(){p.nodeName(this,"body")||p(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.appendChild(a)})},prepend:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.insertBefore(a,this.firstChild)})},before:function(){if(!bh(this[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(a,this),"before",this.selector)}},after:function(){if(!bh(thi
 s[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this.nextSibling)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(this,a),"after",this.selector)}},remove:function(a,b){var c,d=0;for(;(c=this[d])!=null;d++)if(!a||p.filter(a,[c]).length)!b&&c.nodeType===1&&(p.cleanData(c.getElementsByTagName("*")),p.cleanData([c])),c.parentNode&&c.parentNode.removeChild(c);return this},empty:function(){var a,b=0;for(;(a=this[b])!=null;b++){a.nodeType===1&&p.cleanData(a.getElementsByTagName("*"));while(a.firstChild)a.removeChild(a.firstChild)}return this},clone:function(a,b){return a=a==null?!1:a,b=b==null?a:b,this.map(function(){return p.clone(this,a,b)})},html:function(a){return p.access(this,function(a){var c=this[0]||{},d=0,e=this.length;if(a===b)return c.nodeType===1?c.innerHTML.replace(bm,""):b;if(typeof a=="string"&&!bs.test(a)&&(p.support.htmlSerialize||!bu.test(a))&&(p.support.leadingWhitespace||!bn.test(a))&&!bz[(bp.exec(a)||[""
 ,""])[1].toLowerCase()]){a=a.replace(bo,"<$1></$2>");try{for(;d<e;d++)c=this[d]||{},c.nodeType===1&&(p.cleanData(c.getElementsByTagName("*")),c.innerHTML=a);c=0}catch(f){}}c&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(a){return bh(this[0])?this.length?this.pushStack(p(p.isFunction(a)?a():a),"replaceWith",a):this:p.isFunction(a)?this.each(function(b){var c=p(this),d=c.html();c.replaceWith(a.call(this,b,d))}):(typeof a!="string"&&(a=p(a).detach()),this.each(function(){var b=this.nextSibling,c=this.parentNode;p(this).remove(),b?p(b).before(a):p(c).append(a)}))},detach:function(a){return this.remove(a,!0)},domManip:function(a,c,d){a=[].concat.apply([],a);var e,f,g,h,i=0,j=a[0],k=[],l=this.length;if(!p.support.checkClone&&l>1&&typeof j=="string"&&bw.test(j))return this.each(function(){p(this).domManip(a,c,d)});if(p.isFunction(j))return this.each(function(e){var f=p(this);a[0]=j.call(this,e,c?f.html():b),f.domManip(a,c,d)});if(this[0]){e=p.buildFragment(a,this,
 k),g=e.fragment,f=g.firstChild,g.childNodes.length===1&&(g=f);if(f){c=c&&p.nodeName(f,"tr");for(h=e.cacheable||l-1;i<l;i++)d.call(c&&p.nodeName(this[i],"table")?bC(this[i],"tbody"):this[i],i===h?g:p.clone(g,!0,!0))}g=f=null,k.length&&p.each(k,function(a,b){b.src?p.ajax?p.ajax({url:b.src,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0}):p.error("no ajax"):p.globalEval((b.text||b.textContent||b.innerHTML||"").replace(by,"")),b.parentNode&&b.parentNode.removeChild(b)})}return this}}),p.buildFragment=function(a,c,d){var f,g,h,i=a[0];return c=c||e,c=!c.nodeType&&c[0]||c,c=c.ownerDocument||c,a.length===1&&typeof i=="string"&&i.length<512&&c===e&&i.charAt(0)==="<"&&!bt.test(i)&&(p.support.checkClone||!bw.test(i))&&(p.support.html5Clone||!bu.test(i))&&(g=!0,f=p.fragments[i],h=f!==b),f||(f=c.createDocumentFragment(),p.clean(a,c,f,d),g&&(p.fragments[i]=h&&f)),{fragment:f,cacheable:g}},p.fragments={},p.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"a
 fter",replaceAll:"replaceWith"},function(a,b){p.fn[a]=function(c){var d,e=0,f=[],g=p(c),h=g.length,i=this.length===1&&this[0].parentNode;if((i==null||i&&i.nodeType===11&&i.childNodes.length===1)&&h===1)return g[b](this[0]),this;for(;e<h;e++)d=(e>0?this.clone(!0):this).get(),p(g[e])[b](d),f=f.concat(d);return this.pushStack(f,a,g.selector)}}),p.extend({clone:function(a,b,c){var d,e,f,g;p.support.html5Clone||p.isXMLDoc(a)||!bu.test("<"+a.nodeName+">")?g=a.cloneNode(!0):(bB.innerHTML=a.outerHTML,bB.removeChild(g=bB.firstChild));if((!p.support.noCloneEvent||!p.support.noCloneChecked)&&(a.nodeType===1||a.nodeType===11)&&!p.isXMLDoc(a)){bE(a,g),d=bF(a),e=bF(g);for(f=0;d[f];++f)e[f]&&bE(d[f],e[f])}if(b){bD(a,g);if(c){d=bF(a),e=bF(g);for(f=0;d[f];++f)bD(d[f],e[f])}}return d=e=null,g},clean:function(a,b,c,d){var f,g,h,i,j,k,l,m,n,o,q,r,s=b===e&&bA,t=[];if(!b||typeof b.createDocumentFragment=="undefined")b=e;for(f=0;(h=a[f])!=null;f++){typeof h=="number"&&(h+="");if(!h)continue;if(typeof h=="
 string")if(!br.test(h))h=b.createTextNode(h);else{s=s||bk(b),l=b.createElement("div"),s.appendChild(l),h=h.replace(bo,"<$1></$2>"),i=(bp.exec(h)||["",""])[1].toLowerCase(),j=bz[i]||bz._default,k=j[0],l.innerHTML=j[1]+h+j[2];while(k--)l=l.lastChild;if(!p.support.tbody){m=bq.test(h),n=i==="table"&&!m?l.firstChild&&l.firstChild.childNodes:j[1]==="<table>"&&!m?l.childNodes:[];for(g=n.length-1;g>=0;--g)p.nodeName(n[g],"tbody")&&!n[g].childNodes.length&&n[g].parentNode.removeChild(n[g])}!p.support.leadingWhitespace&&bn.test(h)&&l.insertBefore(b.createTextNode(bn.exec(h)[0]),l.firstChild),h=l.childNodes,l.parentNode.removeChild(l)}h.nodeType?t.push(h):p.merge(t,h)}l&&(h=l=s=null);if(!p.support.appendChecked)for(f=0;(h=t[f])!=null;f++)p.nodeName(h,"input")?bG(h):typeof h.getElementsByTagName!="undefined"&&p.grep(h.getElementsByTagName("input"),bG);if(c){q=function(a){if(!a.type||bx.test(a.type))return d?d.push(a.parentNode?a.parentNode.removeChild(a):a):c.appendChild(a)};for(f=0;(h=t[f])!=n
 ull;f++)if(!p.nodeName(h,"script")||!q(h))c.appendChild(h),typeof h.getElementsByTagName!="undefined"&&(r=p.grep(p.merge([],h.getElementsByTagName("script")),q),t.splice.apply(t,[f+1,0].concat(r)),f+=r.length)}return t},cleanData:function(a,b){var c,d,e,f,g=0,h=p.expando,i=p.cache,j=p.support.deleteExpando,k=p.event.special;for(;(e=a[g])!=null;g++)if(b||p.acceptData(e)){d=e[h],c=d&&i[d];if(c){if(c.events)for(f in c.events)k[f]?p.event.remove(e,f):p.removeEvent(e,f,c.handle);i[d]&&(delete i[d],j?delete e[h]:e.removeAttribute?e.removeAttribute(h):e[h]=null,p.deletedIds.push(d))}}}}),function(){var a,b;p.uaMatch=function(a){a=a.toLowerCase();var b=/(chrome)[ \/]([\w.]+)/.exec(a)||/(webkit)[ \/]([\w.]+)/.exec(a)||/(opera)(?:.*version|)[ \/]([\w.]+)/.exec(a)||/(msie) ([\w.]+)/.exec(a)||a.indexOf("compatible")<0&&/(mozilla)(?:.*? rv:([\w.]+)|)/.exec(a)||[];return{browser:b[1]||"",version:b[2]||"0"}},a=p.uaMatch(g.userAgent),b={},a.browser&&(b[a.browser]=!0,b.version=a.version),b.chrome?b.
 webkit=!0:b.webkit&&(b.safari=!0),p.browser=b,p.sub=function(){function a(b,c){return new a.fn.init(b,c)}p.extend(!0,a,this),a.superclass=this,a.fn=a.prototype=this(),a.fn.constructor=a,a.sub=this.sub,a.fn.init=function c(c,d){return d&&d instanceof p&&!(d instanceof a)&&(d=a(d)),p.fn.init.call(this,c,d,b)},a.fn.init.prototype=a.fn;var b=a(e);return a}}();var bH,bI,bJ,bK=/alpha\([^)]*\)/i,bL=/opacity=([^)]*)/,bM=/^(top|right|bottom|left)$/,bN=/^(none|table(?!-c[ea]).+)/,bO=/^margin/,bP=new RegExp("^("+q+")(.*)$","i"),bQ=new RegExp("^("+q+")(?!px)[a-z%]+$","i"),bR=new RegExp("^([-+])=("+q+")","i"),bS={},bT={position:"absolute",visibility:"hidden",display:"block"},bU={letterSpacing:0,fontWeight:400},bV=["Top","Right","Bottom","Left"],bW=["Webkit","O","Moz","ms"],bX=p.fn.toggle;p.fn.extend({css:function(a,c){return p.access(this,function(a,c,d){return d!==b?p.style(a,c,d):p.css(a,c)},a,c,arguments.length>1)},show:function(){return b$(this,!0)},hide:function(){return b$(this)},toggle:fu
 nction(a,b){var c=typeof a=="boolean";return p.isFunction(a)&&p.isFunction(b)?bX.apply(this,arguments):this.each(function(){(c?a:bZ(this))?p(this).show():p(this).hide()})}}),p.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=bH(a,"opacity");return c===""?"1":c}}}},cssNumber:{fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":p.support.cssFloat?"cssFloat":"styleFloat"},style:function(a,c,d,e){if(!a||a.nodeType===3||a.nodeType===8||!a.style)return;var f,g,h,i=p.camelCase(c),j=a.style;c=p.cssProps[i]||(p.cssProps[i]=bY(j,i)),h=p.cssHooks[c]||p.cssHooks[i];if(d===b)return h&&"get"in h&&(f=h.get(a,!1,e))!==b?f:j[c];g=typeof d,g==="string"&&(f=bR.exec(d))&&(d=(f[1]+1)*f[2]+parseFloat(p.css(a,c)),g="number");if(d==null||g==="number"&&isNaN(d))return;g==="number"&&!p.cssNumber[i]&&(d+="px");if(!h||!("set"in h)||(d=h.set(a,d,e))!==b)try{j[c]=d}catch(k){}},css:function(a,c,d,e){var f,g,h,i=p.camelCase(c);return c=p.cssProps[
 i]||(p.cssProps[i]=bY(a.style,i)),h=p.cssHooks[c]||p.cssHooks[i],h&&"get"in h&&(f=h.get(a,!0,e)),f===b&&(f=bH(a,c)),f==="normal"&&c in bU&&(f=bU[c]),d||e!==b?(g=parseFloat(f),d||p.isNumeric(g)?g||0:f):f},swap:function(a,b,c){var d,e,f={};for(e in b)f[e]=a.style[e],a.style[e]=b[e];d=c.call(a);for(e in b)a.style[e]=f[e];return d}}),a.getComputedStyle?bH=function(b,c){var d,e,f,g,h=a.getComputedStyle(b,null),i=b.style;return h&&(d=h[c],d===""&&!p.contains(b.ownerDocument,b)&&(d=p.style(b,c)),bQ.test(d)&&bO.test(c)&&(e=i.width,f=i.minWidth,g=i.maxWidth,i.minWidth=i.maxWidth=i.width=d,d=h.width,i.width=e,i.minWidth=f,i.maxWidth=g)),d}:e.documentElement.currentStyle&&(bH=function(a,b){var c,d,e=a.currentStyle&&a.currentStyle[b],f=a.style;return e==null&&f&&f[b]&&(e=f[b]),bQ.test(e)&&!bM.test(b)&&(c=f.left,d=a.runtimeStyle&&a.runtimeStyle.left,d&&(a.runtimeStyle.left=a.currentStyle.left),f.left=b==="fontSize"?"1em":e,e=f.pixelLeft+"px",f.left=c,d&&(a.runtimeStyle.left=d)),e===""?"auto":e})
 ,p.each(["height","width"],function(a,b){p.cssHooks[b]={get:function(a,c,d){if(c)return a.offsetWidth===0&&bN.test(bH(a,"display"))?p.swap(a,bT,function(){return cb(a,b,d)}):cb(a,b,d)},set:function(a,c,d){return b_(a,c,d?ca(a,b,d,p.support.boxSizing&&p.css(a,"boxSizing")==="border-box"):0)}}}),p.support.opacity||(p.cssHooks.opacity={get:function(a,b){return bL.test((b&&a.currentStyle?a.currentStyle.filter:a.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":b?"1":""},set:function(a,b){var c=a.style,d=a.currentStyle,e=p.isNumeric(b)?"alpha(opacity="+b*100+")":"",f=d&&d.filter||c.filter||"";c.zoom=1;if(b>=1&&p.trim(f.replace(bK,""))===""&&c.removeAttribute){c.removeAttribute("filter");if(d&&!d.filter)return}c.filter=bK.test(f)?f.replace(bK,e):f+" "+e}}),p(function(){p.support.reliableMarginRight||(p.cssHooks.marginRight={get:function(a,b){return p.swap(a,{display:"inline-block"},function(){if(b)return bH(a,"marginRight")})}}),!p.support.pixelPosition&&p.fn.position&&p.each(["top","left"]
 ,function(a,b){p.cssHooks[b]={get:function(a,c){if(c){var d=bH(a,b);return bQ.test(d)?p(a).position()[b]+"px":d}}}})}),p.expr&&p.expr.filters&&(p.expr.filters.hidden=function(a){return a.offsetWidth===0&&a.offsetHeight===0||!p.support.reliableHiddenOffsets&&(a.style&&a.style.display||bH(a,"display"))==="none"},p.expr.filters.visible=function(a){return!p.expr.filters.hidden(a)}),p.each({margin:"",padding:"",border:"Width"},function(a,b){p.cssHooks[a+b]={expand:function(c){var d,e=typeof c=="string"?c.split(" "):[c],f={};for(d=0;d<4;d++)f[a+bV[d]+b]=e[d]||e[d-2]||e[0];return f}},bO.test(a)||(p.cssHooks[a+b].set=b_)});var cd=/%20/g,ce=/\[\]$/,cf=/\r?\n/g,cg=/^(?:color|date|datetime|datetime-local|email|hidden|month|number|password|range|search|tel|text|time|url|week)$/i,ch=/^(?:select|textarea)/i;p.fn.extend({serialize:function(){return p.param(this.serializeArray())},serializeArray:function(){return this.map(function(){return this.elements?p.makeArray(this.elements):this}).filter(func
 tion(){return this.name&&!this.disabled&&(this.checked||ch.test(this.nodeName)||cg.test(this.type))}).map(function(a,b){var c=p(this).val();return c==null?null:p.isArray(c)?p.map(c,function(a,c){return{name:b.name,value:a.replace(cf,"\r\n")}}):{name:b.name,value:c.replace(cf,"\r\n")}}).get()}}),p.param=function(a,c){var d,e=[],f=function(a,b){b=p.isFunction(b)?b():b==null?"":b,e[e.length]=encodeURIComponent(a)+"="+encodeURIComponent(b)};c===b&&(c=p.ajaxSettings&&p.ajaxSettings.traditional);if(p.isArray(a)||a.jquery&&!p.isPlainObject(a))p.each(a,function(){f(this.name,this.value)});else for(d in a)ci(d,a[d],c,f);return e.join("&").replace(cd,"+")};var cj,ck,cl=/#.*$/,cm=/^(.*?):[ \t]*([^\r\n]*)\r?$/mg,cn=/^(?:about|app|app\-storage|.+\-extension|file|res|widget):$/,co=/^(?:GET|HEAD)$/,cp=/^\/\//,cq=/\?/,cr=/<script\b[^<]*(?:(?!<\/script>)<[^<]*)*<\/script>/gi,cs=/([?&])_=[^&]*/,ct=/^([\w\+\.\-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,cu=p.fn.load,cv={},cw={},cx=["*/"]+["*"];try{ck=f.href}
 catch(cy){ck=e.createElement("a"),ck.href="",ck=ck.href}cj=ct.exec(ck.toLowerCase())||[],p.fn.load=function(a,c,d){if(typeof a!="string"&&cu)return cu.apply(this,arguments);if(!this.length)return this;var e,f,g,h=this,i=a.indexOf(" ");return i>=0&&(e=a.slice(i,a.length),a=a.slice(0,i)),p.isFunction(c)?(d=c,c=b):c&&typeof c=="object"&&(f="POST"),p.ajax({url:a,type:f,dataType:"html",data:c,complete:function(a,b){d&&h.each(d,g||[a.responseText,b,a])}}).done(function(a){g=arguments,h.html(e?p("<div>").append(a.replace(cr,"")).find(e):a)}),this},p.each("ajaxStart ajaxStop ajaxComplete ajaxError ajaxSuccess ajaxSend".split(" "),function(a,b){p.fn[b]=function(a){return this.on(b,a)}}),p.each(["get","post"],function(a,c){p[c]=function(a,d,e,f){return p.isFunction(d)&&(f=f||e,e=d,d=b),p.ajax({type:c,url:a,data:d,success:e,dataType:f})}}),p.extend({getScript:function(a,c){return p.get(a,b,c,"script")},getJSON:function(a,b,c){return p.get(a,b,c,"json")},ajaxSetup:function(a,b){return b?cB(a,p.
 ajaxSettings):(b=a,a=p.ajaxSettings),cB(a,b),a},ajaxSettings:{url:ck,isLocal:cn.test(cj[1]),global:!0,type:"GET",contentType:"application/x-www-form-urlencoded; charset=UTF-8",processData:!0,async:!0,accepts:{xml:"application/xml, text/xml",html:"text/html",text:"text/plain",json:"application/json, text/javascript","*":cx},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":a.String,"text html":!0,"text json":p.parseJSON,"text xml":p.parseXML},flatOptions:{context:!0,url:!0}},ajaxPrefilter:cz(cv),ajaxTransport:cz(cw),ajax:function(a,c){function y(a,c,f,i){var k,s,t,u,w,y=c;if(v===2)return;v=2,h&&clearTimeout(h),g=b,e=i||"",x.readyState=a>0?4:0,f&&(u=cC(l,x,f));if(a>=200&&a<300||a===304)l.ifModified&&(w=x.getResponseHeader("Last-Modified"),w&&(p.lastModified[d]=w),w=x.getResponseHeader("Etag"),w&&(p.etag[d]=w)),a===304?(y="notmodified",k=!0):(k=cD(l,u),y=k.state,s=k.data,t=k.error,k=!t);else{t=y;if(!y||a)y="error",a
 <0&&(a=0)}x.status=a,x.statusText=(c||y)+"",k?o.resolveWith(m,[s,y,x]):o.rejectWith(m,[x,y,t]),x.statusCode(r),r=b,j&&n.trigger("ajax"+(k?"Success":"Error"),[x,l,k?s:t]),q.fireWith(m,[x,y]),j&&(n.trigger("ajaxComplete",[x,l]),--p.active||p.event.trigger("ajaxStop"))}typeof a=="object"&&(c=a,a=b),c=c||{};var d,e,f,g,h,i,j,k,l=p.ajaxSetup({},c),m=l.context||l,n=m!==l&&(m.nodeType||m instanceof p)?p(m):p.event,o=p.Deferred(),q=p.Callbacks("once memory"),r=l.statusCode||{},t={},u={},v=0,w="canceled",x={readyState:0,setRequestHeader:function(a,b){if(!v){var c=a.toLowerCase();a=u[c]=u[c]||a,t[a]=b}return this},getAllResponseHeaders:function(){return v===2?e:null},getResponseHeader:function(a){var c;if(v===2){if(!f){f={};while(c=cm.exec(e))f[c[1].toLowerCase()]=c[2]}c=f[a.toLowerCase()]}return c===b?null:c},overrideMimeType:function(a){return v||(l.mimeType=a),this},abort:function(a){return a=a||w,g&&g.abort(a),y(0,a),this}};o.promise(x),x.success=x.done,x.error=x.fail,x.complete=q.add,x.s
 tatusCode=function(a){if(a){var b;if(v<2)for(b in a)r[b]=[r[b],a[b]];else b=a[x.status],x.always(b)}return this},l.url=((a||l.url)+"").replace(cl,"").replace(cp,cj[1]+"//"),l.dataTypes=p.trim(l.dataType||"*").toLowerCase().split(s),l.crossDomain==null&&(i=ct.exec(l.url.toLowerCase())||!1,l.crossDomain=i&&i.join(":")+(i[3]?"":i[1]==="http:"?80:443)!==cj.join(":")+(cj[3]?"":cj[1]==="http:"?80:443)),l.data&&l.processData&&typeof l.data!="string"&&(l.data=p.param(l.data,l.traditional)),cA(cv,l,c,x);if(v===2)return x;j=l.global,l.type=l.type.toUpperCase(),l.hasContent=!co.test(l.type),j&&p.active++===0&&p.event.trigger("ajaxStart");if(!l.hasContent){l.data&&(l.url+=(cq.test(l.url)?"&":"?")+l.data,delete l.data),d=l.url;if(l.cache===!1){var z=p.now(),A=l.url.replace(cs,"$1_="+z);l.url=A+(A===l.url?(cq.test(l.url)?"&":"?")+"_="+z:"")}}(l.data&&l.hasContent&&l.contentType!==!1||c.contentType)&&x.setRequestHeader("Content-Type",l.contentType),l.ifModified&&(d=d||l.url,p.lastModified[d]&&x.se
 tRequestHeader("If-Modified-Since",p.lastModified[d]),p.etag[d]&&x.setRequestHeader("If-None-Match",p.etag[d])),x.setRequestHeader("Accept",l.dataTypes[0]&&l.accepts[l.dataTypes[0]]?l.accepts[l.dataTypes[0]]+(l.dataTypes[0]!=="*"?", "+cx+"; q=0.01":""):l.accepts["*"]);for(k in l.headers)x.setRequestHeader(k,l.headers[k]);if(!l.beforeSend||l.beforeSend.call(m,x,l)!==!1&&v!==2){w="abort";for(k in{success:1,error:1,complete:1})x[k](l[k]);g=cA(cw,l,c,x);if(!g)y(-1,"No Transport");else{x.readyState=1,j&&n.trigger("ajaxSend",[x,l]),l.async&&l.timeout>0&&(h=setTimeout(function(){x.abort("timeout")},l.timeout));try{v=1,g.send(t,y)}catch(B){if(v<2)y(-1,B);else throw B}}return x}return x.abort()},active:0,lastModified:{},etag:{}});var cE=[],cF=/\?/,cG=/(=)\?(?=&|$)|\?\?/,cH=p.now();p.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var a=cE.pop()||p.expando+"_"+cH++;return this[a]=!0,a}}),p.ajaxPrefilter("json jsonp",function(c,d,e){var f,g,h,i=c.data,j=c.url,k=c.jsonp!==!1,l=k&&cG.test(j
 ),m=k&&!l&&typeof i=="string"&&!(c.contentType||"").indexOf("application/x-www-form-urlencoded")&&cG.test(i);if(c.dataTypes[0]==="jsonp"||l||m)return f=c.jsonpCallback=p.isFunction(c.jsonpCallback)?c.jsonpCallback():c.jsonpCallback,g=a[f],l?c.url=j.replace(cG,"$1"+f):m?c.data=i.replace(cG,"$1"+f):k&&(c.url+=(cF.test(j)?"&":"?")+c.jsonp+"="+f),c.converters["script json"]=function(){return h||p.error(f+" was not called"),h[0]},c.dataTypes[0]="json",a[f]=function(){h=arguments},e.always(function(){a[f]=g,c[f]&&(c.jsonpCallback=d.jsonpCallback,cE.push(f)),h&&p.isFunction(g)&&g(h[0]),h=g=b}),"script"}),p.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/javascript|ecmascript/},converters:{"text script":function(a){return p.globalEval(a),a}}}),p.ajaxPrefilter("script",function(a){a.cache===b&&(a.cache=!1),a.crossDomain&&(a.type="GET",a.global=!1)}),p.ajaxTransport("script",function(a){if(a.crossDomain)
 {var c,d=e.head||e.getElementsByTagName("head")[0]||e.documentElement;return{send:function(f,g){c=e.createElement("script"),c.async="async",a.scriptCharset&&(c.charset=a.scriptCharset),c.src=a.url,c.onload=c.onreadystatechange=function(a,e){if(e||!c.readyState||/loaded|complete/.test(c.readyState))c.onload=c.onreadystatechange=null,d&&c.parentNode&&d.removeChild(c),c=b,e||g(200,"success")},d.insertBefore(c,d.firstChild)},abort:function(){c&&c.onload(0,1)}}}});var cI,cJ=a.ActiveXObject?function(){for(var a in cI)cI[a](0,1)}:!1,cK=0;p.ajaxSettings.xhr=a.ActiveXObject?function(){return!this.isLocal&&cL()||cM()}:cL,function(a){p.extend(p.support,{ajax:!!a,cors:!!a&&"withCredentials"in a})}(p.ajaxSettings.xhr()),p.support.ajax&&p.ajaxTransport(function(c){if(!c.crossDomain||p.support.cors){var d;return{send:function(e,f){var g,h,i=c.xhr();c.username?i.open(c.type,c.url,c.async,c.username,c.password):i.open(c.type,c.url,c.async);if(c.xhrFields)for(h in c.xhrFields)i[h]=c.xhrFields[h];c.mi
 meType&&i.overrideMimeType&&i.overrideMimeType(c.mimeType),!c.crossDomain&&!e["X-Requested-With"]&&(e["X-Requested-With"]="XMLHttpRequest");try{for(h in e)i.setRequestHeader(h,e[h])}catch(j){}i.send(c.hasContent&&c.data||null),d=function(a,e){var h,j,k,l,m;try{if(d&&(e||i.readyState===4)){d=b,g&&(i.onreadystatechange=p.noop,cJ&&delete cI[g]);if(e)i.readyState!==4&&i.abort();else{h=i.status,k=i.getAllResponseHeaders(),l={},m=i.responseXML,m&&m.documentElement&&(l.xml=m);try{l.text=i.responseText}catch(a){}try{j=i.statusText}catch(n){j=""}!h&&c.isLocal&&!c.crossDomain?h=l.text?200:404:h===1223&&(h=204)}}}catch(o){e||f(-1,o)}l&&f(h,j,l,k)},c.async?i.readyState===4?setTimeout(d,0):(g=++cK,cJ&&(cI||(cI={},p(a).unload(cJ)),cI[g]=d),i.onreadystatechange=d):d()},abort:function(){d&&d(0,1)}}}});var cN,cO,cP=/^(?:toggle|show|hide)$/,cQ=new RegExp("^(?:([-+])=|)("+q+")([a-z%]*)$","i"),cR=/queueHooks$/,cS=[cY],cT={"*":[function(a,b){var c,d,e=this.createTween(a,b),f=cQ.exec(b),g=e.cur(),h=+g||0
 ,i=1,j=20;if(f){c=+f[2],d=f[3]||(p.cssNumber[a]?"":"px");if(d!=="px"&&h){h=p.css(e.elem,a,!0)||c||1;do i=i||".5",h=h/i,p.style(e.elem,a,h+d);while(i!==(i=e.cur()/g)&&i!==1&&--j)}e.unit=d,e.start=h,e.end=f[1]?h+(f[1]+1)*c:c}return e}]};p.Animation=p.extend(cW,{tweener:function(a,b){p.isFunction(a)?(b=a,a=["*"]):a=a.split(" ");var c,d=0,e=a.length;for(;d<e;d++)c=a[d],cT[c]=cT[c]||[],cT[c].unshift(b)},prefilter:function(a,b){b?cS.unshift(a):cS.push(a)}}),p.Tween=cZ,cZ.prototype={constructor:cZ,init:function(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||"swing",this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(p.cssNumber[c]?"":"px")},cur:function(){var a=cZ.propHooks[this.prop];return a&&a.get?a.get(this):cZ.propHooks._default.get(this)},run:function(a){var b,c=cZ.propHooks[this.prop];return this.options.duration?this.pos=b=p.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):this.pos=b=a,this.now=(this.end-this.start)*b+this.start,this.
 options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):cZ.propHooks._default.set(this),this}},cZ.prototype.init.prototype=cZ.prototype,cZ.propHooks={_default:{get:function(a){var b;return a.elem[a.prop]==null||!!a.elem.style&&a.elem.style[a.prop]!=null?(b=p.css(a.elem,a.prop,!1,""),!b||b==="auto"?0:b):a.elem[a.prop]},set:function(a){p.fx.step[a.prop]?p.fx.step[a.prop](a):a.elem.style&&(a.elem.style[p.cssProps[a.prop]]!=null||p.cssHooks[a.prop])?p.style(a.elem,a.prop,a.now+a.unit):a.elem[a.prop]=a.now}}},cZ.propHooks.scrollTop=cZ.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},p.each(["toggle","show","hide"],function(a,b){var c=p.fn[b];p.fn[b]=function(d,e,f){return d==null||typeof d=="boolean"||!a&&p.isFunction(d)&&p.isFunction(e)?c.apply(this,arguments):this.animate(c$(b,!0),d,e,f)}}),p.fn.extend({fadeTo:function(a,b,c,d){return this.filter(bZ).css("opacity",0).show().end().animate({opacity:b},a,c,d)},an
 imate:function(a,b,c,d){var e=p.isEmptyObject(a),f=p.speed(b,c,d),g=function(){var b=cW(this,p.extend({},a),f);e&&b.stop(!0)};return e||f.queue===!1?this.each(g):this.queue(f.queue,g)},stop:function(a,c,d){var e=function(a){var b=a.stop;delete a.stop

<TRUNCATED>

[34/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java
new file mode 100644
index 0000000..8aaeb58
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java
@@ -0,0 +1,77 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.json.GsonObject;
+
+public abstract class BinaryNode extends LogicalNode implements Cloneable, GsonObject {
+	@Expose LogicalNode leftChild = null;
+	@Expose LogicalNode rightChild = null;
+
+	public BinaryNode(int pid, NodeType nodeType) {
+		super(pid, nodeType);
+	}
+	
+	public <T extends LogicalNode> T getLeftChild() {
+		return (T) this.leftChild;
+	}
+	
+	public void setLeftChild(LogicalNode op) {
+		this.leftChild = op;
+	}
+
+	public <T extends LogicalNode> T getRightChild() {
+		return (T) this.rightChild;
+	}
+
+	public void setRightChild(LogicalNode op) {
+		this.rightChild = op;
+	}
+
+  public boolean deepEquals(Object o) {
+    if (o instanceof BinaryNode) {
+      BinaryNode b = (BinaryNode) o;
+      return equals(o) &&
+          leftChild.deepEquals(b.leftChild) && rightChild.deepEquals(b.rightChild);
+    }
+    return false;
+  }
+	
+	@Override
+  public Object clone() throws CloneNotSupportedException {
+	  BinaryNode binNode = (BinaryNode) super.clone();
+	  binNode.leftChild = (LogicalNode) leftChild.clone();
+	  binNode.rightChild = (LogicalNode) rightChild.clone();
+	  
+	  return binNode;
+	}
+	
+	public void preOrder(LogicalNodeVisitor visitor) {
+	  visitor.visit(this);
+	  leftChild.postOrder(visitor);
+    rightChild.postOrder(visitor);
+  }
+	
+	public void postOrder(LogicalNodeVisitor visitor) {
+    leftChild.postOrder(visitor);
+    rightChild.postOrder(visitor);
+    visitor.visit(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateDatabaseNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateDatabaseNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateDatabaseNode.java
new file mode 100644
index 0000000..9dc73e2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateDatabaseNode.java
@@ -0,0 +1,87 @@
+/*
+ * 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.planner.logical;
+
+import com.google.common.base.Objects;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.engine.planner.PlanString;
+
+public class CreateDatabaseNode extends LogicalNode implements Cloneable {
+  private String databaseName;
+  private boolean ifNotExists;
+
+  public CreateDatabaseNode(int pid) {
+    super(pid, NodeType.CREATE_DATABASE);
+  }
+
+  public void init(String databaseName, boolean ifNotExists) {
+    this.databaseName = databaseName;
+    this.ifNotExists = ifNotExists;
+  }
+
+  public String getDatabaseName() {
+    return this.databaseName;
+  }
+
+  public boolean isIfNotExists() {
+    return ifNotExists;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this).appendTitle(ifNotExists ? " IF NOT EXISTS " : " ").appendTitle(databaseName);
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(databaseName, ifNotExists);
+  }
+
+  public boolean equals(Object obj) {
+    if (obj instanceof CreateDatabaseNode) {
+      CreateDatabaseNode other = (CreateDatabaseNode) obj;
+      return super.equals(other) && this.databaseName.equals(other.databaseName) && ifNotExists == other.ifNotExists;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    CreateDatabaseNode newNode = (CreateDatabaseNode) super.clone();
+    newNode.databaseName = databaseName;
+    newNode.ifNotExists = ifNotExists;
+    return newNode;
+  }
+
+  @Override
+  public String toString() {
+    return "CREATE DATABASE " + (ifNotExists ? " IF NOT EXISTS " : "")
+        + CatalogUtil.denormalizeIdentifier(databaseName);
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
new file mode 100644
index 0000000..c70fb10
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
@@ -0,0 +1,145 @@
+/**
+ * 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.planner.logical;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.util.TUtil;
+
+public class CreateTableNode extends StoreTableNode implements Cloneable {
+  @Expose private Schema schema;
+  @Expose private Path path;
+  @Expose private boolean external;
+  @Expose private boolean ifNotExists;
+
+  public CreateTableNode(int pid) {
+    super(pid, NodeType.CREATE_TABLE);
+  }
+
+  public void setTableSchema(Schema schema) {
+    this.schema = schema;
+  }
+    
+  public Schema getTableSchema() {
+    return this.schema;
+  }
+
+  public Schema getLogicalSchema() {
+    if (hasPartition()) {
+      Schema logicalSchema = new Schema(schema);
+      logicalSchema.addColumns(getPartitionMethod().getExpressionSchema());
+      return logicalSchema;
+    } else {
+      return schema;
+    }
+  }
+
+  public boolean hasPath() {
+    return this.path != null;
+  }
+
+  public void setPath(Path path) {
+    this.path = path;
+  }
+  
+  public Path getPath() {
+    return this.path;
+  }
+
+  public boolean isExternal() {
+    return external;
+  }
+
+  public void setExternal(boolean external) {
+    this.external = external;
+  }
+
+  public boolean hasSubQuery() {
+    return child != null;
+  }
+
+  public void setIfNotExists(boolean ifNotExists) {
+    this.ifNotExists = ifNotExists;
+  }
+
+  public boolean isIfNotExists() {
+    return ifNotExists;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this);
+  }
+
+  public int hashCode() {
+    return super.hashCode() ^ Objects.hashCode(schema, path, external, ifNotExists) * 31;
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof CreateTableNode) {
+      CreateTableNode other = (CreateTableNode) obj;
+      return super.equals(other)
+          && this.schema.equals(other.schema)
+          && this.external == other.external
+          && TUtil.checkEquals(path, other.path)
+          && ifNotExists == other.ifNotExists;
+    } else {
+      return false;
+    }
+  }
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    CreateTableNode createTableNode = (CreateTableNode) super.clone();
+    createTableNode.tableName = tableName;
+    createTableNode.schema = (Schema) schema.clone();
+    createTableNode.storageType = storageType;
+    createTableNode.external = external;
+    createTableNode.path = path != null ? new Path(path.toString()) : null;
+    createTableNode.options = (Options) (options != null ? options.clone() : null);
+    createTableNode.ifNotExists = ifNotExists;
+    return createTableNode;
+  }
+
+  public String toString() {
+    return "CreateTable (table=" + tableName + ", external=" + external + ", storeType=" + storageType +
+        ", ifNotExists=" + ifNotExists +")";
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    if (hasSubQuery()) {
+      child.preOrder(visitor);
+    }
+    visitor.visit(this);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+    if (hasSubQuery()) {
+      child.preOrder(visitor);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropDatabaseNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropDatabaseNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropDatabaseNode.java
new file mode 100644
index 0000000..1578759
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropDatabaseNode.java
@@ -0,0 +1,85 @@
+/*
+ * 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.planner.logical;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.planner.PlanString;
+
+public class DropDatabaseNode extends LogicalNode implements Cloneable {
+  @Expose private String databaseName;
+  @Expose private boolean ifExists;
+
+  public DropDatabaseNode(int pid) {
+    super(pid, NodeType.DROP_DATABASE);
+  }
+
+  public void init(String databaseName, boolean ifExists) {
+    this.databaseName = databaseName;
+    this.ifExists = ifExists;
+  }
+
+  public String getDatabaseName() {
+    return this.databaseName;
+  }
+
+  public boolean isIfExists() {
+    return ifExists;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this).appendTitle(ifExists ? " IF EXISTS " : " ").appendTitle(databaseName);
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(databaseName, ifExists);
+  }
+
+  public boolean equals(Object obj) {
+    if (obj instanceof DropDatabaseNode) {
+      DropDatabaseNode other = (DropDatabaseNode) obj;
+      return super.equals(other) && this.databaseName.equals(other.databaseName) && ifExists == other.ifExists;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    DropDatabaseNode dropTableNode = (DropDatabaseNode) super.clone();
+    dropTableNode.databaseName = databaseName;
+    return dropTableNode;
+  }
+
+  @Override
+  public String toString() {
+    return "DROP DATABASE " + (ifExists ? "IF EXISTS ":"") + databaseName;
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java
new file mode 100644
index 0000000..ac68a9c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java
@@ -0,0 +1,95 @@
+/*
+ * 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.planner.logical;
+
+import com.google.common.base.Objects;
+import org.apache.tajo.engine.planner.PlanString;
+
+public class DropTableNode extends LogicalNode implements Cloneable {
+  private String tableName;
+  private boolean ifExists;
+  private boolean purge;
+
+  public DropTableNode(int pid) {
+    super(pid, NodeType.DROP_TABLE);
+  }
+
+  public void init(String tableName, boolean ifExists, boolean purge) {
+    this.tableName = tableName;
+    this.ifExists = ifExists;
+    this.purge = purge;
+  }
+
+  public String getTableName() {
+    return this.tableName;
+  }
+
+  public boolean isIfExists() {
+    return this.ifExists;
+  }
+
+  public boolean isPurge() {
+    return this.purge;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this).appendTitle(ifExists ? " IF EXISTS" : "").appendTitle(purge ? " PURGE" : "");
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(tableName, ifExists, purge);
+  }
+
+  public boolean equals(Object obj) {
+    if (obj instanceof DropTableNode) {
+      DropTableNode other = (DropTableNode) obj;
+      return super.equals(other) &&
+          this.tableName.equals(other.tableName) &&
+          this.ifExists == other.ifExists &&
+          this.purge == other.purge;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    DropTableNode dropTableNode = (DropTableNode) super.clone();
+    dropTableNode.tableName = tableName;
+    dropTableNode.ifExists = ifExists;
+    dropTableNode.purge = purge;
+    return dropTableNode;
+  }
+
+  @Override
+  public String toString() {
+    return "DROP TABLE " + (ifExists ? "IF EXISTS " : "") + tableName + (purge ? " PURGE" : "");
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java
new file mode 100644
index 0000000..6ea3f40
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java
@@ -0,0 +1,83 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.util.TUtil;
+
+public class EvalExprNode extends LogicalNode implements Projectable {
+  @Expose private Target[] exprs;
+
+  public EvalExprNode(int pid) {
+    super(pid, NodeType.EXPRS);
+  }
+
+  @Override
+  public boolean hasTargets() {
+    return true;
+  }
+
+  @Override
+  public void setTargets(Target[] targets) {
+    this.exprs = targets;
+  }
+
+  @Override
+  public Target[] getTargets() {
+    return exprs;
+  }
+
+  public Target[] getExprs() {
+    return this.exprs;
+  }
+  
+  @Override
+  public String toString() {
+    return "EvalExprNode (" + TUtil.arrayToString(exprs) + ")";
+  }
+
+  public boolean equals(Object object) {
+    if (object instanceof EvalExprNode) {
+      EvalExprNode other = (EvalExprNode) object;
+      return TUtil.checkEquals(this.exprs, other.exprs);
+    } else {
+      return false;
+    }
+  }
+  
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    // nothing
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    // nothing
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java
new file mode 100644
index 0000000..1540e1c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * 
+ */
+package org.apache.tajo.engine.planner.logical;
+
+import org.apache.tajo.engine.planner.PlanString;
+
+public class ExceptNode extends BinaryNode {
+
+  public ExceptNode(int pid) {
+    super(pid, NodeType.EXCEPT);
+  }
+
+  public void init(LogicalNode left, LogicalNode right) {
+    setLeftChild(left);
+    setRightChild(right);
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    planStr.appendTitle(" (L - " + ((TableSubQueryNode)getLeftChild()).getTableName());
+    planStr.appendTitle(", R - " + ((TableSubQueryNode)getRightChild()).getTableName());
+    planStr.appendTitle(")");
+    return planStr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java
new file mode 100644
index 0000000..bafe0c6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java
@@ -0,0 +1,198 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.eval.AggregationFunctionCallEval;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.util.TUtil;
+
+public class GroupbyNode extends UnaryNode implements Projectable, Cloneable {
+	/** Grouping key sets */
+  @Expose private Column [] groupingColumns;
+  /** Aggregation Functions */
+  @Expose private AggregationFunctionCallEval [] aggrFunctions;
+  /**
+   * It's a list of targets. The grouping columns should be followed by aggregation functions.
+   * aggrFunctions keep actual aggregation functions, but it only contains field references.
+   * */
+  @Expose private Target [] targets;
+  @Expose private boolean hasDistinct = false;
+
+  public GroupbyNode(int pid) {
+    super(pid, NodeType.GROUP_BY);
+  }
+
+  public final boolean isEmptyGrouping() {
+    return groupingColumns == null || groupingColumns.length == 0;
+  }
+
+  public void setGroupingColumns(Column [] groupingColumns) {
+    this.groupingColumns = groupingColumns;
+  }
+
+	public final Column [] getGroupingColumns() {
+	  return this.groupingColumns;
+	}
+
+  public final boolean isDistinct() {
+    return hasDistinct;
+  }
+
+  public void setDistinct(boolean distinct) {
+    hasDistinct = distinct;
+  }
+
+  public boolean hasAggFunctions() {
+    return this.aggrFunctions != null;
+  }
+
+  public AggregationFunctionCallEval [] getAggFunctions() {
+    return this.aggrFunctions;
+  }
+
+  public void setAggFunctions(AggregationFunctionCallEval[] evals) {
+    this.aggrFunctions = evals;
+  }
+
+  @Override
+  public boolean hasTargets() {
+    return this.targets != null;
+  }
+
+  @Override
+  public void setTargets(Target[] targets) {
+    this.targets = targets;
+    setOutSchema(PlannerUtil.targetToSchema(targets));
+  }
+
+  @Override
+  public Target[] getTargets() {
+    return this.targets;
+  }
+  
+  public void setChild(LogicalNode subNode) {
+    super.setChild(subNode);
+  }
+  
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GroupBy (");
+    if (groupingColumns != null || groupingColumns.length > 0) {
+      sb.append("grouping set=").append(TUtil.arrayToString(groupingColumns));
+      sb.append(", ");
+    }
+    if (hasAggFunctions()) {
+      sb.append("funcs=").append(TUtil.arrayToString(aggrFunctions));
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof GroupbyNode) {
+      GroupbyNode other = (GroupbyNode) obj;
+      boolean eq = super.equals(other);
+      eq = eq && TUtil.checkEquals(groupingColumns, other.groupingColumns);
+      eq = eq && TUtil.checkEquals(aggrFunctions, other.aggrFunctions);
+      eq = eq && TUtil.checkEquals(targets, other.targets);
+      return eq;
+    } else {
+      return false;  
+    }
+  }
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    GroupbyNode grp = (GroupbyNode) super.clone();
+    if (groupingColumns != null) {
+      grp.groupingColumns = new Column[groupingColumns.length];
+      for (int i = 0; i < groupingColumns.length; i++) {
+        grp.groupingColumns[i] = groupingColumns[i];
+      }
+    }
+
+    if (aggrFunctions != null) {
+      grp.aggrFunctions = new AggregationFunctionCallEval[aggrFunctions.length];
+      for (int i = 0; i < aggrFunctions.length; i++) {
+        grp.aggrFunctions[i] = (AggregationFunctionCallEval) aggrFunctions[i].clone();
+      }
+    }
+
+    if (targets != null) {
+      grp.targets = new Target[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        grp.targets[i] = (Target) targets[i].clone();
+      }
+    }
+
+    return grp;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("(");
+    Column [] groupingColumns = this.groupingColumns;
+    for (int j = 0; j < groupingColumns.length; j++) {
+      sb.append(groupingColumns[j].getSimpleName());
+      if(j < groupingColumns.length - 1) {
+        sb.append(",");
+      }
+    }
+
+    sb.append(")");
+
+    planStr.appendTitle(sb.toString());
+
+    // there can be no aggregation functions
+    if (hasAggFunctions()) {
+      sb = new StringBuilder();
+      sb.append("(");
+
+      for (int j = 0; j < aggrFunctions.length; j++) {
+        sb.append(aggrFunctions[j]);
+        if(j < aggrFunctions.length - 1) {
+          sb.append(",");
+        }
+      }
+      sb.append(")");
+      planStr.appendExplain("exprs: ").appendExplain(sb.toString());
+    }
+
+    sb = new StringBuilder("target list: ");
+    for (int i = 0; i < targets.length; i++) {
+      sb.append(targets[i]);
+      if( i < targets.length - 1) {
+        sb.append(", ");
+      }
+    }
+    planStr.addExplan(sb.toString());
+
+    planStr.addDetail("out schema:").appendDetail(getOutSchema().toString());
+    planStr.addDetail("in schema:").appendDetail(getInSchema().toString());
+
+    return planStr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/HavingNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/HavingNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/HavingNode.java
new file mode 100644
index 0000000..6c45868
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/HavingNode.java
@@ -0,0 +1,67 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
+
+public class HavingNode extends UnaryNode implements Cloneable {
+	@Expose private EvalNode qual;
+
+  public HavingNode(int pid) {
+    super(pid, NodeType.HAVING);
+  }
+
+	public EvalNode getQual() {
+		return this.qual;
+	}
+
+	public void setQual(EvalNode qual) {
+		this.qual = qual;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof HavingNode) {
+      HavingNode other = (HavingNode) obj;
+      return super.equals(other) 
+          && this.qual.equals(other.qual);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    HavingNode selNode = (HavingNode) super.clone();
+    selNode.qual = (EvalNode) this.qual.clone();
+    
+    return selNode;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this).appendTitle(" (").appendTitle(qual.toString()).appendTitle(")");
+  }
+
+  public String toString() {
+    return "Having (filter=" + qual + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java
new file mode 100644
index 0000000..bff0b31
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java
@@ -0,0 +1,122 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.Gson;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+
+public class IndexScanNode extends ScanNode {
+  @Expose private SortSpec [] sortKeys;
+  @Expose private Schema keySchema = null;
+  @Expose private Datum[] datum = null;
+  
+  public IndexScanNode(int pid, ScanNode scanNode ,
+      Schema keySchema , Datum[] datum, SortSpec[] sortKeys ) {
+    super(pid);
+    init(scanNode.getTableDesc());
+    setQual(scanNode.getQual());
+    setInSchema(scanNode.getInSchema());
+    setTargets(scanNode.getTargets());
+    setType(NodeType.BST_INDEX_SCAN);
+    this.sortKeys = sortKeys;
+    this.keySchema = keySchema;
+    this.datum = datum;
+  }
+  
+  public SortSpec[] getSortKeys() {
+    return this.sortKeys;
+  }
+  
+  public Schema getKeySchema() {
+    return this.keySchema;
+  }
+  
+  public Datum[] getDatum() {
+    return this.datum;
+  }
+  
+  public void setSortKeys(SortSpec[] sortKeys) {
+    this.sortKeys = sortKeys;
+  }
+  
+  public void setKeySchema( Schema keySchema ) {
+    this.keySchema = keySchema;
+  }
+
+  @Override
+  public String toString() {
+    Gson gson = CoreGsonHelper.getInstance();
+    StringBuilder builder = new StringBuilder();
+    builder.append("IndexScanNode : {\n");
+    builder.append("  \"keySchema\" : \"" + gson.toJson(this.keySchema) + "\"\n");
+    builder.append("  \"sortKeys\" : \"" + gson.toJson(this.sortKeys) + " \"\n");
+    builder.append("  \"datums\" : \"" + gson.toJson(this.datum) + "\"\n");
+    builder.append("      <<\"superClass\" : " + super.toString());
+    builder.append(">>}");
+    builder.append("}");
+    return builder.toString();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof IndexScanNode) {
+      IndexScanNode other = (IndexScanNode) obj;
+      
+      boolean eq = super.equals(other);
+      eq = eq && this.sortKeys.length == other.sortKeys.length;
+      if(eq) {
+        for(int i = 0 ; i < this.sortKeys.length ; i ++) {
+          eq = eq && this.sortKeys[i].getSortKey().equals(
+              other.sortKeys[i].getSortKey());
+          eq = eq && this.sortKeys[i].isAscending()
+              == other.sortKeys[i].isAscending();
+          eq = eq && this.sortKeys[i].isNullFirst()
+              == other.sortKeys[i].isNullFirst();
+        }
+      }
+      if(eq) {
+        for(int i = 0 ; i < this.datum.length ; i ++ ) {
+          eq = eq && this.datum[i].equals(other.datum[i]);
+        }
+      }
+     return eq;
+    }   
+    return false;
+  } 
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    IndexScanNode indexNode = (IndexScanNode) super.clone();
+    indexNode.keySchema = (Schema) this.keySchema.clone();
+    indexNode.sortKeys = new SortSpec[this.sortKeys.length];
+    for(int i = 0 ; i < sortKeys.length ; i ++ )
+      indexNode.sortKeys[i] = (SortSpec) this.sortKeys[i].clone();
+    indexNode.datum = new Datum[this.datum.length];
+    for(int i = 0 ; i < datum.length ; i ++ ) {
+      indexNode.datum[i] = this.datum[i];
+    }
+    return indexNode;
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/InsertNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/InsertNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/InsertNode.java
new file mode 100644
index 0000000..f5e87ef
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/InsertNode.java
@@ -0,0 +1,182 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.util.TUtil;
+
+public class InsertNode extends StoreTableNode implements Cloneable {
+  @Expose private boolean overwrite;
+  @Expose private Schema tableSchema;
+
+  /** a target schema of a target table */
+  @Expose private Schema targetSchema;
+  /** a output schema of select clause */
+  @Expose private Schema projectedSchema;
+  @Expose private Path path;
+
+  public InsertNode(int pid) {
+    super(pid, NodeType.INSERT);
+  }
+
+  public void setTargetTable(TableDesc desc) {
+    setTableName(desc.getName());
+    if (desc.hasPartition()) {
+      tableSchema = desc.getLogicalSchema();
+    } else {
+      tableSchema = desc.getSchema();
+    }
+    setPath(desc.getPath());
+    setOptions(desc.getMeta().getOptions());
+    setStorageType(desc.getMeta().getStoreType());
+
+    if (desc.hasPartition()) {
+      this.setPartitionMethod(desc.getPartitionMethod());
+    }
+  }
+
+  public void setTargetLocation(Path path) {
+    this.path = path;
+  }
+
+  public void setSubQuery(LogicalNode subQuery) {
+    this.setChild(subQuery);
+    this.setInSchema(subQuery.getOutSchema());
+    this.setOutSchema(subQuery.getOutSchema());
+  }
+
+  public boolean isOverwrite() {
+    return overwrite;
+  }
+
+  public void setOverwrite(boolean overwrite) {
+    this.overwrite = overwrite;
+  }
+
+  public Schema getTableSchema() {
+    return tableSchema;
+  }
+
+  public void setTableSchema(Schema tableSchema) {
+    this.tableSchema = tableSchema;
+  }
+
+  public boolean hasTargetSchema() {
+    return this.targetSchema != null;
+  }
+
+  public Schema getTargetSchema() {
+    return this.targetSchema;
+  }
+
+  public void setTargetSchema(Schema schema) {
+    this.targetSchema = schema;
+  }
+
+  public Schema getProjectedSchema() {
+    return this.projectedSchema;
+  }
+
+  public void setProjectedSchema(Schema projected) {
+    this.projectedSchema = projected;
+  }
+
+  public boolean hasPath() {
+    return this.path != null;
+  }
+
+  public void setPath(Path path) {
+    this.path = path;
+  }
+  
+  public Path getPath() {
+    return this.path;
+  }
+
+  public boolean hasStorageType() {
+    return this.storageType != null;
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof InsertNode) {
+      InsertNode other = (InsertNode) obj;
+      return super.equals(other)
+          && this.overwrite == other.overwrite
+          && TUtil.checkEquals(this.tableSchema, other.tableSchema)
+          && TUtil.checkEquals(this.targetSchema, other.targetSchema)
+          && TUtil.checkEquals(path, other.path);
+    } else {
+      return false;
+    }
+  }
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    InsertNode insertNode = (InsertNode) super.clone();
+    insertNode.overwrite = overwrite;
+    insertNode.tableSchema = new Schema(tableSchema);
+    insertNode.targetSchema = targetSchema != null ? new Schema(targetSchema) : null;
+    insertNode.path = path != null ? new Path(path.toString()) : null;
+    return insertNode;
+  }
+  
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Insert (overwrite=").append(overwrite);
+    if (hasTargetTable()) {
+      sb.append(",table=").append(tableName);
+    }
+    if (hasPath()) {
+      sb.append(", location=").append(path);
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    getChild().preOrder(visitor);
+    visitor.visit(this);
+  }
+
+  @Override
+  public void postOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+    getChild().postOrder(visitor);
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planString = new PlanString(this);
+    planString.appendTitle(" INTO ");
+    if (hasTargetTable()) {
+      planString.appendTitle(getTableName());
+      if (hasTargetSchema()) {
+        planString.addExplan(getTargetSchema().toString());
+      }
+    } else {
+      planString.addExplan("LOCATION " + path);
+    }
+    return planString;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java
new file mode 100644
index 0000000..4bcfd24
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java
@@ -0,0 +1,44 @@
+/**
+ * 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.planner.logical;
+
+import org.apache.tajo.engine.planner.PlanString;
+
+public class IntersectNode extends BinaryNode {
+  public IntersectNode(int pid) {
+    super(pid, NodeType.INTERSECT);
+  }
+
+  public void init(LogicalNode left, LogicalNode right) {
+    setLeftChild(left);
+    setRightChild(right);
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    planStr.appendTitle(" (L - " + ((TableSubQueryNode)getLeftChild()).getTableName());
+    planStr.appendTitle(", R - " + ((TableSubQueryNode)getRightChild()).getTableName());
+    planStr.appendTitle(")");
+    return planStr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java
new file mode 100644
index 0000000..915c66d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java
@@ -0,0 +1,165 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.util.TUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class JoinNode extends BinaryNode implements Projectable, Cloneable {
+  @Expose private JoinType joinType;
+  @Expose private EvalNode joinQual;
+  @Expose private Target[] targets;
+
+  @Expose private boolean candidateBroadcast = false;
+  @Expose private List<LogicalNode> broadcastTargets = new ArrayList<LogicalNode>();
+
+  public JoinNode(int pid) {
+    super(pid, NodeType.JOIN);
+  }
+
+  public void init(JoinType joinType, LogicalNode left, LogicalNode right) {
+    this.joinType = joinType;
+    setLeftChild(left);
+    setRightChild(right);
+  }
+
+  public boolean isCandidateBroadcast() {
+    return candidateBroadcast;
+  }
+
+  public void setCandidateBroadcast(boolean candidateBroadcast) {
+    this.candidateBroadcast = candidateBroadcast;
+  }
+
+  public List<LogicalNode> getBroadcastTargets() {
+    return broadcastTargets;
+  }
+
+  public void setBroadcastTargets(List<LogicalNode> broadcastTargets) {
+    this.broadcastTargets = broadcastTargets;
+  }
+
+  public JoinType getJoinType() {
+    return this.joinType;
+  }
+
+  public void setJoinType(JoinType joinType) {
+    this.joinType = joinType;
+  }
+
+  public void setJoinQual(EvalNode joinQual) {
+    this.joinQual = joinQual;
+  }
+
+  public boolean hasJoinQual() {
+    return this.joinQual != null;
+  }
+
+  public EvalNode getJoinQual() {
+    return this.joinQual;
+  }
+
+  @Override
+  public boolean hasTargets() {
+    return this.targets != null;
+  }
+
+  @Override
+  public Target[] getTargets() {
+    return this.targets;
+  }
+
+  @Override
+  public void setTargets(Target[] targets) {
+    this.targets = targets;
+    this.setOutSchema(PlannerUtil.targetToSchema(targets));
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this).appendTitle("(").appendTitle(joinType.name()).appendTitle(")");
+    if (hasJoinQual()) {
+      planStr.addExplan("Join Cond: " + joinQual.toString());
+    }
+
+    if (hasTargets()) {
+      planStr.addExplan("target list: ");
+      boolean first = true;
+      for (Target target : targets) {
+        if (!first) {
+          planStr.appendExplain(", ");
+        }
+        planStr.appendExplain(target.toString());
+        first = false;
+      }
+    }
+
+    planStr.addDetail("out schema: " + getOutSchema());
+    planStr.addDetail("in schema: " + getInSchema());
+
+    return planStr;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof JoinNode) {
+      JoinNode other = (JoinNode) obj;
+      boolean eq = this.joinType.equals(other.joinType);
+      eq &= TUtil.checkEquals(this.targets, other.targets);
+      eq &= TUtil.checkEquals(joinQual, other.joinQual);
+      return eq && leftChild.equals(other.leftChild) && rightChild.equals(other.rightChild);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    JoinNode join = (JoinNode) super.clone();
+    join.joinType = this.joinType;
+    join.joinQual = this.joinQual == null ? null : (EvalNode) this.joinQual.clone();
+    if (hasTargets()) {
+      join.targets = new Target[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        join.targets[i] = (Target) targets[i].clone();
+      }
+    }
+    return join;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Join (type").append(joinType);
+    if (hasJoinQual()) {
+      sb.append(",filter=").append(joinQual);
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java
new file mode 100644
index 0000000..7f5b258
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java
@@ -0,0 +1,65 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.planner.PlanString;
+
+public final class LimitNode extends UnaryNode implements Cloneable {
+	@Expose private long fetchFirstNum;
+
+  public LimitNode(int pid) {
+    super(pid, NodeType.LIMIT);
+  }
+
+  public void setFetchFirst(long num) {
+    this.fetchFirstNum = num;
+  }
+  
+  public long getFetchFirstNum() {
+    return fetchFirstNum;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this).appendTitle(" " + fetchFirstNum);
+  }
+  
+  @Override 
+  public boolean equals(Object obj) {
+    if (obj instanceof LimitNode) {
+      LimitNode other = (LimitNode) obj;
+      return super.equals(other)
+          && fetchFirstNum == other.fetchFirstNum;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    LimitNode newLimitNode = (LimitNode) super.clone();
+    newLimitNode.fetchFirstNum = fetchFirstNum;
+    return newLimitNode;
+  }
+
+  public String toString() {
+    return "Limit (fetch first=" + fetchFirstNum + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java
new file mode 100644
index 0000000..f62caa4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java
@@ -0,0 +1,128 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.util.TUtil;
+
+public abstract class LogicalNode implements Cloneable, GsonObject {
+  @Expose private int pid;
+  @Expose private NodeType type;
+	@Expose private Schema inputSchema;
+	@Expose	private Schema outputSchema;
+
+	@Expose	private double cost = 0;
+
+	protected LogicalNode(int pid, NodeType type) {
+    this.pid = pid;
+    this.type = type;
+	}
+
+  public int getPID() {
+    return pid;
+  }
+
+  public void setPID(int pid) {
+    this.pid = pid;
+  }
+	
+	public NodeType getType() {
+		return this.type;
+	}
+
+	public void setType(NodeType type) {
+		this.type = type;
+	}
+
+	public double getCost() {
+		return this.cost;
+	}
+
+	public void setCost(double cost) {
+		this.cost = cost;
+	}
+	
+	public void setInSchema(Schema inSchema) {
+	  this.inputSchema = inSchema;
+	}
+	
+	public Schema getInSchema() {
+	  return this.inputSchema;
+	}
+	
+	public void setOutSchema(Schema outSchema) {
+	  this.outputSchema = outSchema;
+	}
+	
+	public Schema getOutSchema() {
+	  return this.outputSchema;
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+	  if (obj instanceof LogicalNode) {
+	    LogicalNode other = (LogicalNode) obj;
+
+      boolean eq = this.type == other.type;
+      eq = eq && TUtil.checkEquals(this.inputSchema, other.inputSchema);
+      eq = eq && TUtil.checkEquals(this.outputSchema, other.outputSchema);
+      eq = eq && this.cost == other.cost;
+
+      return eq;
+	  } else {
+	    return false;
+	  }
+  }
+
+  public boolean deepEquals(Object o) {
+    return equals(o);
+  }
+
+	@Override
+	public Object clone() throws CloneNotSupportedException {
+	  LogicalNode node = (LogicalNode)super.clone();
+    node.pid = pid;
+	  node.type = type;
+	  node.inputSchema =  (Schema) (inputSchema != null ? inputSchema.clone() : null);
+	  node.outputSchema = (Schema) (outputSchema != null ? outputSchema.clone() : null);
+	  return node;
+	}
+
+  @Override
+  public String toJson() {
+    return CoreGsonHelper.toJson(this, LogicalNode.class);
+  }
+
+	public abstract void preOrder(LogicalNodeVisitor visitor);
+  public abstract void postOrder(LogicalNodeVisitor visitor);
+
+  public abstract PlanString getPlanString();
+
+  public String toString() {
+    return PlannerUtil.buildExplainString(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNodeVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNodeVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNodeVisitor.java
new file mode 100644
index 0000000..5b0c1c2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNodeVisitor.java
@@ -0,0 +1,27 @@
+/**
+ * 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.planner.logical;
+
+
+public interface LogicalNodeVisitor {
+  void visit(LogicalNode node);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java
new file mode 100644
index 0000000..bacf14f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java
@@ -0,0 +1,41 @@
+/**
+ * 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.planner.logical;
+
+import org.apache.tajo.engine.planner.PlanString;
+
+public class LogicalRootNode extends UnaryNode implements Cloneable {
+  public LogicalRootNode(int pid) {
+    super(pid, NodeType.ROOT);
+  }
+  
+  public String toString() {
+    return "Logical Plan Root\n\n" + getChild().toString();
+  }
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    return new PlanString(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java
new file mode 100644
index 0000000..f498231
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java
@@ -0,0 +1,66 @@
+/**
+ * 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.planner.logical;
+
+
+import org.apache.tajo.engine.planner.AlterTablespaceNode;
+
+/**
+ * This indicates a logical node type.
+ */
+public enum NodeType {
+  ROOT(LogicalRootNode.class),
+  EXPRS(EvalExprNode.class),
+  PROJECTION(ProjectionNode.class),
+  LIMIT(LimitNode.class),
+  SORT(SortNode.class),
+  HAVING(HavingNode.class),
+  GROUP_BY(GroupbyNode.class),
+  SELECTION(SelectionNode.class),
+  JOIN(JoinNode.class),
+  UNION(UnionNode.class),
+  EXCEPT(ExceptNode.class),
+  INTERSECT(IntersectNode.class),
+  TABLE_SUBQUERY(TableSubQueryNode.class),
+  SCAN(ScanNode.class),
+  PARTITIONS_SCAN(PartitionedTableScanNode.class),
+  BST_INDEX_SCAN(IndexScanNode.class),
+  STORE(StoreTableNode.class),
+  INSERT(InsertNode.class),
+
+  CREATE_DATABASE(CreateDatabaseNode.class),
+  DROP_DATABASE(DropDatabaseNode.class),
+  CREATE_TABLE(CreateTableNode.class),
+  DROP_TABLE(DropTableNode.class),
+  ALTER_TABLESPACE (AlterTablespaceNode.class),
+  ALTER_TABLE (AlterTableNode.class);
+
+  private final Class<? extends LogicalNode> baseClass;
+
+  NodeType(Class<? extends LogicalNode> baseClass) {
+    this.baseClass = baseClass;
+  }
+
+  public Class<? extends LogicalNode> getBaseClass() {
+    return this.baseClass;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PartitionedTableScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PartitionedTableScanNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PartitionedTableScanNode.java
new file mode 100644
index 0000000..45cc578
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PartitionedTableScanNode.java
@@ -0,0 +1,155 @@
+/**
+ * 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.planner.logical;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.util.TUtil;
+
+public class PartitionedTableScanNode extends ScanNode {
+  @Expose Path [] inputPaths;
+
+  public PartitionedTableScanNode(int pid) {
+    super(pid, NodeType.PARTITIONS_SCAN);
+  }
+
+  public void init(ScanNode scanNode, Path[] inputPaths) {
+    tableDesc = scanNode.tableDesc;
+    setInSchema(scanNode.getInSchema());
+    setOutSchema(scanNode.getOutSchema());
+    this.qual = scanNode.qual;
+    this.targets = scanNode.targets;
+    this.inputPaths = inputPaths;
+  }
+
+  public void setInputPaths(Path [] paths) {
+    this.inputPaths = paths;
+  }
+
+  public Path [] getInputPaths() {
+    return inputPaths;
+  }
+	
+	public String toString() {
+    StringBuilder sb = new StringBuilder("Partitions Scan (table=").append(getTableName());
+    if (hasAlias()) {
+      sb.append(", alias=").append(alias);
+    }
+    if (hasQual()) {
+      sb.append(", filter=").append(qual);
+    }
+    sb.append(", path=").append(getTableDesc().getPath()).append(")");
+	  return sb.toString();
+	}
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(this.tableDesc, this.qual, this.targets);
+  }
+	
+	@Override
+	public boolean equals(Object obj) {
+	  if (obj instanceof PartitionedTableScanNode) {
+	    PartitionedTableScanNode other = (PartitionedTableScanNode) obj;
+	    
+	    boolean eq = super.equals(other); 
+	    eq = eq && TUtil.checkEquals(this.tableDesc, other.tableDesc);
+	    eq = eq && TUtil.checkEquals(this.qual, other.qual);
+	    eq = eq && TUtil.checkEquals(this.targets, other.targets);
+      eq = eq && TUtil.checkEquals(this.inputPaths, other.inputPaths);
+	    
+	    return eq;
+	  }	  
+	  
+	  return false;
+	}	
+	
+	@Override
+	public Object clone() throws CloneNotSupportedException {
+	  PartitionedTableScanNode unionScan = (PartitionedTableScanNode) super.clone();
+	  
+	  unionScan.tableDesc = (TableDesc) this.tableDesc.clone();
+	  
+	  if (hasQual()) {
+	    unionScan.qual = (EvalNode) this.qual.clone();
+	  }
+	  
+	  if (hasTargets()) {
+	    unionScan.targets = new Target[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        unionScan.targets[i] = (Target) targets[i].clone();
+      }
+	  }
+
+    unionScan.inputPaths = inputPaths;
+
+    return unionScan;
+	}
+	
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+	
+	public void postOrder(LogicalNodeVisitor visitor) {        
+    visitor.visit(this);
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this).appendTitle(" on " + getTableName());
+    if (hasAlias()) {
+      planStr.appendTitle(" as ").appendTitle(alias);
+    }
+
+    if (hasQual()) {
+      planStr.addExplan("filter: ").appendExplain(this.qual.toString());
+    }
+
+    if (hasTargets()) {
+      planStr.addExplan("target list: ");
+      boolean first = true;
+      for (Target target : targets) {
+        if (!first) {
+          planStr.appendExplain(", ");
+        }
+        planStr.appendExplain(target.toString());
+        first = false;
+      }
+    }
+
+    planStr.addDetail("out schema: ").appendDetail(getOutSchema().toString());
+    planStr.addDetail("in schema: ").appendDetail(getInSchema().toString());
+
+    if (inputPaths != null) {
+      planStr.addExplan("num of filtered paths: ").appendExplain(""+ inputPaths.length);
+      int i = 0;
+      for (Path path : inputPaths) {
+        planStr.addDetail((i++) + ": ").appendDetail(path.toString());
+      }
+    }
+
+    return planStr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PersistentStoreNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PersistentStoreNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PersistentStoreNode.java
new file mode 100644
index 0000000..9d2acf6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/PersistentStoreNode.java
@@ -0,0 +1,90 @@
+/**
+ * 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.planner.logical;
+
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.util.TUtil;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+
+
+/**
+ * <code>PersistentStoreNode</code> an expression for a persistent data store step.
+ * This includes some basic information for materializing data.
+ */
+public abstract class PersistentStoreNode extends UnaryNode implements Cloneable {
+  @Expose protected StoreType storageType = StoreType.CSV;
+  @Expose protected Options options;
+
+  protected PersistentStoreNode(int pid, NodeType nodeType) {
+    super(pid, nodeType);
+  }
+
+  public void setStorageType(StoreType storageType) {
+    this.storageType = storageType;
+  }
+
+  public StoreType getStorageType() {
+    return this.storageType;
+  }
+
+  public boolean hasOptions() {
+    return this.options != null;
+  }
+
+  public Options getOptions() {
+    return this.options;
+  }
+
+  public void setOptions(Options options) {
+    this.options = options;
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+    planStr.addExplan("Store type: " + storageType);
+
+    return planStr;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof PersistentStoreNode) {
+      PersistentStoreNode other = (PersistentStoreNode) obj;
+      boolean eq = super.equals(other);
+      eq = eq && this.storageType.equals(other.storageType);
+      eq = eq && TUtil.checkEquals(options, other.options);
+      return eq;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    PersistentStoreNode store = (PersistentStoreNode) super.clone();
+    store.storageType = storageType != null ? storageType : null;
+    store.options = options != null ? (Options) options.clone() : null;
+    return store;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/Projectable.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/Projectable.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/Projectable.java
new file mode 100644
index 0000000..1e4bdc5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/Projectable.java
@@ -0,0 +1,73 @@
+/**
+ * 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.planner.logical;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.planner.Target;
+
+/**
+ * Projectable is an interface for a LogicalNode which has a list of targets.
+ * What a logical node has a list of targets means that the node evaluated a list of expressions.
+ * For example, {@link org.apache.tajo.engine.planner.logical.ScanNode},
+ * {@link org.apache.tajo.engine.planner.logical.JoinNode},
+ * {@link org.apache.tajo.engine.planner.logical.GroupbyNode}, and
+ * {@link org.apache.tajo.engine.planner.logical.ProjectionNode} are all <i>Projectable</i> nodes.
+ * The expression evaluation occurs only at those projectable nodes.
+ */
+public interface Projectable {
+
+  /**
+   * Get a PlanNode Id
+   * @return PlanNodeId
+   */
+  int getPID();
+
+  /**
+   * check if this node has a target list
+   * @return TRUE if this node has a target list. Otherwise, FALSE.
+   */
+  boolean hasTargets();
+
+  /**
+   * Set a target list
+   *
+   * @param targets The array of targets
+   */
+  void setTargets(Target[] targets);
+
+  /**
+   * Get a list of targets
+   *
+   * @return The array of targets
+   */
+  Target [] getTargets();
+
+  /**
+   * Get an input schema
+   * @return The input schema
+   */
+  public Schema getInSchema();
+
+  /**
+   * Get an output schema
+   *
+   * @return The output schema
+   */
+  public Schema getOutSchema();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java
new file mode 100644
index 0000000..e9fd803
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java
@@ -0,0 +1,114 @@
+/**
+ * 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.planner.logical;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.util.TUtil;
+
+public class ProjectionNode extends UnaryNode implements Projectable {
+  /**
+   * the targets are always filled even if the query is 'select *'
+   */
+  @Expose	private Target [] targets;
+  @Expose private boolean distinct = false;
+
+	public ProjectionNode(int pid) {
+		super(pid, NodeType.PROJECTION);
+	}
+
+  public boolean hasTargets() {
+    return this.targets != null;
+  }
+
+  @Override
+  public void setTargets(Target[] targets) {
+    this.targets = targets;
+    this.setOutSchema(PlannerUtil.targetToSchema(targets));
+  }
+
+  @Override
+  public Target [] getTargets() {
+    return this.targets;
+  }
+	
+	public void setChild(LogicalNode subNode) {
+	  super.setChild(subNode);
+	}
+	
+	public String toString() {
+	  StringBuilder sb = new StringBuilder("Projection (distinct=").append(distinct);
+    if (targets != null) {
+      sb.append(", exprs=").append(TUtil.arrayToString(targets)).append(")");
+    }
+	  return sb.toString();
+	}
+	
+	@Override
+  public boolean equals(Object obj) {
+	  if (obj instanceof ProjectionNode) {
+	    ProjectionNode other = (ProjectionNode) obj;
+	    
+	    boolean b1 = super.equals(other);
+      boolean b2 = TUtil.checkEquals(targets, other.targets);
+      return b1 && b2;
+	  } else {
+	    return false;
+	  }
+	}
+
+	@Override
+  public Object clone() throws CloneNotSupportedException {
+	  ProjectionNode projNode = (ProjectionNode) super.clone();
+	  projNode.targets = targets.clone();
+	  
+	  return projNode;
+	}
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this);
+
+    if (distinct) {
+      planStr.appendTitle(" (distinct)");
+    }
+
+
+    StringBuilder sb = new StringBuilder("Targets: ");
+    if (targets != null) {
+      for (int i = 0; i < targets.length; i++) {
+        sb.append(targets[i]);
+        if (i < targets.length - 1) {
+          sb.append(", ");
+        }
+      }
+    }
+    planStr.addExplan(sb.toString());
+    if (getOutSchema() != null) {
+      planStr.addExplan("out schema: " + getOutSchema().toString());
+    }
+    if (getInSchema() != null) {
+      planStr.addExplan("in  schema: " + getInSchema().toString());
+    }
+
+    return planStr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java
new file mode 100644
index 0000000..83c16cd
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java
@@ -0,0 +1,49 @@
+/**
+ * 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.planner.logical;
+
+import org.apache.tajo.catalog.Schema;
+
+/**
+ * It provides a logical view of a relation. Regarding a table, the main difference between a logical view and a
+ * physical view is as follows:
+ *
+ * <ul>
+ * <li>In logical view, each column in the table has qualified name by table alias name. In addition, the schema of
+ * logical view will includes partition columns if we use column-partitioned tables.</li>
+ * <li>In contrast, in physical view: each column in the table has qualified name by the original table.</li>
+ * </ul>
+ */
+public abstract class RelationNode extends LogicalNode {
+
+  protected RelationNode(int pid, NodeType nodeType) {
+    super(pid, nodeType);
+    assert(nodeType == NodeType.SCAN || nodeType == NodeType.PARTITIONS_SCAN || nodeType == NodeType.TABLE_SUBQUERY);
+  }
+
+  public abstract boolean hasAlias();
+
+  public abstract String getAlias();
+
+  public abstract String getTableName();
+
+  public abstract String getCanonicalName();
+
+  public abstract Schema getTableSchema();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
new file mode 100644
index 0000000..27782a2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
@@ -0,0 +1,234 @@
+/**
+ * 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.planner.logical;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.util.TUtil;
+
+public class ScanNode extends RelationNode implements Projectable, Cloneable {
+	@Expose protected TableDesc tableDesc;
+  @Expose protected String alias;
+  @Expose protected Schema logicalSchema;
+	@Expose protected EvalNode qual;
+	@Expose protected Target[] targets;
+  @Expose protected boolean broadcastTable;
+
+  protected ScanNode(int pid, NodeType nodeType) {
+    super(pid, nodeType);
+  }
+
+  public ScanNode(int pid) {
+    super(pid, NodeType.SCAN);
+  }
+
+  public void init(TableDesc desc) {
+    this.tableDesc = desc;
+    this.setInSchema(tableDesc.getSchema());
+    this.setOutSchema(tableDesc.getSchema());
+    logicalSchema = SchemaUtil.getQualifiedLogicalSchema(tableDesc, null);
+  }
+  
+	public void init(TableDesc desc, String alias) {
+    this.tableDesc = desc;
+    this.alias = alias;
+
+    if (!CatalogUtil.isFQTableName(this.tableDesc.getName())) {
+      throw new IllegalArgumentException("the name in TableDesc must be qualified, but it is \"" +
+          desc.getName() + "\"");
+    }
+
+    String databaseName = CatalogUtil.extractQualifier(this.tableDesc.getName());
+    String qualifiedAlias = CatalogUtil.buildFQName(databaseName, alias);
+    this.setInSchema(tableDesc.getSchema());
+    this.getInSchema().setQualifier(qualifiedAlias);
+    this.setOutSchema(new Schema(getInSchema()));
+    logicalSchema = SchemaUtil.getQualifiedLogicalSchema(tableDesc, qualifiedAlias);
+	}
+	
+	public String getTableName() {
+	  return tableDesc.getName();
+	}
+
+  @Override
+	public boolean hasAlias() {
+	  return alias != null;
+	}
+
+  @Override
+  public String getAlias() {
+    return alias;
+  }
+
+  public void setBroadcastTable(boolean broadcastTable) {
+    this.broadcastTable = broadcastTable;
+  }
+
+  public boolean isBroadcastTable() {
+    return broadcastTable;
+  }
+
+  public String getCanonicalName() {
+    if (CatalogUtil.isFQTableName(this.tableDesc.getName())) {
+      String databaseName = CatalogUtil.extractQualifier(this.tableDesc.getName());
+      return hasAlias() ? CatalogUtil.buildFQName(databaseName, alias) : tableDesc.getName();
+    } else {
+      return hasAlias() ? alias : tableDesc.getName();
+    }
+  }
+
+  public Schema getTableSchema() {
+    return logicalSchema;
+  }
+
+  public Schema getPhysicalSchema() {
+    return getInSchema();
+  }
+	
+	public boolean hasQual() {
+	  return qual != null;
+	}
+	
+	public EvalNode getQual() {
+	  return this.qual;
+	}
+	
+	public void setQual(EvalNode evalTree) {
+	  this.qual = evalTree;
+	}
+
+  @Override
+	public boolean hasTargets() {
+	  return this.targets != null;
+	}
+
+  @Override
+	public void setTargets(Target [] targets) {
+	  this.targets = targets;
+    setOutSchema(PlannerUtil.targetToSchema(targets));
+	}
+
+  @Override
+	public Target [] getTargets() {
+	  return this.targets;
+	}
+
+  public TableDesc getTableDesc() {
+    return tableDesc;
+  }
+	
+	public String toString() {
+    StringBuilder sb = new StringBuilder("Scan (table=").append(getTableName());
+    if (hasAlias()) {
+      sb.append(", alias=").append(alias);
+    }
+    if (hasQual()) {
+      sb.append(", filter=").append(qual);
+    }
+    sb.append(", path=").append(getTableDesc().getPath()).append(")");
+    return sb.toString();
+	}
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(this.tableDesc, this.qual, this.targets);
+  }
+	
+	@Override
+	public boolean equals(Object obj) {
+	  if (obj instanceof ScanNode) {
+	    ScanNode other = (ScanNode) obj;
+	    
+	    boolean eq = super.equals(other); 
+	    eq = eq && TUtil.checkEquals(this.tableDesc, other.tableDesc);
+	    eq = eq && TUtil.checkEquals(this.qual, other.qual);
+	    eq = eq && TUtil.checkEquals(this.targets, other.targets);
+	    
+	    return eq;
+	  }	  
+	  
+	  return false;
+	}	
+	
+	@Override
+	public Object clone() throws CloneNotSupportedException {
+	  ScanNode scanNode = (ScanNode) super.clone();
+	  
+	  scanNode.tableDesc = (TableDesc) this.tableDesc.clone();
+	  
+	  if (hasQual()) {
+	    scanNode.qual = (EvalNode) this.qual.clone();
+	  }
+	  
+	  if (hasTargets()) {
+	    scanNode.targets = new Target[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        scanNode.targets[i] = (Target) targets[i].clone();
+      }
+	  }
+	  
+	  return scanNode;
+	}
+	
+  @Override
+  public void preOrder(LogicalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+	
+	public void postOrder(LogicalNodeVisitor visitor) {        
+    visitor.visit(this);
+  }
+
+  @Override
+  public PlanString getPlanString() {
+    PlanString planStr = new PlanString(this).appendTitle(" on ").appendTitle(getTableName());
+    if (hasAlias()) {
+      planStr.appendTitle(" as ").appendTitle(alias);
+    }
+
+    if (hasQual()) {
+      planStr.addExplan("filter: ").appendExplain(this.qual.toString());
+    }
+
+    if (hasTargets()) {
+      planStr.addExplan("target list: ");
+      boolean first = true;
+      for (Target target : targets) {
+        if (!first) {
+          planStr.appendExplain(", ");
+        }
+        planStr.appendExplain(target.toString());
+        first = false;
+      }
+    }
+
+    planStr.addDetail("out schema: ").appendDetail(getOutSchema().toString());
+    planStr.addDetail("in schema: ").appendDetail(getInSchema().toString());
+
+    return planStr;
+  }
+}


[47/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/BinaryEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/BinaryEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BinaryEval.java
new file mode 100644
index 0000000..d362927
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BinaryEval.java
@@ -0,0 +1,205 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+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 static org.apache.tajo.common.TajoDataTypes.Type;
+
+public class BinaryEval extends EvalNode implements Cloneable {
+  @Expose private DataType returnType = null;
+
+  /**
+   * @param type
+   */
+  public BinaryEval(EvalType type, EvalNode left, EvalNode right) {
+    super(type, left, right);
+    Preconditions.checkNotNull(type);
+    Preconditions.checkNotNull(left);
+    Preconditions.checkNotNull(right);
+
+    if(
+        type == EvalType.AND ||
+            type == EvalType.OR ||
+            type == EvalType.EQUAL ||
+            type == EvalType.NOT_EQUAL ||
+            type == EvalType.LTH ||
+            type == EvalType.GTH ||
+            type == EvalType.LEQ ||
+            type == EvalType.GEQ ) {
+      this.returnType = CatalogUtil.newSimpleDataType(Type.BOOLEAN);
+    } else if (
+        type == EvalType.PLUS ||
+            type == EvalType.MINUS ||
+            type == EvalType.MULTIPLY ||
+            type == EvalType.DIVIDE ||
+            type == EvalType.MODULAR ) {
+      this.returnType = determineType(left.getValueType(), right.getValueType());
+
+    } else if (type == EvalType.CONCATENATE) {
+      this.returnType = CatalogUtil.newSimpleDataType(Type.TEXT);
+    }
+  }
+
+  public BinaryEval(PartialBinaryExpr expr) {
+    this(expr.type, expr.leftExpr, expr.rightExpr);
+  }
+
+  /**
+   * This is verified by ExprsVerifier.checkArithmeticOperand().
+   */
+  private DataType determineType(DataType left, DataType right) throws InvalidEvalException {
+    switch (left.getType()) {
+    case INT4: {
+      switch(right.getType()) {
+      case INT2:
+      case INT4: return CatalogUtil.newSimpleDataType(Type.INT4);
+      case INT8: return CatalogUtil.newSimpleDataType(Type.INT8);
+      case FLOAT4: return CatalogUtil.newSimpleDataType(Type.FLOAT4);
+      case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+      }
+    }
+
+    case INT8: {
+      switch(right.getType()) {
+      case INT2:
+      case INT4:
+      case INT8: return CatalogUtil.newSimpleDataType(Type.INT8);
+      case FLOAT4:
+      case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+      }
+    }
+
+    case FLOAT4: {
+      switch(right.getType()) {
+      case INT2:
+      case INT4:
+      case INT8:
+      case FLOAT4:
+      case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+      }
+    }
+
+    case FLOAT8: {
+      switch(right.getType()) {
+      case INT2:
+      case INT4:
+      case INT8:
+      case FLOAT4:
+      case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+      }
+    }
+
+    default: return left;
+    }
+  }
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    Datum lhs = leftExpr.eval(schema, tuple);
+    Datum rhs = rightExpr.eval(schema, tuple);
+
+    switch(type) {
+    case AND:
+      return lhs.and(rhs);
+    case OR:
+      return lhs.or(rhs);
+
+    case EQUAL:
+      return lhs.equalsTo(rhs);
+    case NOT_EQUAL:
+      return lhs.notEqualsTo(rhs);
+    case LTH:
+      return lhs.lessThan(rhs);
+    case LEQ:
+      return lhs.lessThanEqual(rhs);
+    case GTH:
+      return lhs.greaterThan(rhs);
+    case GEQ:
+      return lhs.greaterThanEqual(rhs);
+
+    case PLUS:
+      return lhs.plus(rhs);
+    case MINUS:
+      return lhs.minus(rhs);
+    case MULTIPLY:
+      return lhs.multiply(rhs);
+    case DIVIDE:
+      return lhs.divide(rhs);
+    case MODULAR:
+      return lhs.modular(rhs);
+
+    case CONCATENATE:
+      if (lhs.type() == Type.NULL_TYPE || rhs.type() == Type.NULL_TYPE) {
+        return NullDatum.get();
+      }
+      return DatumFactory.createText(lhs.asChars() + rhs.asChars());
+    default:
+      throw new InvalidEvalException("We does not support " + type + " expression yet");
+    }
+  }
+
+  @Override
+	public String getName() {
+		return type.name();
+	}
+
+	@Override
+	public DataType getValueType() {
+	  return returnType;
+	}
+
+	public String toString() {
+		return leftExpr +" " + type.getOperatorName() + " "+rightExpr;
+	}
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof BinaryEval) {
+      BinaryEval other = (BinaryEval) obj;
+
+      boolean b1 = this.type == other.type;
+      boolean b2 = leftExpr.equals(other.leftExpr);
+      boolean b3 = rightExpr.equals(other.rightExpr);
+      return b1 && b2 && b3;
+    }
+    return false;
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(this.type, leftExpr, rightExpr);
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    BinaryEval eval = (BinaryEval) super.clone();
+    eval.returnType = returnType;
+
+    return eval;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java
new file mode 100644
index 0000000..d08bfd3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java
@@ -0,0 +1,208 @@
+/**
+ * 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.Lists;
+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.NullDatum;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.TUtil;
+
+import java.util.List;
+
+public class CaseWhenEval extends EvalNode implements GsonObject {
+  @Expose private List<IfThenEval> whens = Lists.newArrayList();
+  @Expose private EvalNode elseResult;
+
+  public CaseWhenEval() {
+    super(EvalType.CASE);
+  }
+
+  public void addWhen(EvalNode condition, EvalNode result) {
+    whens.add(new IfThenEval(condition, result));
+  }
+
+  public List<IfThenEval> getIfThenEvals() {
+    return whens;
+  }
+
+  public boolean hasElse() {
+    return this.elseResult != null;
+  }
+
+  public EvalNode getElse() {
+    return elseResult;
+  }
+
+  public void setElseResult(EvalNode elseResult) {
+    this.elseResult = elseResult;
+  }
+
+  @Override
+  public DataType getValueType() {
+    return whens.get(0).getResultExpr().getValueType();
+  }
+
+  @Override
+  public String getName() {
+    return "?";
+  }
+
+  public Datum eval(Schema schema, Tuple tuple) {
+    for (int i = 0; i < whens.size(); i++) {
+      if (whens.get(i).checkIfCondition(schema, tuple)) {
+        return whens.get(i).eval(schema, tuple);
+      }
+    }
+
+    if (elseResult != null) { // without else clause
+      return elseResult.eval(schema, tuple);
+    }
+
+    return NullDatum.get();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("CASE ");
+    for (IfThenEval when : whens) {
+     sb.append(when).append(" ");
+    }
+
+    sb.append("ELSE ").append(elseResult).append(" END");
+
+    return sb.toString();
+  }
+
+  @Override
+  public void preOrder(EvalNodeVisitor visitor) {
+    visitor.visit(this);
+    for (IfThenEval when : whens) {
+      when.preOrder(visitor);
+    }
+    if (elseResult != null) { // without else clause
+      elseResult.preOrder(visitor);
+    }
+  }
+
+  @Override
+  public void postOrder(EvalNodeVisitor visitor) {
+    for (IfThenEval when : whens) {
+      when.postOrder(visitor);
+    }
+    if (elseResult != null) { // without else clause
+      elseResult.postOrder(visitor);
+    }
+    visitor.visit(this);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof CaseWhenEval) {
+      CaseWhenEval other = (CaseWhenEval) obj;
+
+      for (int i = 0; i < other.whens.size(); i++) {
+        if (!whens.get(i).equals(other.whens.get(i))) {
+          return false;
+        }
+      }
+      return TUtil.checkEquals(elseResult, other.elseResult);
+    } else {
+      return false;
+    }
+  }
+
+  public static class IfThenEval extends EvalNode implements GsonObject {
+    @Expose private EvalNode condition;
+    @Expose private EvalNode result;
+
+    public IfThenEval(EvalNode condition, EvalNode result) {
+      super(EvalType.IF_THEN);
+      this.condition = condition;
+      this.result = result;
+    }
+
+    @Override
+    public DataType getValueType() {
+      return CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN);
+    }
+
+    @Override
+    public String getName() {
+      return "when?";
+    }
+
+    public boolean checkIfCondition(Schema schema, Tuple tuple) {
+      return condition.eval(schema, tuple).isTrue();
+    }
+
+    public Datum eval(Schema schema, Tuple tuple) {
+      return result.eval(schema, tuple);
+    }
+
+    public EvalNode getConditionExpr() {
+      return this.condition;
+    }
+
+    public EvalNode getResultExpr() {
+      return this.result;
+    }
+
+    @Override
+    public boolean equals(Object object) {
+      if (object instanceof IfThenEval) {
+        IfThenEval other = (IfThenEval) object;
+        return condition.equals(other.condition) && result.equals(other.result);
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "WHEN " + condition + " THEN " + result;
+    }
+
+    @Override
+    public String toJson() {
+      return CoreGsonHelper.toJson(IfThenEval.this, IfThenEval.class);
+    }
+
+    @Override
+    public void preOrder(EvalNodeVisitor visitor) {
+      visitor.visit(this);
+      condition.preOrder(visitor);
+      result.preOrder(visitor);
+    }
+
+    @Override
+    public void postOrder(EvalNodeVisitor visitor) {
+      condition.postOrder(visitor);
+      result.postOrder(visitor);
+      visitor.visit(this);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java
new file mode 100644
index 0000000..a024b01
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java
@@ -0,0 +1,87 @@
+/**
+ * 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.Schema;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class CastEval extends EvalNode {
+  @Expose private EvalNode operand;
+  @Expose private DataType target;
+
+  public CastEval(EvalNode operand, DataType target) {
+    super(EvalType.CAST);
+    this.operand = operand;
+    this.target = target;
+  }
+
+  public EvalNode getOperand() {
+    return operand;
+  }
+
+  @Override
+  public DataType getValueType() {
+    return target;
+  }
+
+  @Override
+  public String getName() {
+    return target.getType().name();
+  }
+
+  public Datum eval(Schema schema, Tuple tuple) {
+    Datum operandDatum = operand.eval(schema, tuple);
+    if (operandDatum.isNull()) {
+      return operandDatum;
+    }
+
+    return DatumFactory.cast(operandDatum, target);
+  }
+
+  public String toString() {
+    return "CAST (" + operand + " AS " + target.getType() + ")";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    boolean valid = obj != null && obj instanceof CastEval;
+    if (valid) {
+      CastEval another = (CastEval) obj;
+      return operand.equals(another.operand) && target.equals(another.target);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void preOrder(EvalNodeVisitor visitor) {
+    visitor.visit(this);
+    operand.preOrder(visitor);
+  }
+
+  public void postOrder(EvalNodeVisitor visitor) {
+    operand.postOrder(visitor);
+    visitor.visit(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/ConstEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/ConstEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/ConstEval.java
new file mode 100644
index 0000000..2cb530d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/ConstEval.java
@@ -0,0 +1,99 @@
+/**
+ * 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.CatalogUtil;
+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 ConstEval extends EvalNode implements Comparable<ConstEval>, Cloneable {
+	@Expose Datum datum = null;
+	
+	public ConstEval(Datum datum) {
+		super(EvalType.CONST);
+		this.datum = datum;
+	}
+
+  public Datum getValue() {
+    return this.datum;
+  }
+	
+	public String toString() {
+		return datum.toString();
+	}
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    return datum;
+  }
+
+  @Override
+	public DataType getValueType() {
+    return CatalogUtil.newSimpleDataType(datum.type());
+	}
+
+	@Override
+	public String getName() {
+		return this.datum.toString();
+	}
+	
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof ConstEval) {
+      ConstEval other = (ConstEval) obj;
+
+      if (this.type == other.type && this.datum.equals(other.datum)) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(type, datum.type(), datum);
+  }
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    ConstEval eval = (ConstEval) super.clone();
+    eval.datum = datum;
+    
+    return eval;
+  }
+
+  @Override
+  public int compareTo(ConstEval other) {    
+    return datum.compareTo(other.datum);
+  }
+  
+  @Override
+  public void preOrder(EvalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+  
+  @Override
+  public void postOrder(EvalNodeVisitor visitor) {
+    visitor.visit(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNode.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNode.java
new file mode 100644
index 0000000..1180bde
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNode.java
@@ -0,0 +1,118 @@
+/**
+ * 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.Schema;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * An annotated expression which includes actual data domains.
+ * It is also used for evaluation.
+ */
+public abstract class EvalNode implements Cloneable, GsonObject {
+	@Expose protected EvalType type;
+	@Expose protected EvalNode leftExpr;
+	@Expose protected EvalNode rightExpr;
+	
+	public EvalNode(EvalType type) {
+		this.type = type;
+	}
+	
+	public EvalNode(EvalType type, EvalNode left, EvalNode right) {
+		this(type);
+		this.leftExpr = left;
+		this.rightExpr = right;
+	}
+	
+	public EvalType getType() {
+		return this.type;
+	}
+	
+	public void setLeftExpr(EvalNode expr) {
+		this.leftExpr = expr;
+	}
+	
+	public <T extends EvalNode> T getLeftExpr() {
+		return (T) this.leftExpr;
+	}
+	
+	public void setRightExpr(EvalNode expr) {
+		this.rightExpr = expr;
+	}
+	
+	public <T extends EvalNode> T getRightExpr() {
+		return (T) this.rightExpr;
+	}
+
+  public EvalNode getExpr(int id) {
+    if (id == 0) {
+      return this.leftExpr;
+    } else if (id == 1) {
+      return this.rightExpr;
+    } else {
+      throw new ArrayIndexOutOfBoundsException("only 0 or 1 is available (" + id + " is not available)");
+    }
+  }
+	
+	public abstract DataType getValueType();
+	
+	public abstract String getName();
+	
+	public String toString() {
+		return "(" + this.type + "(" + leftExpr.toString() + " " + rightExpr.toString() + "))";
+	}
+
+  @Override
+	public String toJson() {
+    return CoreGsonHelper.toJson(this, EvalNode.class);
+	}
+	
+	public abstract <T extends Datum> T eval(Schema schema, Tuple tuple);
+
+  @Deprecated
+	public void preOrder(EvalNodeVisitor visitor) {
+	  visitor.visit(this);
+	  leftExpr.preOrder(visitor);
+	  rightExpr.preOrder(visitor);
+	}
+
+  @Deprecated
+	public void postOrder(EvalNodeVisitor visitor) {
+	  leftExpr.postOrder(visitor);
+	  rightExpr.postOrder(visitor);	  	  
+	  visitor.visit(this);
+	}
+
+  public abstract boolean equals(Object obj);
+	
+	@Override
+	public Object clone() throws CloneNotSupportedException {
+	  EvalNode node = (EvalNode) super.clone();
+	  node.type = type;
+	  node.leftExpr = leftExpr != null ? (EvalNode) leftExpr.clone() : null;
+	  node.rightExpr = rightExpr != null ? (EvalNode) rightExpr.clone() : null;
+	  
+	  return node;
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor.java
new file mode 100644
index 0000000..1680b31
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor.java
@@ -0,0 +1,24 @@
+/**
+ * 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;
+
+@Deprecated
+public interface EvalNodeVisitor {
+  public void visit(EvalNode node);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor2.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor2.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor2.java
new file mode 100644
index 0000000..024a988
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalNodeVisitor2.java
@@ -0,0 +1,71 @@
+/**
+ * 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.Stack;
+
+public interface EvalNodeVisitor2<CONTEXT, RESULT> {
+  RESULT visitChild(CONTEXT context, EvalNode evalNode, Stack<EvalNode> stack);
+
+  // Column and Value reference expressions
+  RESULT visitConst(CONTEXT context, ConstEval evalNode, Stack<EvalNode> stack);
+  RESULT visitRowConstant(CONTEXT context, RowConstantEval evalNode, Stack<EvalNode> stack);
+  RESULT visitField(CONTEXT context, Stack<EvalNode> stack, FieldEval evalNode);
+
+  // Arithmetic expression
+  RESULT visitPlus(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitMinus(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitMultiply(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitDivide(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitModular(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+
+  // Logical Predicates
+  RESULT visitAnd(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitOr(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitNot(CONTEXT context, NotEval evalNode, Stack<EvalNode> stack);
+
+  // Comparison Predicates
+  RESULT visitEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitNotEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitLessThan(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitLessThanOrEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitGreaterThan(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+  RESULT visitGreaterThanOrEqual(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+
+  // Other Predicates
+  RESULT visitIsNull(CONTEXT context, IsNullEval evalNode, Stack<EvalNode> stack);
+  RESULT visitBetween(CONTEXT context, BetweenPredicateEval evalNode, Stack<EvalNode> stack);
+  RESULT visitCaseWhen(CONTEXT context, CaseWhenEval evalNode, Stack<EvalNode> stack);
+  RESULT visitIfThen(CONTEXT context, CaseWhenEval.IfThenEval evalNode, Stack<EvalNode> stack);
+  RESULT visitInPredicate(CONTEXT context, InEval evalNode, Stack<EvalNode> stack);
+
+  // String operator and Pattern matching predicates
+  RESULT visitLike(CONTEXT context, LikePredicateEval evalNode, Stack<EvalNode> stack);
+  RESULT visitSimilarTo(CONTEXT context, SimilarToPredicateEval evalNode, Stack<EvalNode> stack);
+  RESULT visitRegex(CONTEXT context, RegexPredicateEval evalNode, Stack<EvalNode> stack);
+  RESULT visitConcatenate(CONTEXT context, BinaryEval evalNode, Stack<EvalNode> stack);
+
+  // Functions
+  RESULT visitFuncCall(CONTEXT context, GeneralFunctionEval evalNode, Stack<EvalNode> stack);
+  RESULT visitAggrFuncCall(CONTEXT context, AggregationFunctionCallEval evalNode, Stack<EvalNode> stack);
+
+  RESULT visitSigned(CONTEXT context, SignedEval signedEval, Stack<EvalNode> stack);
+
+  RESULT visitCast(CONTEXT context, CastEval signedEval, Stack<EvalNode> stack);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeFactory.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeFactory.java
new file mode 100644
index 0000000..aa72f25
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeFactory.java
@@ -0,0 +1,32 @@
+/**
+ * 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.datum.Datum;
+
+public class EvalTreeFactory {
+	public static ConstEval newConst(Datum datum) {
+		return new ConstEval(datum);
+	}
+	
+	public static BinaryEval create(EvalType type, EvalNode e1,
+	    EvalNode e2) {
+		return new BinaryEval(type, e1, e2);
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/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
new file mode 100644
index 0000000..7dcc26a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
@@ -0,0 +1,350 @@
+/**
+ * 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.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.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 void replace(EvalNode expr, EvalNode targetExpr, EvalNode tobeReplaced) {
+    EvalReplaceVisitor replacer = new EvalReplaceVisitor(targetExpr, tobeReplaced);
+    replacer.visitChild(null, expr, new Stack<EvalNode>());
+  }
+
+  public static class EvalReplaceVisitor extends BasicEvalNodeVisitor<EvalNode, EvalNode> {
+    private EvalNode target;
+    private EvalNode tobeReplaced;
+
+    public EvalReplaceVisitor(EvalNode target, EvalNode tobeReplaced) {
+      this.target = target;
+      this.tobeReplaced = tobeReplaced;
+    }
+
+    @Override
+    public EvalNode visitChild(EvalNode context, EvalNode evalNode, Stack<EvalNode> stack) {
+      super.visitChild(context, evalNode, stack);
+
+      if (evalNode.equals(target)) {
+        EvalNode parent = stack.peek();
+
+        if (parent.getLeftExpr().equals(evalNode)) {
+          parent.setLeftExpr(tobeReplaced);
+        }
+        if (parent.getRightExpr().equals(evalNode)) {
+          parent.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 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<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.getSimpleName())) {
+        exprSet.add(field);
+      }
+      break;
+    case CONST:
+      return;
+    default: 
+      _containColumnRef(expr.getLeftExpr(), target, exprSet);
+      _containColumnRef(expr.getRightExpr(), target, exprSet);
+    }    
+  }
+
+  /**
+   * If a given expression is join condition, it returns TRUE. Otherwise, it returns FALSE.
+   *
+   * @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) {
+    boolean joinComparator;
+    if (includeThetaJoin) {
+      joinComparator = AlgebraicUtil.isComparisonOperator(expr);
+    } else {
+      joinComparator = expr.getType() == EvalType.EQUAL;
+    }
+
+    return joinComparator && expr.getLeftExpr().getType() == EvalType.FIELD &&
+        expr.getRightExpr().getType() == EvalType.FIELD;
+  }
+  
+  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 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 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/6594ac1c/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
new file mode 100644
index 0000000..71db934
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalType.java
@@ -0,0 +1,83 @@
+/**
+ * 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, "/"),
+
+  // Function
+  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 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/6594ac1c/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
new file mode 100644
index 0000000..ea2b031
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
@@ -0,0 +1,114 @@
+/**
+ * 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) {
+	    fieldId = schema.getColumnId(column.getQualifiedName());
+      if (fieldId == -1) {
+        throw new IllegalStateException("No Such Column Reference: " + column + ", schema: " + schema);
+      }
+	  }
+	  return tuple.get(fieldId);
+  }
+
+  @Override
+	public DataType getValueType() {
+		return column.getDataType();
+	}
+	
+  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/6594ac1c/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
new file mode 100644
index 0000000..de982e5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
@@ -0,0 +1,144 @@
+/**
+ * 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.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;
+		this.argEvals = argEvals;
+	}
+
+  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 setArgs(EvalNode [] args) {
+    this.argEvals = args;
+  }
+	
+	public DataType getValueType() {
+		return this.funcDesc.getReturnType();
+	}
+
+	@Override
+	public abstract Datum eval(Schema schema, Tuple tuple);
+
+	@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() + "(" + (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();
+    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);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/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
new file mode 100644
index 0000000..9446d70
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/GeneralFunctionEval.java
@@ -0,0 +1,81 @@
+/**
+ * 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/6594ac1c/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
new file mode 100644
index 0000000..51097e6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/InEval.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.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/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidCastException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidCastException.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidCastException.java
new file mode 100644
index 0000000..b907f06
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidCastException.java
@@ -0,0 +1,36 @@
+/**
+ * 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 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/tajo/blob/6594ac1c/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
new file mode 100644
index 0000000..96eda91
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/InvalidEvalException.java
@@ -0,0 +1,36 @@
+/**
+ * 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/6594ac1c/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
new file mode 100644
index 0000000..5704aa5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java
@@ -0,0 +1,84 @@
+/**
+ * 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 BinaryEval {
+  // 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, DUMMY_EVAL);
+    this.isNot = not;
+  }
+
+  @Override
+  public DataType getValueType() {
+    return RES_TYPE;
+  }
+
+  @Override
+  public String getName() {
+    return "?";
+  }
+
+  @Override
+  public String toString() {
+    return leftExpr + " IS " + (isNot ? "NOT NULL" : "NULL");
+  }
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    boolean isNull = leftExpr.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/6594ac1c/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
new file mode 100644
index 0000000..bfac33e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/LikePredicateEval.java
@@ -0,0 +1,50 @@
+/**
+ * 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, 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/6594ac1c/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
new file mode 100644
index 0000000..1a16af4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java
@@ -0,0 +1,92 @@
+/**
+ * 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 NotEval extends EvalNode implements Cloneable {
+  @Expose private EvalNode childEval;
+  private static final DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN);
+
+  public NotEval(EvalNode childEval) {
+    super(EvalType.NOT);
+    this.childEval = childEval;
+  }
+
+  public EvalNode getChild() {
+    return childEval;
+  }
+
+  @Override
+  public DataType getValueType() {
+    return RES_TYPE;
+  }
+
+  @Override
+  public String getName() {
+    return "?";
+  }
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    Datum datum = childEval.eval(schema, tuple);
+    return !datum.isNull() ? DatumFactory.createBool(!datum.asBool()) : datum;
+  }
+
+  @Override
+  public String toString() {
+    return "NOT " + childEval.toString();
+  }
+
+  @Override
+  public void preOrder(EvalNodeVisitor visitor) {
+    visitor.visit(this);
+    childEval.preOrder(visitor);
+  }
+
+  @Override
+  public void postOrder(EvalNodeVisitor visitor) {    
+    childEval.postOrder(visitor);
+    visitor.visit(this);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof NotEval) {
+      NotEval other = (NotEval) obj;
+      return this.childEval.equals(other.childEval);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    NotEval eval = (NotEval) super.clone();
+    eval.childEval = (EvalNode) this.childEval.clone();
+    return eval;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/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
new file mode 100644
index 0000000..0f4411d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java
@@ -0,0 +1,70 @@
+/**
+ * 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.storage.Tuple;
+
+public class PartialBinaryExpr extends EvalNode {
+  
+  public PartialBinaryExpr(EvalType type) {
+    super(type);
+  }
+
+  public PartialBinaryExpr(EvalType type, EvalNode left, EvalNode right) {
+    super(type, left, right);
+  }
+
+  @Override
+  public DataType getValueType() {
+    return null;
+  }
+
+  @Override
+  public String getName() {
+    return "nonamed";
+  }
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    throw new InvalidEvalException(
+        "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/6594ac1c/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
new file mode 100644
index 0000000..8d78b0b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/PatternMatchPredicateEval.java
@@ -0,0 +1,82 @@
+/**
+ * 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;
+
+  @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/6594ac1c/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
new file mode 100644
index 0000000..f1e0241
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/RegexPredicateEval.java
@@ -0,0 +1,53 @@
+/**
+ * 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/6594ac1c/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
new file mode 100644
index 0000000..646a627
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/RowConstantEval.java
@@ -0,0 +1,79 @@
+/**
+ * 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 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/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
new file mode 100644
index 0000000..3c2eeb4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
@@ -0,0 +1,99 @@
+/**
+ * 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.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 EvalNode implements Cloneable {
+  @Expose private EvalNode childEval;
+  @Expose private boolean negative;
+
+  public SignedEval(boolean negative, EvalNode childEval) {
+    super(EvalType.SIGNED);
+    this.negative = negative;
+    this.childEval = childEval;
+  }
+
+  public boolean isNegative() {
+    return negative;
+  }
+
+  public EvalNode getChild() {
+    return childEval;
+  }
+
+  @Override
+  public DataType getValueType() {
+    return childEval.getValueType();
+  }
+
+  @Override
+  public String getName() {
+    return "?";
+  }
+
+  @Override
+  public Datum eval(Schema schema, Tuple tuple) {
+    NumericDatum result = childEval.eval(schema, tuple);
+    if (negative) {
+      return result.inverseSign();
+    }
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return (negative ? "-" : "+") + childEval.toString();
+  }
+
+  @Override
+  public void preOrder(EvalNodeVisitor visitor) {
+    visitor.visit(this);
+    childEval.preOrder(visitor);
+  }
+
+  @Override
+  public void postOrder(EvalNodeVisitor visitor) {    
+    childEval.postOrder(visitor);
+    visitor.visit(this);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof SignedEval) {
+      SignedEval other = (SignedEval) obj;
+      return negative == other.negative && this.childEval.equals(other.childEval);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    SignedEval eval = (SignedEval) super.clone();
+    eval.negative = negative;
+    eval.childEval = (EvalNode) this.childEval.clone();
+    return eval;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/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
new file mode 100644
index 0000000..9ac0e62
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/SimilarToPredicateEval.java
@@ -0,0 +1,43 @@
+/**
+ * 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) {
+    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


[39/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprNormalizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprNormalizer.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprNormalizer.java
new file mode 100644
index 0000000..9030629
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprNormalizer.java
@@ -0,0 +1,257 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.catalog.CatalogUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Stack;
+
+/**
+ * ExprNormalizer performs two kinds of works:
+ *
+ * <h3>1. Duplication Removal.</h3>
+ *
+ * For example, assume a simple query as follows:
+ * <pre>
+ *   select price * rate as total_price, ..., order by price * rate
+ * </pre>
+ *
+ * The expression <code>price * rate</code> is duplicated in both select list and order by clause.
+ * Against those cases, ExprNormalizer removes duplicated expressions and replaces one with one reference.
+ * In the case, ExprNormalizer replaces price * rate with total_price reference.
+ *
+ * <h3>2. Dissection of Expression</h3>
+ *
+ * A expression can be a complex expressions, including a mixed of scalar and aggregation expressions.
+ * For example, assume an aggregation query as follows:
+ * <pre>
+ *   select sum(price * rate) * (1 - avg(discount_rate))), ...
+ * </pre>
+ *
+ * In this case, ExprNormalizer dissects the expression 'sum(price * rate) * (1 - avg(discount_rate)))'
+ * into the following expressions:
+ * <ul>
+ *   <li>$1 = price * rage</li>
+ *   <li>$2 = sum($1)</li>
+ *   <li>$3 = avg(discount_rate)</li>
+ *   <li>$4 = $2 * (1 - $3)</li>
+ * </ul>
+ *
+ * It mainly two advantages. Firstly, it makes complex expression evaluations easier across multiple physical executors.
+ * Second, it gives move opportunities to remove duplicated expressions.
+ *
+ * <h3>3. Name Normalization</h3>
+ *
+ * Users can use qualified column names, unqualified column names or aliased column references.
+ *
+ * Consider the following example:
+ *
+ * <pre>
+ *   select rate_a as total_rate, rate_a * 100, table1.rate_a, ... WHERE total_rate * 100
+ * </pre>
+ *
+ * <code>total_rate</code>, <code>rate_a</code>, and <code>table1.rate_a</code> are all the same references. But,
+ * they have different forms. Due to their different forms, duplication removal can be hard.
+ *
+ * In order to solve this problem, ExprNormalizer normalizes all column references as qualified names while it keeps
+ * its points..
+ */
+class ExprNormalizer extends SimpleAlgebraVisitor<ExprNormalizer.ExprNormalizedResult, Object> {
+
+  public static class ExprNormalizedResult {
+    private final LogicalPlan plan;
+    private final LogicalPlan.QueryBlock block;
+
+    Expr baseExpr; // outmost expressions, which can includes one or more references of the results of aggregation
+                   // function.
+    List<NamedExpr> aggExprs = new ArrayList<NamedExpr>(); // aggregation functions
+    List<NamedExpr> scalarExprs = new ArrayList<NamedExpr>(); // scalar expressions which can be referred
+
+    private ExprNormalizedResult(LogicalPlanner.PlanContext context) {
+      this.plan = context.plan;
+      this.block = context.queryBlock;
+    }
+
+    @Override
+    public String toString() {
+      return baseExpr.toString() + ", agg=" + aggExprs.size() + ", scalar=" + scalarExprs.size();
+    }
+  }
+
+  public ExprNormalizedResult normalize(LogicalPlanner.PlanContext context, Expr expr) throws PlanningException {
+    ExprNormalizedResult exprNormalizedResult = new ExprNormalizedResult(context);
+    Stack<Expr> stack = new Stack<Expr>();
+    stack.push(expr);
+    visit(exprNormalizedResult, new Stack<Expr>(), expr);
+    exprNormalizedResult.baseExpr = stack.pop();
+    return exprNormalizedResult;
+  }
+
+  @Override
+  public Object visitCaseWhen(ExprNormalizedResult ctx, Stack<Expr> stack, CaseWhenPredicate expr)
+      throws PlanningException {
+    stack.push(expr);
+    for (CaseWhenPredicate.WhenExpr when : expr.getWhens()) {
+      visit(ctx, stack, when.getCondition());
+      visit(ctx, stack, when.getResult());
+
+      if (OpType.isAggregationFunction(when.getCondition().getType())) {
+        String referenceName = ctx.block.namedExprsMgr.addExpr(when.getCondition());
+        ctx.aggExprs.add(new NamedExpr(when.getCondition(), referenceName));
+        when.setCondition(new ColumnReferenceExpr(referenceName));
+      }
+
+      if (OpType.isAggregationFunction(when.getResult().getType())) {
+        String referenceName = ctx.block.namedExprsMgr.addExpr(when.getResult());
+        ctx.aggExprs.add(new NamedExpr(when.getResult(), referenceName));
+        when.setResult(new ColumnReferenceExpr(referenceName));
+      }
+    }
+
+    if (expr.hasElseResult()) {
+      visit(ctx, stack, expr.getElseResult());
+      if (OpType.isAggregationFunction(expr.getElseResult().getType())) {
+        String referenceName = ctx.block.namedExprsMgr.addExpr(expr.getElseResult());
+        ctx.aggExprs.add(new NamedExpr(expr.getElseResult(), referenceName));
+        expr.setElseResult(new ColumnReferenceExpr(referenceName));
+      }
+    }
+    stack.pop();
+    return expr;
+  }
+
+  @Override
+  public Expr visitUnaryOperator(ExprNormalizedResult ctx, Stack<Expr> stack, UnaryOperator expr) throws PlanningException {
+    super.visitUnaryOperator(ctx, stack, expr);
+    if (OpType.isAggregationFunction(expr.getChild().getType())) {
+      // Get an anonymous column name and replace the aggregation function by the column name
+      String refName = ctx.block.namedExprsMgr.addExpr(expr.getChild());
+      ctx.aggExprs.add(new NamedExpr(expr.getChild(), refName));
+      expr.setChild(new ColumnReferenceExpr(refName));
+    }
+
+    return expr;
+  }
+
+  @Override
+  public Expr visitBinaryOperator(ExprNormalizedResult ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    super.visitBinaryOperator(ctx, stack, expr);
+
+    ////////////////////////
+    // For Left Term
+    ////////////////////////
+
+    if (OpType.isAggregationFunction(expr.getLeft().getType())) {
+      String leftRefName = ctx.block.namedExprsMgr.addExpr(expr.getLeft());
+      ctx.aggExprs.add(new NamedExpr(expr.getLeft(), leftRefName));
+      expr.setLeft(new ColumnReferenceExpr(leftRefName));
+    }
+
+
+    ////////////////////////
+    // For Right Term
+    ////////////////////////
+    if (OpType.isAggregationFunction(expr.getRight().getType())) {
+      String rightRefName = ctx.block.namedExprsMgr.addExpr(expr.getRight());
+      ctx.aggExprs.add(new NamedExpr(expr.getRight(), rightRefName));
+      expr.setRight(new ColumnReferenceExpr(rightRefName));
+    }
+
+    return expr;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Function Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public Expr visitFunction(ExprNormalizedResult ctx, Stack<Expr> stack, FunctionExpr expr) throws PlanningException {
+    stack.push(expr);
+
+    Expr param;
+    for (int i = 0; i < expr.getParams().length; i++) {
+      param = expr.getParams()[i];
+      visit(ctx, stack, param);
+
+      if (OpType.isAggregationFunction(param.getType())) {
+        String referenceName = ctx.plan.generateUniqueColumnName(param);
+        ctx.aggExprs.add(new NamedExpr(param, referenceName));
+        expr.getParams()[i] = new ColumnReferenceExpr(referenceName);
+      }
+    }
+
+    stack.pop();
+
+    return expr;
+  }
+
+  @Override
+  public Expr visitGeneralSetFunction(ExprNormalizedResult ctx, Stack<Expr> stack, GeneralSetFunctionExpr expr)
+      throws PlanningException {
+    stack.push(expr);
+
+    Expr param;
+    for (int i = 0; i < expr.getParams().length; i++) {
+      param = expr.getParams()[i];
+      visit(ctx, stack, param);
+
+
+      // If parameters are all constants, we don't need to dissect an aggregation expression into two parts:
+      // function and parameter parts.
+      if (!OpType.isLiteral(param.getType()) && param.getType() != OpType.Column) {
+        String referenceName = ctx.block.namedExprsMgr.addExpr(param);
+        ctx.scalarExprs.add(new NamedExpr(param, referenceName));
+        expr.getParams()[i] = new ColumnReferenceExpr(referenceName);
+      }
+    }
+    stack.pop();
+    return expr;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Literal Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public Expr visitCastExpr(ExprNormalizedResult ctx, Stack<Expr> stack, CastExpr expr) throws PlanningException {
+    super.visitCastExpr(ctx, stack, expr);
+    if (OpType.isAggregationFunction(expr.getType())) {
+      String referenceName = ctx.block.namedExprsMgr.addExpr(expr.getChild());
+      ctx.aggExprs.add(new NamedExpr(expr.getChild(), referenceName));
+      expr.setChild(new ColumnReferenceExpr(referenceName));
+    }
+    return expr;
+  }
+
+  @Override
+  public Expr visitColumnReference(ExprNormalizedResult ctx, Stack<Expr> stack, ColumnReferenceExpr expr)
+      throws PlanningException {
+    // if a column reference is not qualified, it finds and sets the qualified column name.
+    if (!(expr.hasQualifier() && CatalogUtil.isFQTableName(expr.getQualifier()))) {
+      if (!ctx.block.namedExprsMgr.contains(expr.getCanonicalName())) {
+        String normalized = ctx.plan.getNormalizedColumnName(ctx.block, expr);
+        expr.setName(normalized);
+      }
+    }
+    return expr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprsVerifier.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprsVerifier.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprsVerifier.java
new file mode 100644
index 0000000..551393c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprsVerifier.java
@@ -0,0 +1,216 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+
+import java.util.Set;
+import java.util.Stack;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+import static org.apache.tajo.common.TajoDataTypes.Type;
+
+/**
+ * It verifies one predicate or expression with the semantic and data type checks as follows:
+ * <ul>
+ *   <ul>Both expressions in a binary expression are compatible to each other</ul>
+ *   <ul>All column references of one expression are avilable at this node</ul>
+ * </ul>
+ */
+public class ExprsVerifier extends BasicEvalNodeVisitor<VerificationState, EvalNode> {
+  private static final ExprsVerifier instance;
+
+  static {
+    instance = new ExprsVerifier();
+  }
+
+  public static VerificationState verify(VerificationState state, LogicalNode currentNode, EvalNode expression)
+      throws PlanningException {
+    instance.visitChild(state, expression, new Stack<EvalNode>());
+    Set<Column> referredColumns = EvalTreeUtil.findUniqueColumns(expression);
+    for (Column referredColumn : referredColumns) {
+      if (!currentNode.getInSchema().contains(referredColumn)) {
+        throw new PlanningException("Invalid State: " + referredColumn + " cannot be accessible at Node ("
+            + currentNode.getPID() + ")");
+      }
+    }
+    return state;
+  }
+
+  /**
+   * It checks the compatibility of two data types.
+   */
+  private static boolean isCompatibleType(DataType dataType1, DataType dataType2) {
+    if (checkNumericType(dataType1) && checkNumericType(dataType2)) {
+      return true;
+    }
+
+    if (checkTextData(dataType1) && checkTextData(dataType2)) {
+      return true;
+    }
+
+    if (checkDateTime(dataType1) && checkDateTime(dataType2)) {
+      return true;
+    }
+
+    if (checkNetworkType(dataType1) && checkNetworkType(dataType2)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * It checks both expressions in a comparison operator are compatible to each other.
+   */
+  private static void verifyComparisonOperator(VerificationState state, BinaryEval expr) {
+    DataType leftType = expr.getLeftExpr().getValueType();
+    DataType rightType = expr.getRightExpr().getValueType();
+    if (!isCompatibleType(leftType, rightType)) {
+      state.addVerification("No operator matches the given name and argument type(s): " + expr.toString());
+    }
+  }
+
+  public EvalNode visitEqual(VerificationState context, BinaryEval expr, Stack<EvalNode> stack) {
+    super.visitEqual(context, expr, stack);
+    verifyComparisonOperator(context, expr);
+    return expr;
+  }
+
+  public EvalNode visitNotEqual(VerificationState context, BinaryEval expr, Stack<EvalNode> stack) {
+    super.visitNotEqual(context, expr, stack);
+    verifyComparisonOperator(context, expr);
+    return expr;
+  }
+
+  @Override
+  public EvalNode visitLessThan(VerificationState context, BinaryEval expr, Stack<EvalNode> stack) {
+    super.visitLessThan(context, expr, stack);
+    verifyComparisonOperator(context, expr);
+    return expr;
+  }
+
+  @Override
+  public EvalNode visitLessThanOrEqual(VerificationState context, BinaryEval expr, Stack<EvalNode> stack) {
+    super.visitLessThanOrEqual(context, expr, stack);
+    verifyComparisonOperator(context, expr);
+    return expr;
+  }
+
+  @Override
+  public EvalNode visitGreaterThan(VerificationState context, BinaryEval expr, Stack<EvalNode> stack) {
+    super.visitGreaterThan(context, expr, stack);
+    verifyComparisonOperator(context, expr);
+    return expr;
+  }
+
+  @Override
+  public EvalNode visitGreaterThanOrEqual(VerificationState context, BinaryEval expr, Stack<EvalNode> stack) {
+    super.visitGreaterThanOrEqual(context, expr, stack);
+    verifyComparisonOperator(context, expr);
+    return expr;
+  }
+
+  private static void checkDivisionByZero(VerificationState state, BinaryEval evalNode) {
+    if (evalNode.getRightExpr().getType() == EvalType.CONST) {
+      ConstEval constEval = evalNode.getRightExpr();
+      if (constEval.getValue().asFloat8() == 0) {
+        state.addVerification("division by zero");
+      }
+    }
+  }
+
+  private static void checkArithmeticOperand(VerificationState state, BinaryEval evalNode) {
+    EvalNode leftExpr = evalNode.getLeftExpr();
+    EvalNode rightExpr = evalNode.getRightExpr();
+    if (!(checkNumericType(leftExpr.getValueType()) && checkNumericType(rightExpr.getValueType()))) {
+      state.addVerification("No operator matches the given name and argument type(s): " + evalNode.toString());
+    }
+  }
+
+  private static boolean checkNetworkType(DataType dataType) {
+    return dataType.getType() == Type.INET4 || dataType.getType() == Type.INET6;
+  }
+
+  private static boolean checkNumericType(DataType dataType) {
+    int typeNumber = dataType.getType().getNumber();
+    return Type.INT1.getNumber() < typeNumber && typeNumber <= Type.NUMERIC.getNumber();
+  }
+
+  private static boolean checkTextData(DataType dataType) {
+    int typeNumber = dataType.getType().getNumber();
+    return Type.CHAR.getNumber() < typeNumber && typeNumber <= Type.TEXT.getNumber();
+  }
+
+  private static boolean checkDateTime(DataType dataType) {
+    int typeNumber = dataType.getType().getNumber();
+    return (Type.DATE.getNumber() < typeNumber && typeNumber <= Type.INTERVAL.getNumber()) ||
+        (Type.TIMEZ.getNumber() < typeNumber && typeNumber <= Type.TIMESTAMPZ.getNumber());
+  }
+
+  @Override
+  public EvalNode visitPlus(VerificationState context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    super.visitPlus(context, evalNode, stack);
+    checkArithmeticOperand(context, evalNode);
+    return evalNode;
+  }
+
+  @Override
+  public EvalNode visitMinus(VerificationState context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    super.visitMinus(context, evalNode, stack);
+    checkArithmeticOperand(context, evalNode);
+    return evalNode;
+  }
+
+  @Override
+  public EvalNode visitMultiply(VerificationState context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    super.visitMultiply(context, evalNode, stack);
+    checkArithmeticOperand(context, evalNode);
+    return evalNode;
+  }
+
+  @Override
+  public EvalNode visitDivide(VerificationState context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    super.visitDivide(context, evalNode, stack);
+    checkArithmeticOperand(context, evalNode);
+    checkDivisionByZero(context, evalNode);
+    return evalNode;
+  }
+
+  @Override
+  public EvalNode visitModular(VerificationState context, BinaryEval evalNode, Stack<EvalNode> stack) {
+    super.visitDivide(context, evalNode, stack);
+    checkArithmeticOperand(context, evalNode);
+    checkDivisionByZero(context, evalNode);
+    return evalNode;
+  }
+
+  @Override
+  public EvalNode visitFuncCall(VerificationState context, GeneralFunctionEval evalNode, Stack<EvalNode> stack) {
+    super.visitFuncCall(context, evalNode, stack);
+    if (evalNode.getArgs() != null) {
+      for (EvalNode param : evalNode.getArgs()) {
+        visitChild(context, param, stack);
+      }
+    }
+    return evalNode;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/GroupElement.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/GroupElement.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/GroupElement.java
new file mode 100644
index 0000000..3fb05c2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/GroupElement.java
@@ -0,0 +1,64 @@
+/**
+ * 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.planner;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.algebra.Aggregation;
+import org.apache.tajo.engine.eval.EvalNode;
+
+public class GroupElement implements Cloneable {
+  @Expose private Aggregation.GroupType type;
+  @Expose private EvalNode [] groupingSets;
+
+  @SuppressWarnings("unused")
+  public GroupElement() {
+    // for gson
+  }
+
+  public GroupElement(Aggregation.GroupType type, EvalNode [] groupingSets) {
+    this.type = type;
+    this.groupingSets = groupingSets;
+  }
+
+  public Aggregation.GroupType getType() {
+    return this.type;
+  }
+
+  public EvalNode [] getGroupingSets() {
+    return this.groupingSets;
+  }
+
+  public String toString() {
+    Gson gson = new GsonBuilder().excludeFieldsWithoutExposeAnnotation()
+        .setPrettyPrinting().create();
+    return gson.toJson(this);
+  }
+
+  public Object clone() throws CloneNotSupportedException {
+    GroupElement groups = (GroupElement) super.clone();
+    groups.type = type;
+    groups.groupingSets = new EvalNode [groupingSets.length];
+    for (int i = 0; i < groupingSets.length; i++) {
+      groups.groupingSets[i++] = (EvalNode) groupingSets[i].clone();
+    }
+    return groups;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java
new file mode 100644
index 0000000..974dc60
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java
@@ -0,0 +1,280 @@
+/**
+ * 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.planner;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.eval.AlgebraicUtil;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.graph.DirectedGraphCursor;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.planner.logical.join.FoundJoinOrder;
+import org.apache.tajo.engine.planner.logical.join.GreedyHeuristicJoinOrderAlgorithm;
+import org.apache.tajo.engine.planner.logical.join.JoinGraph;
+import org.apache.tajo.engine.planner.logical.join.JoinOrderAlgorithm;
+import org.apache.tajo.engine.planner.rewrite.BasicQueryRewriteEngine;
+import org.apache.tajo.engine.planner.rewrite.FilterPushDownRule;
+import org.apache.tajo.engine.planner.rewrite.PartitionedTableRewriter;
+import org.apache.tajo.engine.planner.rewrite.ProjectionPushDownRule;
+
+import java.util.LinkedHashSet;
+import java.util.Set;
+import java.util.Stack;
+
+import static org.apache.tajo.engine.planner.LogicalPlan.BlockEdge;
+import static org.apache.tajo.engine.planner.logical.join.GreedyHeuristicJoinOrderAlgorithm.getCost;
+
+/**
+ * This class optimizes a logical plan.
+ */
+@InterfaceStability.Evolving
+public class LogicalOptimizer {
+  private BasicQueryRewriteEngine rulesBeforeJoinOpt;
+  private BasicQueryRewriteEngine rulesAfterToJoinOpt;
+  private JoinOrderAlgorithm joinOrderAlgorithm = new GreedyHeuristicJoinOrderAlgorithm();
+
+  public LogicalOptimizer(TajoConf systemConf) {
+    rulesBeforeJoinOpt = new BasicQueryRewriteEngine();
+    rulesBeforeJoinOpt.addRewriteRule(new FilterPushDownRule());
+
+    rulesAfterToJoinOpt = new BasicQueryRewriteEngine();
+    rulesAfterToJoinOpt.addRewriteRule(new ProjectionPushDownRule());
+    rulesAfterToJoinOpt.addRewriteRule(new PartitionedTableRewriter(systemConf));
+  }
+
+  public LogicalNode optimize(LogicalPlan plan) throws PlanningException {
+    rulesBeforeJoinOpt.rewrite(plan);
+
+    DirectedGraphCursor<String, BlockEdge> blockCursor =
+        new DirectedGraphCursor<String, BlockEdge>(plan.getQueryBlockGraph(), plan.getRootBlock().getName());
+
+    while(blockCursor.hasNext()) {
+      optimizeJoinOrder(plan, blockCursor.nextBlock());
+    }
+
+    rulesAfterToJoinOpt.rewrite(plan);
+    return plan.getRootBlock().getRoot();
+  }
+
+  private void optimizeJoinOrder(LogicalPlan plan, String blockName) throws PlanningException {
+    LogicalPlan.QueryBlock block = plan.getBlock(blockName);
+
+    if (block.hasNode(NodeType.JOIN)) {
+      String originalOrder = JoinOrderStringBuilder.buildJoinOrderString(plan, block);
+      double nonOptimizedJoinCost = JoinCostComputer.computeCost(plan, block);
+
+      // finding relations and filter expressions
+      JoinGraphContext joinGraphContext = JoinGraphBuilder.buildJoinGraph(plan, block);
+
+      // finding join order and restore remain filter order
+      FoundJoinOrder order = joinOrderAlgorithm.findBestOrder(plan, block,
+          joinGraphContext.joinGraph, joinGraphContext.relationsForProduct);
+      JoinNode newJoinNode = order.getOrderedJoin();
+      JoinNode old = PlannerUtil.findTopNode(block.getRoot(), NodeType.JOIN);
+
+      JoinTargetCollector collector = new JoinTargetCollector();
+      Set<Target> targets = new LinkedHashSet<Target>();
+      collector.visitJoin(targets, plan, block, old, new Stack<LogicalNode>());
+
+      if (targets.size() == 0) {
+        newJoinNode.setTargets(PlannerUtil.schemaToTargets(old.getOutSchema()));
+      } else {
+        newJoinNode.setTargets(targets.toArray(new Target[targets.size()]));
+      }
+
+      PlannerUtil.replaceNode(plan, block.getRoot(), old, newJoinNode);
+      String optimizedOrder = JoinOrderStringBuilder.buildJoinOrderString(plan, block);
+      block.addPlanHistory("Non-optimized join order: " + originalOrder + " (cost: " + nonOptimizedJoinCost + ")");
+      block.addPlanHistory("Optimized join order    : " + optimizedOrder + " (cost: " + order.getCost() + ")");
+    }
+  }
+
+  private static class JoinTargetCollector extends BasicLogicalPlanVisitor<Set<Target>, LogicalNode> {
+    @Override
+    public LogicalNode visitJoin(Set<Target> ctx, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode node,
+                                 Stack<LogicalNode> stack)
+        throws PlanningException {
+      super.visitJoin(ctx, plan, block, node, stack);
+
+      if (node.hasTargets()) {
+        for (Target target : node.getTargets()) {
+          ctx.add(target);
+        }
+      }
+      return node;
+    }
+  }
+
+  private static class JoinGraphContext {
+    JoinGraph joinGraph = new JoinGraph();
+    Set<EvalNode> quals = Sets.newHashSet();
+    Set<String> relationsForProduct = Sets.newHashSet();
+  }
+
+  private static class JoinGraphBuilder extends BasicLogicalPlanVisitor<JoinGraphContext, LogicalNode> {
+    private final static JoinGraphBuilder instance;
+
+    static {
+      instance = new JoinGraphBuilder();
+    }
+
+    /**
+     * This is based on the assumtion that all join and filter conditions are placed on the right join and
+     * scan operators. In other words, filter push down must be performed before this method.
+     * Otherwise, this method may build incorrectly a join graph.
+     */
+    public static JoinGraphContext buildJoinGraph(LogicalPlan plan, LogicalPlan.QueryBlock block)
+        throws PlanningException {
+      JoinGraphContext joinGraphContext = new JoinGraphContext();
+      instance.visit(joinGraphContext, plan, block);
+      return joinGraphContext;
+    }
+
+    public LogicalNode visitFilter(JoinGraphContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                   SelectionNode node, Stack<LogicalNode> stack) throws PlanningException {
+      super.visitFilter(context, plan, block, node, stack);
+      context.quals.addAll(Lists.newArrayList(AlgebraicUtil.toConjunctiveNormalFormArray(node.getQual())));
+      return node;
+    }
+
+    @Override
+    public LogicalNode visitJoin(JoinGraphContext joinGraphContext, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 JoinNode joinNode, Stack<LogicalNode> stack)
+        throws PlanningException {
+      super.visitJoin(joinGraphContext, plan, block, joinNode, stack);
+      if (joinNode.hasJoinQual()) {
+        joinGraphContext.joinGraph.addJoin(plan, block, joinNode);
+      } else {
+        LogicalNode leftChild = joinNode.getLeftChild();
+        LogicalNode rightChild = joinNode.getRightChild();
+        if (leftChild instanceof RelationNode) {
+          RelationNode rel = (RelationNode) leftChild;
+          joinGraphContext.relationsForProduct.add(rel.getCanonicalName());
+        }
+        if (rightChild instanceof RelationNode) {
+          RelationNode rel = (RelationNode) rightChild;
+          joinGraphContext.relationsForProduct.add(rel.getCanonicalName());
+        }
+      }
+      return joinNode;
+    }
+  }
+
+  public static class JoinOrderStringBuilder extends BasicLogicalPlanVisitor<StringBuilder, LogicalNode> {
+    private static final JoinOrderStringBuilder instance;
+    static {
+      instance = new JoinOrderStringBuilder();
+    }
+
+    public static JoinOrderStringBuilder getInstance() {
+      return instance;
+    }
+
+    public static String buildJoinOrderString(LogicalPlan plan, LogicalPlan.QueryBlock block) throws PlanningException {
+      StringBuilder originalOrder = new StringBuilder();
+      instance.visit(originalOrder, plan, block);
+      return originalOrder.toString();
+    }
+
+    @Override
+    public LogicalNode visitJoin(StringBuilder sb, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode joinNode,
+                                 Stack<LogicalNode> stack)
+        throws PlanningException {
+      stack.push(joinNode);
+      sb.append("(");
+      visit(sb, plan, block, joinNode.getLeftChild(), stack);
+      sb.append(" ").append(getJoinNotation(joinNode.getJoinType())).append(" ");
+      visit(sb, plan, block, joinNode.getRightChild(), stack);
+      sb.append(")");
+      stack.pop();
+      return joinNode;
+    }
+
+    private static String getJoinNotation(JoinType joinType) {
+      switch (joinType) {
+      case CROSS: return "⋈";
+      case INNER: return "⋈θ";
+      case LEFT_OUTER: return "⟕";
+      case RIGHT_OUTER: return "⟖";
+      case FULL_OUTER: return "⟗";
+      case LEFT_SEMI: return "⋉";
+      case RIGHT_SEMI: return "⋊";
+      case LEFT_ANTI: return "▷";
+      }
+      return ",";
+    }
+
+    @Override
+    public LogicalNode visitTableSubQuery(StringBuilder sb, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                          TableSubQueryNode node, Stack<LogicalNode> stack) {
+      sb.append(node.getTableName());
+      return node;
+    }
+
+    public LogicalNode visitScan(StringBuilder sb, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode node,
+                                 Stack<LogicalNode> stack) {
+      sb.append(node.getTableName());
+      return node;
+    }
+  }
+
+  private static class CostContext {
+    double accumulatedCost = 0;
+  }
+
+  public static class JoinCostComputer extends BasicLogicalPlanVisitor<CostContext, LogicalNode> {
+    private static final JoinCostComputer instance;
+
+    static {
+      instance = new JoinCostComputer();
+    }
+
+    public static double computeCost(LogicalPlan plan, LogicalPlan.QueryBlock block) throws PlanningException {
+      CostContext costContext = new CostContext();
+      instance.visit(costContext, plan, block);
+      return costContext.accumulatedCost;
+    }
+
+    @Override
+    public LogicalNode visitJoin(CostContext joinGraphContext, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 JoinNode joinNode, Stack<LogicalNode> stack)
+        throws PlanningException {
+      super.visitJoin(joinGraphContext, plan, block, joinNode, stack);
+
+      double filterFactor = 1;
+      if (joinNode.hasJoinQual()) {
+        EvalNode [] quals = AlgebraicUtil.toConjunctiveNormalFormArray(joinNode.getJoinQual());
+        filterFactor = Math.pow(GreedyHeuristicJoinOrderAlgorithm.DEFAULT_SELECTION_FACTOR, quals.length);
+      }
+
+      if (joinNode.getLeftChild() instanceof RelationNode) {
+        joinGraphContext.accumulatedCost = getCost(joinNode.getLeftChild()) * getCost(joinNode.getRightChild())
+            * filterFactor;
+      } else {
+        joinGraphContext.accumulatedCost = joinGraphContext.accumulatedCost +
+            (joinGraphContext.accumulatedCost * getCost(joinNode.getRightChild()) * filterFactor);
+      }
+
+      return joinNode;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
new file mode 100644
index 0000000..98fbf42
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
@@ -0,0 +1,782 @@
+/*
+ * 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.planner;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.annotation.NotThreadSafe;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.exception.NoSuchColumnException;
+import org.apache.tajo.engine.exception.VerifyException;
+import org.apache.tajo.engine.planner.graph.DirectedGraphCursor;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.RelationNode;
+import org.apache.tajo.util.TUtil;
+
+import java.lang.reflect.Constructor;
+import java.util.*;
+
+/**
+ * This represents and keeps every information about a query plan for a query.
+ */
+@NotThreadSafe
+public class LogicalPlan {
+  /** the prefix character for virtual tables */
+  public static final char VIRTUAL_TABLE_PREFIX='#';
+  public static final char NONAMED_COLUMN_PREFIX='?';
+  /** it indicates the root block */
+  public static final String ROOT_BLOCK = VIRTUAL_TABLE_PREFIX + "ROOT";
+  public static final String NONAME_BLOCK_PREFIX = VIRTUAL_TABLE_PREFIX + "QB_";
+  private static final int NO_SEQUENCE_PID = -1;
+  private int nextPid = 0;
+  private Integer noNameBlockId = 0;
+  private Integer noNameColumnId = 0;
+
+  /** a map from between a block name to a block plan */
+  private Map<String, QueryBlock> queryBlocks = new LinkedHashMap<String, QueryBlock>();
+  private Map<Integer, LogicalNode> nodeMap = new HashMap<Integer, LogicalNode>();
+  private Map<Integer, QueryBlock> queryBlockByPID = new HashMap<Integer, QueryBlock>();
+  private Map<String, String> exprToBlockNameMap = TUtil.newHashMap();
+  private SimpleDirectedGraph<String, BlockEdge> queryBlockGraph = new SimpleDirectedGraph<String, BlockEdge>();
+
+  /** planning and optimization log */
+  private List<String> planingHistory = Lists.newArrayList();
+  LogicalPlanner planner;
+
+  private boolean isExplain;
+  private final String currentDatabase;
+
+  public LogicalPlan(String currentDatabase, LogicalPlanner planner) {
+    this.currentDatabase = currentDatabase;
+    this.planner = planner;
+  }
+
+  /**
+   * Create a LogicalNode instance for a type. Each a LogicalNode instance is given an unique plan node id (PID).
+   *
+   * @param theClass The class to be created
+   * @return a LogicalNode instance identified by an unique plan node id (PID).
+   */
+  public <T extends LogicalNode> T createNode(Class<T> theClass) {
+    try {
+      Constructor<T> ctor = theClass.getConstructor(int.class);
+      return ctor.newInstance(newPID());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Create a LogicalNode instance for a type. Each a LogicalNode instance is not given an unique plan node id (PID).
+   * This method must be only used after all query planning and optimization phases.
+   *
+   * @param theClass The class to be created
+   * @return a LogicalNode instance
+   */
+  public static <T extends LogicalNode> T createNodeWithoutPID(Class<T> theClass) {
+    try {
+      Constructor<T> ctor = theClass.getConstructor(int.class);
+      return ctor.newInstance(NO_SEQUENCE_PID);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void setExplain() {
+    isExplain = true;
+  }
+
+  public boolean isExplain() {
+    return isExplain;
+  }
+
+  /**
+   * Create a new {@link QueryBlock} and Get
+   *
+   * @param blockName the query block name
+   * @return a created query block
+   */
+  public QueryBlock newAndGetBlock(String blockName) {
+    QueryBlock block = new QueryBlock(blockName);
+    queryBlocks.put(blockName, block);
+    return block;
+  }
+
+  public int newPID() {
+    return nextPid++;
+  }
+
+  public QueryBlock newQueryBlock() {
+    return newAndGetBlock(NONAME_BLOCK_PREFIX + (noNameBlockId++));
+  }
+
+  /**
+   * It generates an unique column name from EvalNode. It is usually used for an expression or predicate without
+   * a specified name (i.e., alias).
+   */
+  public String generateUniqueColumnName(EvalNode evalNode) {
+    String prefix = evalNode.getName();
+    return attachSeqIdToGeneratedColumnName(prefix).toLowerCase();
+  }
+
+  /**
+   * It generates an unique column name from Expr. It is usually used for an expression or predicate without
+   * a specified name (i.e., alias).
+   */
+  public String generateUniqueColumnName(Expr expr) {
+    String generatedName;
+    if (expr.getType() == OpType.Column) {
+      generatedName = ((ColumnReferenceExpr) expr).getCanonicalName();
+    } else { // if a generated column name
+      generatedName = attachSeqIdToGeneratedColumnName(getGeneratedPrefixFromExpr(expr));
+    }
+    return generatedName;
+  }
+
+  /**
+   * It attaches a generated column name with a sequence id. It always keeps generated names unique.
+   */
+  private String attachSeqIdToGeneratedColumnName(String prefix) {
+    int sequence = noNameColumnId++;
+    return NONAMED_COLUMN_PREFIX + prefix.toLowerCase() + (sequence > 0 ? "_" + sequence : "");
+  }
+
+  /**
+   * It generates a column reference prefix name. It is usually used for an expression or predicate without
+   * a specified name (i.e., alias). For example, a predicate in WHERE does not have any alias name.
+   * It just returns a prefix name. In other words, it always returns the same prefix for the same type of expressions.
+   * So, you must add some suffix to the returned name in order to distinguish reference names.
+   */
+  private static String getGeneratedPrefixFromExpr(Expr expr) {
+    String prefix;
+
+    switch (expr.getType()) {
+    case Column:
+      prefix = ((ColumnReferenceExpr) expr).getCanonicalName();
+      break;
+    case CountRowsFunction:
+      prefix = "count";
+      break;
+    case GeneralSetFunction:
+      GeneralSetFunctionExpr setFunction = (GeneralSetFunctionExpr) expr;
+      prefix = setFunction.getSignature();
+      break;
+    case Function:
+      FunctionExpr function = (FunctionExpr) expr;
+      prefix = function.getSignature();
+      break;
+    default:
+      prefix = expr.getType().name();
+    }
+    return prefix;
+  }
+
+  public QueryBlock getRootBlock() {
+    return queryBlocks.get(ROOT_BLOCK);
+  }
+
+  public QueryBlock getBlock(String blockName) {
+    return queryBlocks.get(blockName);
+  }
+
+  public QueryBlock getBlock(LogicalNode node) {
+    return queryBlockByPID.get(node.getPID());
+  }
+
+  public void removeBlock(QueryBlock block) {
+    queryBlocks.remove(block.getName());
+    List<Integer> tobeRemoved = new ArrayList<Integer>();
+    for (Map.Entry<Integer, QueryBlock> entry : queryBlockByPID.entrySet()) {
+      tobeRemoved.add(entry.getKey());
+    }
+    for (Integer rn : tobeRemoved) {
+      queryBlockByPID.remove(rn);
+    }
+  }
+
+  public void connectBlocks(QueryBlock srcBlock, QueryBlock targetBlock, BlockType type) {
+    queryBlockGraph.addEdge(srcBlock.getName(), targetBlock.getName(), new BlockEdge(srcBlock, targetBlock, type));
+  }
+
+  public QueryBlock getParentBlock(QueryBlock block) {
+    return queryBlocks.get(queryBlockGraph.getParent(block.getName(), 0));
+  }
+
+  public List<QueryBlock> getChildBlocks(QueryBlock block) {
+    List<QueryBlock> childBlocks = TUtil.newList();
+    for (String blockName : queryBlockGraph.getChilds(block.getName())) {
+      childBlocks.add(queryBlocks.get(blockName));
+    }
+    return childBlocks;
+  }
+
+  public void mapExprToBlock(Expr expr, String blockName) {
+    exprToBlockNameMap.put(ObjectUtils.identityToString(expr), blockName);
+  }
+
+  public QueryBlock getBlockByExpr(Expr expr) {
+    return getBlock(exprToBlockNameMap.get(ObjectUtils.identityToString(expr)));
+  }
+
+  public String getBlockNameByExpr(Expr expr) {
+    return exprToBlockNameMap.get(ObjectUtils.identityToString(expr));
+  }
+
+  public Collection<QueryBlock> getQueryBlocks() {
+    return queryBlocks.values();
+  }
+
+  public SimpleDirectedGraph<String, BlockEdge> getQueryBlockGraph() {
+    return queryBlockGraph;
+  }
+
+  public String getNormalizedColumnName(QueryBlock block, ColumnReferenceExpr columnRef)
+      throws PlanningException {
+    Column found = resolveColumn(block, columnRef);
+    if (found == null) {
+      throw new NoSuchColumnException(columnRef.getCanonicalName());
+    }
+    return found.getQualifiedName();
+  }
+
+  public String resolveDatabase(QueryBlock block, String tableName) throws PlanningException {
+    List<String> found = new ArrayList<String>();
+    for (RelationNode relation : block.getRelations()) {
+      // check alias name or table name
+      if (CatalogUtil.extractSimpleName(relation.getCanonicalName()).equals(tableName) ||
+          CatalogUtil.extractSimpleName(relation.getTableName()).equals(tableName)) {
+        // obtain the database name
+        found.add(CatalogUtil.extractQualifier(relation.getTableName()));
+      }
+    }
+
+    if (found.size() == 0) {
+      return null;
+    } else if (found.size() > 1) {
+      throw new PlanningException("Ambiguous table name \"" + tableName + "\"");
+    }
+
+    return found.get(0);
+  }
+
+  /**
+   * It resolves a column.
+   */
+  public Column resolveColumn(QueryBlock block, ColumnReferenceExpr columnRef) throws PlanningException {
+
+    if (columnRef.hasQualifier()) { // if a column reference is qualified
+
+      String qualifier;
+      String canonicalName;
+      String qualifiedName;
+
+      if (CatalogUtil.isFQTableName(columnRef.getQualifier())) {
+        qualifier = columnRef.getQualifier();
+        canonicalName = columnRef.getCanonicalName();
+      } else {
+        String resolvedDatabaseName = resolveDatabase(block, columnRef.getQualifier());
+        if (resolvedDatabaseName == null) {
+          throw new NoSuchColumnException(columnRef.getQualifier());
+        }
+        qualifier = CatalogUtil.buildFQName(resolvedDatabaseName, columnRef.getQualifier());
+        canonicalName = CatalogUtil.buildFQName(qualifier, columnRef.getName());
+      }
+      qualifiedName = CatalogUtil.buildFQName(qualifier, columnRef.getName());
+
+      RelationNode relationOp = block.getRelation(qualifier);
+
+      // if a column name is outside of this query block
+      if (relationOp == null) {
+        // TODO - nested query can only refer outer query block? or not?
+        for (QueryBlock eachBlock : queryBlocks.values()) {
+          if (eachBlock.existsRelation(qualifier)) {
+            relationOp = eachBlock.getRelation(qualifier);
+          }
+        }
+      }
+
+      // If we cannot find any relation against a qualified column name
+      if (relationOp == null) {
+        throw new NoSuchColumnException(canonicalName);
+      }
+
+      if (block.isAlreadyRenamedTableName(CatalogUtil.extractQualifier(canonicalName))) {
+        String changedName = CatalogUtil.buildFQName(
+            relationOp.getCanonicalName(),
+            CatalogUtil.extractSimpleName(canonicalName));
+        canonicalName = changedName;
+      }
+
+      Schema schema = relationOp.getTableSchema();
+      Column column = schema.getColumn(canonicalName);
+      if (column == null) {
+        throw new NoSuchColumnException(canonicalName);
+      }
+
+      // If code reach here, a column is found.
+      // But, it may be aliased from bottom logical node.
+      // If the column is aliased, the found name may not be used in upper node.
+
+      // Here, we try to check if column reference is already aliased.
+      // If so, it replaces the name with aliased name.
+      LogicalNode currentNode = block.getCurrentNode();
+
+      // The condition (currentNode.getInSchema().contains(column)) means
+      // the column can be used at the current node. So, we don't need to find aliase name.
+      if (currentNode != null && !currentNode.getInSchema().contains(column)
+          && currentNode.getType() != NodeType.TABLE_SUBQUERY) {
+        List<Column> candidates = TUtil.newList();
+        if (block.namedExprsMgr.isAliased(qualifiedName)) {
+          String alias = block.namedExprsMgr.getAlias(canonicalName);
+          Column found = resolveColumn(block, new ColumnReferenceExpr(alias));
+          if (found != null) {
+            candidates.add(found);
+          }
+        }
+        if (!candidates.isEmpty()) {
+          return ensureUniqueColumn(candidates);
+        }
+      }
+
+      return column;
+    } else { // if a column reference is not qualified
+
+      // Trying to find the column within the current block
+
+      if (block.currentNode != null && block.currentNode.getInSchema() != null) {
+        Column found = block.currentNode.getInSchema().getColumn(columnRef.getCanonicalName());
+        if (found != null) {
+          return found;
+        }
+      }
+
+      if (block.getLatestNode() != null) {
+        Column found = block.getLatestNode().getOutSchema().getColumn(columnRef.getName());
+        if (found != null) {
+          return found;
+        }
+      }
+
+      List<Column> candidates = TUtil.newList();
+      // Trying to find columns from aliased references.
+      if (block.namedExprsMgr.isAliased(columnRef.getCanonicalName())) {
+        String originalName = block.namedExprsMgr.getAlias(columnRef.getCanonicalName());
+        Column found = resolveColumn(block, new ColumnReferenceExpr(originalName));
+        if (found != null) {
+          candidates.add(found);
+        }
+      }
+      if (!candidates.isEmpty()) {
+        return ensureUniqueColumn(candidates);
+      }
+
+      // Trying to find columns from other relations in the current block
+      for (RelationNode rel : block.getRelations()) {
+        Column found = rel.getTableSchema().getColumn(columnRef.getName());
+        if (found != null) {
+          candidates.add(found);
+        }
+      }
+
+      if (!candidates.isEmpty()) {
+        return ensureUniqueColumn(candidates);
+      }
+
+      // Trying to find columns from other relations in other blocks
+      for (QueryBlock eachBlock : queryBlocks.values()) {
+        for (RelationNode rel : eachBlock.getRelations()) {
+          Column found = rel.getTableSchema().getColumn(columnRef.getName());
+          if (found != null) {
+            candidates.add(found);
+          }
+        }
+      }
+
+      if (!candidates.isEmpty()) {
+        return ensureUniqueColumn(candidates);
+      }
+
+      // Trying to find columns from schema in current block.
+      if (block.getSchema() != null) {
+        Column found = block.getSchema().getColumn(columnRef.getName());
+        if (found != null) {
+          candidates.add(found);
+        }
+      }
+
+      if (!candidates.isEmpty()) {
+        return ensureUniqueColumn(candidates);
+      }
+
+      throw new VerifyException("ERROR: no such a column name "+ columnRef.getCanonicalName());
+    }
+  }
+
+  private static Column ensureUniqueColumn(List<Column> candidates)
+      throws VerifyException {
+    if (candidates.size() == 1) {
+      return candidates.get(0);
+    } else if (candidates.size() > 2) {
+      StringBuilder sb = new StringBuilder();
+      boolean first = true;
+      for (Column column : candidates) {
+        if (first) {
+          first = false;
+        } else {
+          sb.append(", ");
+        }
+        sb.append(column);
+      }
+      throw new VerifyException("Ambiguous Column Name: " + sb.toString());
+    } else {
+      return null;
+    }
+  }
+
+  public String getQueryGraphAsString() {
+    StringBuilder sb = new StringBuilder();
+
+    sb.append("\n-----------------------------\n");
+    sb.append("Query Block Graph\n");
+    sb.append("-----------------------------\n");
+    sb.append(queryBlockGraph.toStringGraph(getRootBlock().getName()));
+    sb.append("-----------------------------\n");
+    sb.append("Optimization Log:\n");
+    DirectedGraphCursor<String, BlockEdge> cursor =
+        new DirectedGraphCursor<String, BlockEdge>(queryBlockGraph, getRootBlock().getName());
+    while(cursor.hasNext()) {
+      QueryBlock block = getBlock(cursor.nextBlock());
+      if (block.getPlanHistory().size() > 0) {
+        sb.append("\n[").append(block.getName()).append("]\n");
+        for (String log : block.getPlanHistory()) {
+          sb.append("> ").append(log).append("\n");
+        }
+      }
+    }
+    sb.append("-----------------------------\n");
+    sb.append("\n");
+
+    sb.append(getLogicalPlanAsString());
+
+    return sb.toString();
+  }
+
+  public String getLogicalPlanAsString() {
+    ExplainLogicalPlanVisitor explain = new ExplainLogicalPlanVisitor();
+
+    StringBuilder explains = new StringBuilder();
+    try {
+      ExplainLogicalPlanVisitor.Context explainContext = explain.getBlockPlanStrings(this, getRootBlock().getRoot());
+      while(!explainContext.explains.empty()) {
+        explains.append(
+            ExplainLogicalPlanVisitor.printDepthString(explainContext.getMaxDepth(), explainContext.explains.pop()));
+      }
+    } catch (PlanningException e) {
+      throw new RuntimeException(e);
+    }
+
+    return explains.toString();
+  }
+
+  public void addHistory(String string) {
+    planingHistory.add(string);
+  }
+
+  public List<String> getHistory() {
+    return planingHistory;
+  }
+
+  @Override
+  public String toString() {
+    return getQueryGraphAsString();
+  }
+
+  ///////////////////////////////////////////////////////////////////////////
+  //                             Query Block
+  ///////////////////////////////////////////////////////////////////////////
+
+  public static enum BlockType {
+    TableSubQuery,
+    ScalarSubQuery
+  }
+
+  public static class BlockEdge {
+    private String childName;
+    private String parentName;
+    private BlockType blockType;
+
+
+    public BlockEdge(String childName, String parentName, BlockType blockType) {
+      this.childName = childName;
+      this.parentName = parentName;
+      this.blockType = blockType;
+    }
+
+    public BlockEdge(QueryBlock child, QueryBlock parent, BlockType blockType) {
+      this(child.getName(), parent.getName(), blockType);
+    }
+
+    public String getParentName() {
+      return parentName;
+    }
+
+    public String getChildName() {
+      return childName;
+    }
+
+    public BlockType getBlockType() {
+      return blockType;
+    }
+  }
+
+  public class QueryBlock {
+    private final String blockName;
+    private LogicalNode rootNode;
+    private NodeType rootType;
+
+    // transient states
+    private final Map<String, RelationNode> canonicalNameToRelationMap = TUtil.newHashMap();
+    private final Map<String, List<String>> aliasMap = TUtil.newHashMap();
+    private final Map<OpType, List<Expr>> operatorToExprMap = TUtil.newHashMap();
+    /**
+     * It's a map between nodetype and node. node types can be duplicated. So, latest node type is only kept.
+     */
+    private final Map<NodeType, LogicalNode> nodeTypeToNodeMap = TUtil.newHashMap();
+    private final Map<String, LogicalNode> exprToNodeMap = TUtil.newHashMap();
+    final NamedExprsManager namedExprsMgr;
+
+    private LogicalNode currentNode;
+    private LogicalNode latestNode;
+    private final Set<JoinType> includedJoinTypes = TUtil.newHashSet();
+    /**
+     * Set true value if this query block has either implicit or explicit aggregation.
+     */
+    private boolean aggregationRequired = false;
+    private Schema schema;
+
+    /** It contains a planning log for this block */
+    private final List<String> planingHistory = Lists.newArrayList();
+    /** It is for debugging or unit tests */
+    private Target [] rawTargets;
+
+    public QueryBlock(String blockName) {
+      this.blockName = blockName;
+      this.namedExprsMgr = new NamedExprsManager(LogicalPlan.this);
+    }
+
+    public String getName() {
+      return blockName;
+    }
+
+    public void refresh() {
+      setRoot(rootNode);
+    }
+
+    public void setRoot(LogicalNode blockRoot) {
+      this.rootNode = blockRoot;
+      if (blockRoot instanceof LogicalRootNode) {
+        LogicalRootNode rootNode = (LogicalRootNode) blockRoot;
+        rootType = rootNode.getChild().getType();
+      }
+    }
+
+    public <NODE extends LogicalNode> NODE getRoot() {
+      return (NODE) rootNode;
+    }
+
+    public NodeType getRootType() {
+      return rootType;
+    }
+
+    public Target [] getRawTargets() {
+      return rawTargets;
+    }
+
+    public void setRawTargets(Target[] rawTargets) {
+      this.rawTargets = rawTargets;
+    }
+
+    public boolean existsRelation(String name) {
+      return canonicalNameToRelationMap.containsKey(name);
+    }
+
+    public boolean isAlreadyRenamedTableName(String name) {
+      return aliasMap.containsKey(name);
+    }
+
+    public RelationNode getRelation(String name) {
+      if (canonicalNameToRelationMap.containsKey(name)) {
+        return canonicalNameToRelationMap.get(name);
+      }
+
+      if (aliasMap.containsKey(name)) {
+        return canonicalNameToRelationMap.get(aliasMap.get(name).get(0));
+      }
+
+      return null;
+    }
+
+    public void addRelation(RelationNode relation) {
+      if (relation.hasAlias()) {
+        TUtil.putToNestedList(aliasMap, relation.getTableName(), relation.getCanonicalName());
+      }
+      canonicalNameToRelationMap.put(relation.getCanonicalName(), relation);
+    }
+
+    public Collection<RelationNode> getRelations() {
+      return this.canonicalNameToRelationMap.values();
+    }
+
+    public boolean hasTableExpression() {
+      return this.canonicalNameToRelationMap.size() > 0;
+    }
+
+    public void setSchema(Schema schema) {
+      this.schema = schema;
+    }
+
+    public Schema getSchema() {
+      return schema;
+    }
+
+    public NamedExprsManager getNamedExprsManager() {
+      return namedExprsMgr;
+    }
+
+    public void updateCurrentNode(Expr expr) throws PlanningException {
+
+      if (expr.getType() != OpType.RelationList) { // skip relation list because it is a virtual expr.
+        this.currentNode = exprToNodeMap.get(ObjectUtils.identityToString(expr));
+        if (currentNode == null) {
+          throw new PlanningException("Unregistered Algebra Expression: " + expr.getType());
+        }
+      }
+    }
+
+    public <T extends LogicalNode> T getCurrentNode() {
+      return (T) this.currentNode;
+    }
+
+    public void updateLatestNode(LogicalNode node) {
+      this.latestNode = node;
+    }
+
+    public <T extends LogicalNode> T getLatestNode() {
+      return (T) this.latestNode;
+    }
+
+    public void setAlgebraicExpr(Expr expr) {
+      TUtil.putToNestedList(operatorToExprMap, expr.getType(), expr);
+    }
+
+    public boolean hasAlgebraicExpr(OpType opType) {
+      return operatorToExprMap.containsKey(opType);
+    }
+
+    public <T extends Expr> List<T> getAlgebraicExpr(OpType opType) {
+      return (List<T>) operatorToExprMap.get(opType);
+    }
+
+    public <T extends Expr> T getSingletonExpr(OpType opType) {
+      if (hasAlgebraicExpr(opType)) {
+        return (T) operatorToExprMap.get(opType).get(0);
+      } else {
+        return null;
+      }
+    }
+
+    public boolean hasNode(NodeType nodeType) {
+      return nodeTypeToNodeMap.containsKey(nodeType);
+    }
+
+    public void registerNode(LogicalNode node) {
+      // id -> node
+      nodeMap.put(node.getPID(), node);
+
+      // So, this is only for filter, groupby, sort, limit, projection, which exists once at a query block.
+      nodeTypeToNodeMap.put(node.getType(), node);
+
+      queryBlockByPID.put(node.getPID(), this);
+    }
+
+    public <T extends LogicalNode> T getNode(NodeType nodeType) {
+      return (T) nodeTypeToNodeMap.get(nodeType);
+    }
+
+    // expr -> node
+    public void registerExprWithNode(Expr expr, LogicalNode node) {
+      exprToNodeMap.put(ObjectUtils.identityToString(expr), node);
+    }
+
+    public <T extends LogicalNode> T getNodeFromExpr(Expr expr) {
+      return (T) exprToNodeMap.get(ObjectUtils.identityToString(expr));
+    }
+
+    /**
+     * This flag can be changed as a plan is generated.
+     *
+     * True value means that this query should have aggregation phase. If aggregation plan is added to this block,
+     * it becomes false because it doesn't need aggregation phase anymore. It is usually used to add aggregation
+     * phase from SELECT statement without group-by clause.
+     *
+     * @return True if aggregation is needed but this query hasn't had aggregation phase.
+     */
+    public boolean isAggregationRequired() {
+      return this.aggregationRequired;
+    }
+
+    /**
+     * Unset aggregation required flag. It has to be called after an aggregation phase is added to this block.
+     */
+    public void unsetAggregationRequire() {
+      this.aggregationRequired = false;
+    }
+
+    public void setAggregationRequire() {
+      aggregationRequired = true;
+    }
+
+    public boolean containsJoinType(JoinType joinType) {
+      return includedJoinTypes.contains(joinType);
+    }
+
+    public void addJoinType(JoinType joinType) {
+      includedJoinTypes.add(joinType);
+    }
+
+    public List<String> getPlanHistory() {
+      return planingHistory;
+    }
+
+    public void addPlanHistory(String history) {
+      this.planingHistory.add(history);
+    }
+
+    public String toString() {
+      return blockName;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
new file mode 100644
index 0000000..56863f7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
@@ -0,0 +1,442 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.eval.EvalType;
+import org.apache.tajo.engine.eval.FieldEval;
+import org.apache.tajo.engine.exception.NoSuchColumnException;
+import org.apache.tajo.engine.planner.LogicalPlan.QueryBlock;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+/**
+ * It finds all relations for each block and builds base schema information.
+ */
+class LogicalPlanPreprocessor extends BaseAlgebraVisitor<LogicalPlanPreprocessor.PreprocessContext, LogicalNode> {
+  private ExprAnnotator annotator;
+
+  static class PreprocessContext {
+    Session session;
+    LogicalPlan plan;
+    LogicalPlan.QueryBlock currentBlock;
+
+    public PreprocessContext(Session session, LogicalPlan plan, LogicalPlan.QueryBlock currentBlock) {
+      this.session = session;
+      this.plan = plan;
+      this.currentBlock = currentBlock;
+    }
+
+    public PreprocessContext(PreprocessContext context, LogicalPlan.QueryBlock currentBlock) {
+      this.session = context.session;
+      this.plan = context.plan;
+      this.currentBlock = currentBlock;
+    }
+  }
+
+  /** Catalog service */
+  private CatalogService catalog;
+
+  LogicalPlanPreprocessor(CatalogService catalog, ExprAnnotator annotator) {
+    this.catalog = catalog;
+    this.annotator = annotator;
+  }
+
+  @Override
+  public void preHook(PreprocessContext ctx, Stack<Expr> stack, Expr expr) throws PlanningException {
+    ctx.currentBlock.setAlgebraicExpr(expr);
+    ctx.plan.mapExprToBlock(expr, ctx.currentBlock.getName());
+  }
+
+  @Override
+  public LogicalNode postHook(PreprocessContext ctx, Stack<Expr> stack, Expr expr, LogicalNode result) throws PlanningException {
+    // If non-from statement, result can be null. It avoids that case.
+    if (result != null) {
+      // setNode method registers each node to corresponding block and plan.
+      ctx.currentBlock.registerNode(result);
+      // It makes a map between an expr and a logical node.
+      ctx.currentBlock.registerExprWithNode(expr, result);
+    }
+    return result;
+  }
+
+  /**
+   * Get all columns of the relations correspondent to the asterisk expression.
+   * @param ctx
+   * @param asteriskExpr
+   * @return array of columns
+   * @throws PlanningException
+   */
+  public static Column[] getColumns(PreprocessContext ctx, QualifiedAsteriskExpr asteriskExpr)
+      throws PlanningException {
+    RelationNode relationOp = null;
+    QueryBlock block = ctx.currentBlock;
+    Collection<QueryBlock> queryBlocks = ctx.plan.getQueryBlocks();
+    if (asteriskExpr.hasQualifier()) {
+      String qualifier;
+
+      if (CatalogUtil.isFQTableName(asteriskExpr.getQualifier())) {
+        qualifier = asteriskExpr.getQualifier();
+      } else {
+        qualifier = CatalogUtil.buildFQName(ctx.session.getCurrentDatabase(), asteriskExpr.getQualifier());
+      }
+
+      relationOp = block.getRelation(qualifier);
+
+      // if a column name is outside of this query block
+      if (relationOp == null) {
+        // TODO - nested query can only refer outer query block? or not?
+        for (QueryBlock eachBlock : queryBlocks) {
+          if (eachBlock.existsRelation(qualifier)) {
+            relationOp = eachBlock.getRelation(qualifier);
+          }
+        }
+      }
+
+      // If we cannot find any relation against a qualified column name
+      if (relationOp == null) {
+        throw new NoSuchColumnException(CatalogUtil.buildFQName(qualifier, "*"));
+      }
+
+      Schema schema = relationOp.getTableSchema();
+      Column[] resolvedColumns = new Column[schema.size()];
+      return schema.getColumns().toArray(resolvedColumns);
+    } else { // if a column reference is not qualified
+      // columns of every relation should be resolved.
+      Iterator<RelationNode> iterator = block.getRelations().iterator();
+      Schema schema;
+      List<Column> resolvedColumns = TUtil.newList();
+
+      while (iterator.hasNext()) {
+        relationOp = iterator.next();
+        schema = relationOp.getTableSchema();
+        resolvedColumns.addAll(schema.getColumns());
+      }
+
+      if (resolvedColumns.size() == 0) {
+        throw new NoSuchColumnException(asteriskExpr.toString());
+      }
+
+      return resolvedColumns.toArray(new Column[resolvedColumns.size()]);
+    }
+  }
+
+  /**
+   * Resolve an asterisk expression to the real column reference expressions.
+   * @param ctx context
+   * @param asteriskExpr asterisk expression
+   * @return a list of NamedExpr each of which has ColumnReferenceExprs as its child
+   * @throws PlanningException
+   */
+  private static List<NamedExpr> resolveAsterisk(PreprocessContext ctx, QualifiedAsteriskExpr asteriskExpr)
+      throws PlanningException {
+    Column[] columns = getColumns(ctx, asteriskExpr);
+    List<NamedExpr> newTargetExprs = new ArrayList<NamedExpr>(columns.length);
+    int i;
+    for (i = 0; i < columns.length; i++) {
+      newTargetExprs.add(new NamedExpr(new ColumnReferenceExpr(columns[i].getQualifier(), columns[i].getSimpleName())));
+    }
+    return newTargetExprs;
+  }
+
+  private static boolean hasAsterisk(Projection projection) {
+    for (NamedExpr eachTarget : projection.getNamedExprs()) {
+      if (eachTarget.getExpr().getType() == OpType.Asterisk) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public LogicalNode visitProjection(PreprocessContext ctx, Stack<Expr> stack, Projection expr) throws PlanningException {
+    // If Non-from statement, it immediately returns.
+    if (!expr.hasChild()) {
+      return ctx.plan.createNode(EvalExprNode.class);
+    }
+
+    stack.push(expr); // <--- push
+    LogicalNode child = visit(ctx, stack, expr.getChild());
+
+    // Resolve the asterisk expression
+    if (hasAsterisk(expr)) {
+      List<NamedExpr> rewrittenTargets = TUtil.newList();
+      for (NamedExpr originTarget : expr.getNamedExprs()) {
+        if (originTarget.getExpr().getType() == OpType.Asterisk) {
+          // rewrite targets
+          rewrittenTargets.addAll(resolveAsterisk(ctx, (QualifiedAsteriskExpr) originTarget.getExpr()));
+        } else {
+          rewrittenTargets.add(originTarget);
+        }
+      }
+      expr.setNamedExprs(rewrittenTargets.toArray(new NamedExpr[rewrittenTargets.size()]));
+    }
+
+    NamedExpr[] projectTargetExprs = expr.getNamedExprs();
+
+    Target [] targets;
+    targets = new Target[projectTargetExprs.length];
+
+    for (int i = 0; i < expr.getNamedExprs().length; i++) {
+      NamedExpr namedExpr = expr.getNamedExprs()[i];
+      EvalNode evalNode = annotator.createEvalNode(ctx.plan, ctx.currentBlock, namedExpr.getExpr());
+
+      if (namedExpr.hasAlias()) {
+        targets[i] = new Target(evalNode, namedExpr.getAlias());
+      } else if (evalNode.getType() == EvalType.FIELD) {
+        targets[i] = new Target((FieldEval) evalNode);
+      } else {
+        String generatedName = ctx.plan.generateUniqueColumnName(namedExpr.getExpr());
+        targets[i] = new Target(evalNode, generatedName);
+        namedExpr.setAlias(generatedName);
+      }
+    }
+    stack.pop(); // <--- Pop
+
+    ProjectionNode projectionNode = ctx.plan.createNode(ProjectionNode.class);
+    projectionNode.setInSchema(child.getOutSchema());
+    projectionNode.setOutSchema(PlannerUtil.targetToSchema(targets));
+    return projectionNode;
+  }
+
+  @Override
+  public LogicalNode visitLimit(PreprocessContext ctx, Stack<Expr> stack, Limit expr) throws PlanningException {
+    stack.push(expr);
+    LogicalNode child = visit(ctx, stack, expr.getChild());
+    stack.pop();
+
+    LimitNode limitNode = ctx.plan.createNode(LimitNode.class);
+    limitNode.setInSchema(child.getOutSchema());
+    limitNode.setOutSchema(child.getOutSchema());
+    return limitNode;
+  }
+
+  @Override
+  public LogicalNode visitSort(PreprocessContext ctx, Stack<Expr> stack, Sort expr) throws PlanningException {
+    stack.push(expr);
+    LogicalNode child = visit(ctx, stack, expr.getChild());
+    stack.pop();
+
+    SortNode sortNode = ctx.plan.createNode(SortNode.class);
+    sortNode.setInSchema(child.getOutSchema());
+    sortNode.setOutSchema(child.getOutSchema());
+    return sortNode;
+  }
+
+  @Override
+  public LogicalNode visitHaving(PreprocessContext ctx, Stack<Expr> stack, Having expr) throws PlanningException {
+    stack.push(expr);
+    LogicalNode child = visit(ctx, stack, expr.getChild());
+    stack.pop();
+
+    HavingNode havingNode = ctx.plan.createNode(HavingNode.class);
+    havingNode.setInSchema(child.getOutSchema());
+    havingNode.setOutSchema(child.getOutSchema());
+    return havingNode;
+  }
+
+  @Override
+  public LogicalNode visitGroupBy(PreprocessContext ctx, Stack<Expr> stack, Aggregation expr) throws PlanningException {
+    stack.push(expr); // <--- push
+    LogicalNode child = visit(ctx, stack, expr.getChild());
+
+    Projection projection = ctx.currentBlock.getSingletonExpr(OpType.Projection);
+    int finalTargetNum = projection.getNamedExprs().length;
+    Target [] targets = new Target[finalTargetNum];
+
+    for (int i = 0; i < finalTargetNum; i++) {
+      NamedExpr namedExpr = projection.getNamedExprs()[i];
+      EvalNode evalNode = annotator.createEvalNode(ctx.plan, ctx.currentBlock, namedExpr.getExpr());
+
+      if (namedExpr.hasAlias()) {
+        targets[i] = new Target(evalNode, namedExpr.getAlias());
+      } else {
+        targets[i] = new Target(evalNode, "?name_" + i);
+      }
+    }
+    stack.pop();
+
+    GroupbyNode groupByNode = ctx.plan.createNode(GroupbyNode.class);
+    groupByNode.setInSchema(child.getOutSchema());
+    groupByNode.setOutSchema(PlannerUtil.targetToSchema(targets));
+    return groupByNode;
+  }
+
+  @Override
+  public LogicalNode visitUnion(PreprocessContext ctx, Stack<Expr> stack, SetOperation expr) throws PlanningException {
+    LogicalPlan.QueryBlock leftBlock = ctx.plan.newQueryBlock();
+    PreprocessContext leftContext = new PreprocessContext(ctx, leftBlock);
+    LogicalNode leftChild = visit(leftContext, new Stack<Expr>(), expr.getLeft());
+    ctx.currentBlock.registerExprWithNode(expr.getLeft(), leftChild);
+
+    LogicalPlan.QueryBlock rightBlock = ctx.plan.newQueryBlock();
+    PreprocessContext rightContext = new PreprocessContext(ctx, rightBlock);
+    LogicalNode rightChild = visit(rightContext, new Stack<Expr>(), expr.getRight());
+    ctx.currentBlock.registerExprWithNode(expr.getRight(), rightChild);
+
+    UnionNode unionNode = new UnionNode(ctx.plan.newPID());
+    unionNode.setLeftChild(leftChild);
+    unionNode.setRightChild(rightChild);
+    unionNode.setInSchema(leftChild.getOutSchema());
+    unionNode.setOutSchema(leftChild.getOutSchema());
+
+    return unionNode;
+  }
+
+  public LogicalNode visitFilter(PreprocessContext ctx, Stack<Expr> stack, Selection expr) throws PlanningException {
+    stack.push(expr);
+    LogicalNode child = visit(ctx, stack, expr.getChild());
+    stack.pop();
+
+    SelectionNode selectionNode = ctx.plan.createNode(SelectionNode.class);
+    selectionNode.setInSchema(child.getOutSchema());
+    selectionNode.setOutSchema(child.getOutSchema());
+    return selectionNode;
+  }
+
+  @Override
+  public LogicalNode visitJoin(PreprocessContext ctx, Stack<Expr> stack, Join expr) throws PlanningException {
+    stack.push(expr);
+    LogicalNode left = visit(ctx, stack, expr.getLeft());
+    LogicalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+    JoinNode joinNode = ctx.plan.createNode(JoinNode.class);
+    joinNode.setJoinType(expr.getJoinType());
+    Schema merged = SchemaUtil.merge(left.getOutSchema(), right.getOutSchema());
+    joinNode.setInSchema(merged);
+    joinNode.setOutSchema(merged);
+
+    ctx.currentBlock.addJoinType(expr.getJoinType());
+    return joinNode;
+  }
+
+  @Override
+  public LogicalNode visitRelation(PreprocessContext ctx, Stack<Expr> stack, Relation expr)
+      throws PlanningException {
+    Relation relation = expr;
+
+    String actualRelationName;
+    if (CatalogUtil.isFQTableName(expr.getName())) {
+      actualRelationName = relation.getName();
+    } else {
+      actualRelationName = CatalogUtil.buildFQName(ctx.session.getCurrentDatabase(), relation.getName());
+    }
+
+    TableDesc desc = catalog.getTableDesc(actualRelationName);
+    ScanNode scanNode = ctx.plan.createNode(ScanNode.class);
+    if (relation.hasAlias()) {
+      scanNode.init(desc, relation.getAlias());
+    } else {
+      scanNode.init(desc);
+    }
+    ctx.currentBlock.addRelation(scanNode);
+
+    return scanNode;
+  }
+
+  @Override
+  public LogicalNode visitTableSubQuery(PreprocessContext ctx, Stack<Expr> stack, TablePrimarySubQuery expr)
+      throws PlanningException {
+
+    PreprocessContext newContext;
+    // Note: TableSubQuery always has a table name.
+    // SELECT .... FROM (SELECT ...) TB_NAME <-
+    newContext = new PreprocessContext(ctx, ctx.plan.newQueryBlock());
+    LogicalNode child = super.visitTableSubQuery(newContext, stack, expr);
+
+    // a table subquery should be dealt as a relation.
+    TableSubQueryNode node = ctx.plan.createNode(TableSubQueryNode.class);
+    node.init(CatalogUtil.buildFQName(ctx.session.getCurrentDatabase(), expr.getName()), child);
+    ctx.currentBlock.addRelation(node);
+    return node;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Data Definition Language Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public LogicalNode visitCreateDatabase(PreprocessContext ctx, Stack<Expr> stack, CreateDatabase expr)
+      throws PlanningException {
+    CreateDatabaseNode createDatabaseNode = ctx.plan.createNode(CreateDatabaseNode.class);
+    return createDatabaseNode;
+  }
+
+  @Override
+  public LogicalNode visitDropDatabase(PreprocessContext ctx, Stack<Expr> stack, DropDatabase expr)
+      throws PlanningException {
+    DropDatabaseNode dropDatabaseNode = ctx.plan.createNode(DropDatabaseNode.class);
+    return dropDatabaseNode;
+  }
+
+  @Override
+  public LogicalNode visitCreateTable(PreprocessContext ctx, Stack<Expr> stack, CreateTable expr)
+      throws PlanningException {
+
+    CreateTableNode createTableNode = ctx.plan.createNode(CreateTableNode.class);
+
+    if (expr.hasSubQuery()) {
+      stack.push(expr);
+      visit(ctx, stack, expr.getSubQuery());
+      stack.pop();
+    }
+
+    return createTableNode;
+  }
+
+  @Override
+  public LogicalNode visitDropTable(PreprocessContext ctx, Stack<Expr> stack, DropTable expr)
+      throws PlanningException {
+    DropTableNode dropTable = ctx.plan.createNode(DropTableNode.class);
+    return dropTable;
+  }
+
+  @Override
+  public LogicalNode visitAlterTablespace(PreprocessContext ctx, Stack<Expr> stack, AlterTablespace expr)
+      throws PlanningException {
+    AlterTablespaceNode alterTablespace = ctx.plan.createNode(AlterTablespaceNode.class);
+    return alterTablespace;
+  }
+
+  @Override
+  public LogicalNode visitAlterTable(PreprocessContext ctx, Stack<Expr> stack, AlterTable expr)
+      throws PlanningException {
+    AlterTableNode alterTableNode = ctx.plan.createNode(AlterTableNode.class);
+    return alterTableNode;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Insert or Update Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public LogicalNode visitInsert(PreprocessContext ctx, Stack<Expr> stack, Insert expr) throws PlanningException {
+    LogicalNode child = super.visitInsert(ctx, stack, expr);
+
+    InsertNode insertNode = new InsertNode(ctx.plan.newPID());
+    insertNode.setInSchema(child.getOutSchema());
+    insertNode.setOutSchema(child.getOutSchema());
+    return insertNode;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java
new file mode 100644
index 0000000..bb8192f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java
@@ -0,0 +1,249 @@
+/**
+ * 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.planner;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.master.session.Session;
+
+import java.util.Stack;
+
+public class LogicalPlanVerifier extends BasicLogicalPlanVisitor<LogicalPlanVerifier.Context, LogicalNode> {
+  private TajoConf conf;
+  private CatalogService catalog;
+
+  public LogicalPlanVerifier(TajoConf conf, CatalogService catalog) {
+    this.conf = conf;
+    this.catalog = catalog;
+  }
+
+  public static class Context {
+    Session session;
+    VerificationState state;
+
+    public Context(Session session, VerificationState state) {
+      this.session = session;
+      this.state = state;
+    }
+  }
+
+  public VerificationState verify(Session session, VerificationState state, LogicalPlan plan) throws PlanningException {
+    Context context = new Context(session, state);
+    visit(context, plan, plan.getRootBlock());
+    return context.state;
+  }
+
+  /**
+   * It checks if an output schema of a projectable node and target's output data types are equivalent to each other.
+   */
+  private static void verifyProjectableOutputSchema(Projectable node) throws PlanningException {
+
+    Schema outputSchema = node.getOutSchema();
+    Schema targetSchema = PlannerUtil.targetToSchema(node.getTargets());
+
+    if (outputSchema.size() != node.getTargets().length) {
+      throw new PlanningException(String.format("Output schema and Target's schema are mismatched at Node (%d)",
+          + node.getPID()));
+    }
+
+    for (int i = 0; i < outputSchema.size(); i++) {
+      if (!outputSchema.getColumn(i).getDataType().equals(targetSchema.getColumn(i).getDataType())) {
+        Column targetColumn = targetSchema.getColumn(i);
+        Column insertColumn = outputSchema.getColumn(i);
+        throw new PlanningException("ERROR: " +
+            insertColumn.getSimpleName() + " is of type " + insertColumn.getDataType().getType().name() +
+            ", but target column '" + targetColumn.getSimpleName() + "' is of type " +
+            targetColumn.getDataType().getType().name());
+      }
+    }
+  }
+
+  @Override
+  public LogicalNode visitProjection(Context state, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                     ProjectionNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitProjection(state, plan, block, node, stack);
+
+    for (Target target : node.getTargets()) {
+      ExprsVerifier.verify(state.state, node, target.getEvalTree());
+    }
+
+    verifyProjectableOutputSchema(node);
+
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitLimit(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                LimitNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitLimit(context, plan, block, node, stack);
+
+    if (node.getFetchFirstNum() < 0) {
+      context.state.addVerification("LIMIT must not be negative");
+    }
+
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitGroupBy(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                  GroupbyNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitGroupBy(context, plan, block, node, stack);
+
+    verifyProjectableOutputSchema(node);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitFilter(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 SelectionNode node, Stack<LogicalNode> stack) throws PlanningException {
+    visit(context, plan, block, node.getChild(), stack);
+    ExprsVerifier.verify(context.state, node, node.getQual());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitJoin(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode node,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    visit(context, plan, block, node.getLeftChild(), stack);
+    visit(context, plan, block, node.getRightChild(), stack);
+
+    if (node.hasJoinQual()) {
+      ExprsVerifier.verify(context.state, node, node.getJoinQual());
+    }
+
+    verifyProjectableOutputSchema(node);
+
+    return node;
+  }
+
+  private void verifySetStatement(VerificationState state, BinaryNode setNode) {
+    Preconditions.checkArgument(setNode.getType() == NodeType.UNION || setNode.getType() == NodeType.INTERSECT ||
+      setNode.getType() == NodeType.EXCEPT);
+    Schema left = setNode.getLeftChild().getOutSchema();
+    Schema right = setNode.getRightChild().getOutSchema();
+    NodeType type = setNode.getType();
+
+    if (left.size() != right.size()) {
+      state.addVerification("each " + type.name() + " query must have the same number of columns");
+      return;
+    }
+
+    Column[] leftColumns = left.toArray();
+    Column[] rightColumns = right.toArray();
+
+    for (int i = 0; i < leftColumns.length; i++) {
+      if (!leftColumns[i].getDataType().equals(rightColumns[i].getDataType())) {
+        state.addVerification(type + " types " + leftColumns[i].getDataType().getType() + " and "
+            + rightColumns[i].getDataType().getType() + " cannot be matched");
+      }
+    }
+  }
+
+  @Override
+  public LogicalNode visitUnion(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                UnionNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitUnion(context, plan, block, node, stack);
+    verifySetStatement(context.state, node);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitExcept(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 ExceptNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitExcept(context, plan, block, node, stack);
+    verifySetStatement(context.state, node);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitIntersect(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                    IntersectNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitIntersect(context, plan, block, node, stack);
+    verifySetStatement(context.state, node);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitScan(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode node,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    if (node.hasTargets()) {
+      for (Target target : node.getTargets()) {
+        ExprsVerifier.verify(context.state, node, target.getEvalTree());
+      }
+    }
+
+    if (node.hasQual()) {
+      ExprsVerifier.verify(context.state, node, node.getQual());
+    }
+
+    verifyProjectableOutputSchema(node);
+
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitStoreTable(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                     StoreTableNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitStoreTable(context, plan, block, node, stack);
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitInsert(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                 InsertNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitInsert(context, plan, block, node, stack);
+    return node;
+  }
+
+  /**
+   * This ensures that corresponding columns in both tables are equivalent to each other.
+   */
+  private static void ensureDomains(VerificationState state, Schema targetTableScheme, Schema schema)
+      throws PlanningException {
+    for (int i = 0; i < schema.size(); i++) {
+      if (!schema.getColumn(i).getDataType().equals(targetTableScheme.getColumn(i).getDataType())) {
+        Column targetColumn = targetTableScheme.getColumn(i);
+        Column insertColumn = schema.getColumn(i);
+        state.addVerification("ERROR: " +
+            insertColumn.getSimpleName() + " is of type " + insertColumn.getDataType().getType().name() +
+            ", but target column '" + targetColumn.getSimpleName() + "' is of type " +
+            targetColumn.getDataType().getType().name());
+      }
+    }
+  }
+
+  @Override
+  public LogicalNode visitCreateTable(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                      CreateTableNode node, Stack<LogicalNode> stack) throws PlanningException {
+    super.visitCreateTable(context, plan, block, node, stack);
+    // here, we don't need check table existence because this check is performed in PreLogicalPlanVerifier.
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitDropTable(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                    DropTableNode node, Stack<LogicalNode> stack) {
+    // here, we don't need check table existence because this check is performed in PreLogicalPlanVerifier.
+    return node;
+  }
+}


[51/51] [partial] git commit: TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)


Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/6594ac1c
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/6594ac1c
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/6594ac1c

Branch: refs/heads/master
Commit: 6594ac1c00192537b0796d0404e0ddf9d6e37111
Parents: 795077d
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Apr 18 18:19:00 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Apr 18 18:19:00 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |     3 +
 pom.xml                                         |     1 +
 tajo-client/pom.xml                             |     2 +-
 .../java/org/apache/tajo/client/TajoDump.java   |    13 +-
 .../java/org/apache/tajo/util/FileUtil.java     |     5 +
 tajo-core/benchmark/simple/groupby1.sql         |     6 +
 tajo-core/benchmark/simple/groupby2.sql         |     6 +
 tajo-core/benchmark/simple/selection1.sql       |     5 +
 tajo-core/benchmark/simple/selection2.sql       |     5 +
 tajo-core/benchmark/simple/selection3.sql       |     7 +
 tajo-core/benchmark/tpch/customer.schema        |     9 +
 tajo-core/benchmark/tpch/lineitem.schema        |    19 +
 tajo-core/benchmark/tpch/nation.schema          |     6 +
 tajo-core/benchmark/tpch/orders.schema          |    11 +
 tajo-core/benchmark/tpch/part.schema            |    11 +
 tajo-core/benchmark/tpch/partsupp.schema        |     8 +
 tajo-core/benchmark/tpch/q1.sql                 |    21 +
 tajo-core/benchmark/tpch/q10.sql                |    31 +
 tajo-core/benchmark/tpch/q11.sql                |    27 +
 tajo-core/benchmark/tpch/q12.sql                |    24 +
 tajo-core/benchmark/tpch/q13.sql                |    18 +
 tajo-core/benchmark/tpch/q14.sql                |    13 +
 tajo-core/benchmark/tpch/q15.sql                |    32 +
 tajo-core/benchmark/tpch/q16.sql                |    30 +
 tajo-core/benchmark/tpch/q17.sql                |    17 +
 tajo-core/benchmark/tpch/q18.sql                |    33 +
 tajo-core/benchmark/tpch/q19.sql                |    35 +
 tajo-core/benchmark/tpch/q2.sql                 |    38 +
 tajo-core/benchmark/tpch/q20.sql                |    36 +
 tajo-core/benchmark/tpch/q21.sql                |    39 +
 tajo-core/benchmark/tpch/q22.sql                |    36 +
 tajo-core/benchmark/tpch/q3.sql                 |    22 +
 tajo-core/benchmark/tpch/q4.sql                 |    21 +
 tajo-core/benchmark/tpch/q5.sql                 |    24 +
 tajo-core/benchmark/tpch/q6.sql                 |    10 +
 tajo-core/benchmark/tpch/q7.sql                 |    38 +
 tajo-core/benchmark/tpch/q8.sql                 |    37 +
 tajo-core/benchmark/tpch/q9.sql                 |    31 +
 tajo-core/benchmark/tpch/region.schema          |     5 +
 tajo-core/benchmark/tpch/supplier.schema        |     9 +
 tajo-core/pom.xml                               |   649 +-
 .../apache/tajo/engine/parser/HiveQLLexer.g4    |   390 +
 .../apache/tajo/engine/parser/HiveQLParser.g4   |  2067 ++++
 .../org/apache/tajo/engine/parser/SQLLexer.g4   |   479 +
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  1398 +++
 tajo-core/src/main/java/log4j.properties        |    30 +
 .../org/apache/tajo/benchmark/BenchmarkSet.java |   120 +
 .../java/org/apache/tajo/benchmark/Driver.java  |    46 +
 .../org/apache/tajo/benchmark/SimpleQuery.java  |    29 +
 .../java/org/apache/tajo/benchmark/TPCH.java    |   200 +
 .../eval/AggregationFunctionCallEval.java       |    92 +
 .../tajo/engine/eval/AlgebraicException.java    |    41 +
 .../apache/tajo/engine/eval/AlgebraicUtil.java  |   398 +
 .../tajo/engine/eval/BasicEvalNodeVisitor.java  |   341 +
 .../tajo/engine/eval/BetweenPredicateEval.java  |   219 +
 .../org/apache/tajo/engine/eval/BinaryEval.java |   205 +
 .../apache/tajo/engine/eval/CaseWhenEval.java   |   208 +
 .../org/apache/tajo/engine/eval/CastEval.java   |    87 +
 .../org/apache/tajo/engine/eval/ConstEval.java  |    99 +
 .../org/apache/tajo/engine/eval/EvalNode.java   |   118 +
 .../tajo/engine/eval/EvalNodeVisitor.java       |    24 +
 .../tajo/engine/eval/EvalNodeVisitor2.java      |    71 +
 .../tajo/engine/eval/EvalTreeFactory.java       |    32 +
 .../apache/tajo/engine/eval/EvalTreeUtil.java   |   350 +
 .../org/apache/tajo/engine/eval/EvalType.java   |    83 +
 .../org/apache/tajo/engine/eval/FieldEval.java  |   114 +
 .../apache/tajo/engine/eval/FunctionEval.java   |   144 +
 .../tajo/engine/eval/GeneralFunctionEval.java   |    81 +
 .../org/apache/tajo/engine/eval/InEval.java     |    86 +
 .../tajo/engine/eval/InvalidCastException.java  |    36 +
 .../tajo/engine/eval/InvalidEvalException.java  |    36 +
 .../org/apache/tajo/engine/eval/IsNullEval.java |    84 +
 .../tajo/engine/eval/LikePredicateEval.java     |    50 +
 .../org/apache/tajo/engine/eval/NotEval.java    |    92 +
 .../tajo/engine/eval/PartialBinaryExpr.java     |    70 +
 .../engine/eval/PatternMatchPredicateEval.java  |    82 +
 .../tajo/engine/eval/RegexPredicateEval.java    |    53 +
 .../tajo/engine/eval/RowConstantEval.java       |    79 +
 .../org/apache/tajo/engine/eval/SignedEval.java |    99 +
 .../engine/eval/SimilarToPredicateEval.java     |    43 +
 .../exception/AmbiguousFieldException.java      |    30 +
 .../exception/IllegalQueryStatusException.java  |    38 +
 .../engine/exception/InvalidQueryException.java |    35 +
 .../engine/exception/NoSuchColumnException.java |    25 +
 .../exception/RangeOverflowException.java       |    28 +
 .../tajo/engine/exception/VerifyException.java  |    27 +
 .../tajo/engine/function/AggFunction.java       |    58 +
 .../tajo/engine/function/FunctionContext.java   |    22 +
 .../tajo/engine/function/GeneralFunction.java   |    56 +
 .../engine/function/annotation/Description.java |    50 +
 .../function/annotation/ParamOptionTypes.java   |    29 +
 .../engine/function/annotation/ParamTypes.java  |    30 +
 .../tajo/engine/function/builtin/AvgDouble.java |    97 +
 .../tajo/engine/function/builtin/AvgFloat.java  |    46 +
 .../tajo/engine/function/builtin/AvgInt.java    |    47 +
 .../tajo/engine/function/builtin/AvgLong.java   |    96 +
 .../tajo/engine/function/builtin/Coalesce.java  |    44 +
 .../engine/function/builtin/CoalesceDouble.java |    43 +
 .../engine/function/builtin/CoalesceLong.java   |    44 +
 .../engine/function/builtin/CoalesceString.java |    44 +
 .../tajo/engine/function/builtin/CountRows.java |    84 +
 .../engine/function/builtin/CountValue.java     |    55 +
 .../function/builtin/CountValueDistinct.java    |    90 +
 .../tajo/engine/function/builtin/Date.java      |    62 +
 .../tajo/engine/function/builtin/MaxDouble.java |    78 +
 .../tajo/engine/function/builtin/MaxFloat.java  |    76 +
 .../tajo/engine/function/builtin/MaxInt.java    |    77 +
 .../tajo/engine/function/builtin/MaxLong.java   |    77 +
 .../tajo/engine/function/builtin/MaxString.java |    81 +
 .../tajo/engine/function/builtin/MinDouble.java |    77 +
 .../tajo/engine/function/builtin/MinFloat.java  |    78 +
 .../tajo/engine/function/builtin/MinInt.java    |    77 +
 .../tajo/engine/function/builtin/MinLong.java   |    78 +
 .../tajo/engine/function/builtin/MinString.java |    82 +
 .../tajo/engine/function/builtin/RandomInt.java |    57 +
 .../tajo/engine/function/builtin/Sleep.java     |    52 +
 .../tajo/engine/function/builtin/SumDouble.java |    82 +
 .../function/builtin/SumDoubleDistinct.java     |    98 +
 .../tajo/engine/function/builtin/SumFloat.java  |    80 +
 .../function/builtin/SumFloatDistinct.java      |    98 +
 .../tajo/engine/function/builtin/SumInt.java    |    82 +
 .../engine/function/builtin/SumIntDistinct.java |    98 +
 .../tajo/engine/function/builtin/SumLong.java   |    82 +
 .../function/builtin/SumLongDistinct.java       |    98 +
 .../tajo/engine/function/builtin/Today.java     |    46 +
 .../function/datetime/DatePartFromDate.java     |   194 +
 .../function/datetime/DatePartFromTime.java     |   140 +
 .../datetime/DatePartFromTimestamp.java         |   259 +
 .../datetime/DateTimePartFromUnixTimeStamp.java |   141 +
 .../function/datetime/ToCharTimestamp.java      |    79 +
 .../engine/function/datetime/ToTimestamp.java   |    55 +
 .../function/geoip/GeoIPCountryInet4.java       |    55 +
 .../engine/function/geoip/GeoIPCountryText.java |    56 +
 .../function/geoip/GeoIPInCountryInet4.java     |    61 +
 .../function/geoip/GeoIPInCountryText.java      |    60 +
 .../tajo/engine/function/math/AbsDouble.java    |    61 +
 .../tajo/engine/function/math/AbsFloat.java     |    61 +
 .../tajo/engine/function/math/AbsInt.java       |    61 +
 .../tajo/engine/function/math/AbsLong.java      |    60 +
 .../apache/tajo/engine/function/math/Acos.java  |    61 +
 .../apache/tajo/engine/function/math/Asin.java  |    60 +
 .../apache/tajo/engine/function/math/Atan.java  |    60 +
 .../apache/tajo/engine/function/math/Atan2.java |    62 +
 .../apache/tajo/engine/function/math/Cbrt.java  |    61 +
 .../apache/tajo/engine/function/math/Ceil.java  |    63 +
 .../apache/tajo/engine/function/math/Cos.java   |    60 +
 .../tajo/engine/function/math/Degrees.java      |    61 +
 .../apache/tajo/engine/function/math/Div.java   |    77 +
 .../apache/tajo/engine/function/math/Exp.java   |    61 +
 .../apache/tajo/engine/function/math/Floor.java |    61 +
 .../apache/tajo/engine/function/math/Mod.java   |    77 +
 .../apache/tajo/engine/function/math/Pi.java    |    51 +
 .../apache/tajo/engine/function/math/Pow.java   |    78 +
 .../tajo/engine/function/math/Radians.java      |    62 +
 .../apache/tajo/engine/function/math/Round.java |    71 +
 .../apache/tajo/engine/function/math/Sign.java  |    64 +
 .../apache/tajo/engine/function/math/Sin.java   |    60 +
 .../apache/tajo/engine/function/math/Sqrt.java  |    62 +
 .../apache/tajo/engine/function/math/Tan.java   |    60 +
 .../tajo/engine/function/string/Ascii.java      |    67 +
 .../tajo/engine/function/string/BTrim.java      |    77 +
 .../tajo/engine/function/string/BitLength.java  |    58 +
 .../tajo/engine/function/string/CharLength.java |    59 +
 .../apache/tajo/engine/function/string/Chr.java |    65 +
 .../tajo/engine/function/string/Concat.java     |    73 +
 .../tajo/engine/function/string/Concat_ws.java  |    78 +
 .../tajo/engine/function/string/Decode.java     |    91 +
 .../tajo/engine/function/string/Digest.java     |    87 +
 .../tajo/engine/function/string/Encode.java     |    93 +
 .../tajo/engine/function/string/FindInSet.java  |   116 +
 .../function/string/HexStringConverter.java     |    65 +
 .../tajo/engine/function/string/InitCap.java    |    60 +
 .../tajo/engine/function/string/LTrim.java      |    77 +
 .../tajo/engine/function/string/Left.java       |    84 +
 .../tajo/engine/function/string/Length.java     |    61 +
 .../tajo/engine/function/string/Locate.java     |   106 +
 .../tajo/engine/function/string/Lower.java      |    58 +
 .../tajo/engine/function/string/Lpad.java       |    90 +
 .../apache/tajo/engine/function/string/Md5.java |    65 +
 .../engine/function/string/OctetLength.java     |    61 +
 .../tajo/engine/function/string/QuoteIdent.java |    67 +
 .../tajo/engine/function/string/RTrim.java      |    76 +
 .../engine/function/string/RegexpReplace.java   |   115 +
 .../tajo/engine/function/string/Repeat.java     |    72 +
 .../tajo/engine/function/string/Reverse.java    |    58 +
 .../tajo/engine/function/string/Right.java      |    85 +
 .../tajo/engine/function/string/Rpad.java       |    90 +
 .../tajo/engine/function/string/SplitPart.java  |    73 +
 .../tajo/engine/function/string/StrPos.java     |    73 +
 .../tajo/engine/function/string/StrPosb.java    |   120 +
 .../tajo/engine/function/string/Substr.java     |    94 +
 .../tajo/engine/function/string/ToBin.java      |    65 +
 .../tajo/engine/function/string/ToHex.java      |    76 +
 .../tajo/engine/function/string/Upper.java      |    58 +
 .../apache/tajo/engine/json/CoreGsonHelper.java |    90 +
 .../tajo/engine/json/EvalNodeAdapter.java       |    50 +
 .../tajo/engine/json/LogicalNodeAdapter.java    |    50 +
 .../tajo/engine/parser/HiveQLAnalyzer.java      |  1550 +++
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  1516 +++
 .../tajo/engine/parser/SQLErrorListener.java    |    38 +
 .../tajo/engine/parser/SQLErrorStrategy.java    |    66 +
 .../tajo/engine/parser/SQLParseError.java       |   107 +
 .../tajo/engine/parser/SQLSyntaxError.java      |    51 +
 .../tajo/engine/planner/AlgebraVisitor.java     |   110 +
 .../engine/planner/AlterTablespaceNode.java     |   103 +
 .../tajo/engine/planner/BaseAlgebraVisitor.java |   739 ++
 .../engine/planner/BasicLogicalPlanVisitor.java |   319 +
 .../planner/BroadcastJoinPlanVisitor.java       |    86 +
 .../planner/ExplainLogicalPlanVisitor.java      |   236 +
 .../tajo/engine/planner/ExprAnnotator.java      |   631 +
 .../apache/tajo/engine/planner/ExprFinder.java  |    74 +
 .../tajo/engine/planner/ExprNormalizer.java     |   257 +
 .../tajo/engine/planner/ExprsVerifier.java      |   216 +
 .../tajo/engine/planner/GroupElement.java       |    64 +
 .../tajo/engine/planner/LogicalOptimizer.java   |   280 +
 .../apache/tajo/engine/planner/LogicalPlan.java |   782 ++
 .../engine/planner/LogicalPlanPreprocessor.java |   442 +
 .../engine/planner/LogicalPlanVerifier.java     |   249 +
 .../tajo/engine/planner/LogicalPlanVisitor.java |    91 +
 .../tajo/engine/planner/LogicalPlanner.java     |  1579 +++
 .../tajo/engine/planner/NamedExprsManager.java  |   369 +
 .../tajo/engine/planner/PhysicalPlanner.java    |    36 +
 .../engine/planner/PhysicalPlannerImpl.java     |  1057 ++
 .../planner/PhysicalPlanningException.java      |    31 +
 .../apache/tajo/engine/planner/PlanString.java  |   119 +
 .../apache/tajo/engine/planner/PlannerUtil.java |   762 ++
 .../tajo/engine/planner/PlanningException.java  |    29 +
 .../engine/planner/PreLogicalPlanVerifier.java  |   274 +
 .../apache/tajo/engine/planner/Projector.java   |    51 +
 .../engine/planner/RangePartitionAlgorithm.java |   187 +
 .../engine/planner/SimpleAlgebraVisitor.java    |   215 +
 .../org/apache/tajo/engine/planner/Target.java  |   129 +
 .../engine/planner/UniformRangePartition.java   |   451 +
 .../tajo/engine/planner/VerificationState.java  |    44 +
 .../tajo/engine/planner/enforce/Enforcer.java   |   262 +
 .../tajo/engine/planner/global/DataChannel.java |   200 +
 .../engine/planner/global/ExecutionBlock.java   |   127 +
 .../planner/global/ExecutionBlockCursor.java    |    93 +
 .../engine/planner/global/GlobalPlanner.java    |  1187 ++
 .../tajo/engine/planner/global/MasterPlan.java  |   268 +
 .../engine/planner/graph/DirectedGraph.java     |    64 +
 .../planner/graph/DirectedGraphCursor.java      |    65 +
 .../planner/graph/DirectedGraphVisitor.java     |    25 +
 .../apache/tajo/engine/planner/graph/Graph.java |    45 +
 .../planner/graph/SimpleDirectedGraph.java      |   270 +
 .../planner/graph/SimpleUndirectedGraph.java    |   102 +
 .../engine/planner/graph/UndirectedGraph.java   |    30 +
 .../engine/planner/logical/AlterTableNode.java  |   134 +
 .../tajo/engine/planner/logical/BinaryNode.java |    77 +
 .../planner/logical/CreateDatabaseNode.java     |    87 +
 .../engine/planner/logical/CreateTableNode.java |   145 +
 .../planner/logical/DropDatabaseNode.java       |    85 +
 .../engine/planner/logical/DropTableNode.java   |    95 +
 .../engine/planner/logical/EvalExprNode.java    |    83 +
 .../tajo/engine/planner/logical/ExceptNode.java |    45 +
 .../engine/planner/logical/GroupbyNode.java     |   198 +
 .../tajo/engine/planner/logical/HavingNode.java |    67 +
 .../engine/planner/logical/IndexScanNode.java   |   122 +
 .../tajo/engine/planner/logical/InsertNode.java |   182 +
 .../engine/planner/logical/IntersectNode.java   |    44 +
 .../tajo/engine/planner/logical/JoinNode.java   |   165 +
 .../tajo/engine/planner/logical/LimitNode.java  |    65 +
 .../engine/planner/logical/LogicalNode.java     |   128 +
 .../planner/logical/LogicalNodeVisitor.java     |    27 +
 .../engine/planner/logical/LogicalRootNode.java |    41 +
 .../tajo/engine/planner/logical/NodeType.java   |    66 +
 .../logical/PartitionedTableScanNode.java       |   155 +
 .../planner/logical/PersistentStoreNode.java    |    90 +
 .../engine/planner/logical/Projectable.java     |    73 +
 .../engine/planner/logical/ProjectionNode.java  |   114 +
 .../engine/planner/logical/RelationNode.java    |    49 +
 .../tajo/engine/planner/logical/ScanNode.java   |   234 +
 .../engine/planner/logical/SelectionNode.java   |    69 +
 .../planner/logical/ShuffleFileWriteNode.java   |   104 +
 .../tajo/engine/planner/logical/SortNode.java   |    94 +
 .../engine/planner/logical/StoreTableNode.java  |   100 +
 .../planner/logical/TableSubQueryNode.java      |   181 +
 .../tajo/engine/planner/logical/UnaryNode.java  |    69 +
 .../tajo/engine/planner/logical/UnionNode.java  |    37 +
 .../tajo/engine/planner/logical/join/Edge.java  |    50 +
 .../planner/logical/join/FoundJoinOrder.java    |    47 +
 .../join/GreedyHeuristicJoinOrderAlgorithm.java |   262 +
 .../engine/planner/logical/join/JoinEdge.java   |    76 +
 .../engine/planner/logical/join/JoinGraph.java  |   115 +
 .../logical/join/JoinOrderAlgorithm.java        |    46 +
 .../planner/physical/AggregationExec.java       |    69 +
 .../engine/planner/physical/BNLJoinExec.java    |   223 +
 .../planner/physical/BSTIndexScanExec.java      |   143 +
 .../physical/BasicPhysicalExecutorVisitor.java  |   265 +
 .../planner/physical/BinaryPhysicalExec.java    |   109 +
 .../planner/physical/ColPartitionStoreExec.java |   100 +
 .../engine/planner/physical/EvalExprExec.java   |    66 +
 .../planner/physical/ExternalSortExec.java      |   782 ++
 .../planner/physical/HashAggregateExec.java     |   115 +
 .../HashBasedColPartitionStoreExec.java         |   132 +
 .../planner/physical/HashFullOuterJoinExec.java |   252 +
 .../engine/planner/physical/HashJoinExec.java   |   202 +
 .../planner/physical/HashLeftAntiJoinExec.java  |   109 +
 .../planner/physical/HashLeftOuterJoinExec.java |   212 +
 .../planner/physical/HashLeftSemiJoinExec.java  |   107 +
 .../planner/physical/HashPartitioner.java       |    49 +
 .../physical/HashShuffleFileWriteExec.java      |   160 +
 .../engine/planner/physical/HavingExec.java     |    50 +
 .../planner/physical/JoinTupleComparator.java   |    87 +
 .../tajo/engine/planner/physical/LimitExec.java |    55 +
 .../engine/planner/physical/MemSortExec.java    |    89 +
 .../physical/MergeFullOuterJoinExec.java        |   335 +
 .../engine/planner/physical/MergeJoinExec.java  |   193 +
 .../engine/planner/physical/NLJoinExec.java     |   104 +
 .../planner/physical/NLLeftOuterJoinExec.java   |   122 +
 .../physical/PartitionMergeScanExec.java        |   143 +
 .../engine/planner/physical/Partitioner.java    |    45 +
 .../engine/planner/physical/PhysicalExec.java   |    79 +
 .../physical/PhysicalExecutorVisitor.java       |   107 +
 .../planner/physical/PhysicalPlanUtil.java      |    39 +
 .../physical/PhysicalPlanningException.java     |    31 +
 .../engine/planner/physical/ProjectionExec.java |    69 +
 .../physical/RangeShuffleFileWriteExec.java     |   131 +
 .../physical/RightOuterMergeJoinExec.java       |   346 +
 .../engine/planner/physical/SelectionExec.java  |    49 +
 .../engine/planner/physical/SeqScanExec.java    |   316 +
 .../planner/physical/SortAggregateExec.java     |   125 +
 .../SortBasedColPartitionStoreExec.java         |   149 +
 .../tajo/engine/planner/physical/SortExec.java  |    51 +
 .../engine/planner/physical/StoreTableExec.java |   102 +
 .../engine/planner/physical/TunnelExec.java     |    42 +
 .../planner/physical/UnaryPhysicalExec.java     |    96 +
 .../tajo/engine/planner/physical/UnionExec.java |    62 +
 .../rewrite/BasicQueryRewriteEngine.java        |    70 +
 .../planner/rewrite/FilterPushDownRule.java     |   323 +
 .../rewrite/PartitionedTableRewriter.java       |   373 +
 .../planner/rewrite/ProjectionPushDownRule.java |   966 ++
 .../planner/rewrite/QueryRewriteEngine.java     |    32 +
 .../engine/planner/rewrite/RewriteRule.java     |    56 +
 .../apache/tajo/engine/query/QueryContext.java  |   191 +
 .../tajo/engine/query/QueryUnitRequest.java     |    50 +
 .../tajo/engine/query/QueryUnitRequestImpl.java |   328 +
 .../apache/tajo/engine/utils/SchemaUtil.java    |    72 +
 .../apache/tajo/engine/utils/ThreadUtil.java    |   149 +
 .../apache/tajo/engine/utils/TupleCache.java    |   122 +
 .../apache/tajo/engine/utils/TupleCacheKey.java |    60 +
 .../tajo/engine/utils/TupleCacheScanner.java    |   109 +
 .../org/apache/tajo/engine/utils/TupleUtil.java |   231 +
 .../tajo/master/AbstractTaskScheduler.java      |    56 +
 .../org/apache/tajo/master/ContainerProxy.java  |    81 +
 .../DefaultFragmentScheduleAlgorithm.java       |   247 +
 .../tajo/master/DefaultTaskScheduler.java       |   896 ++
 .../apache/tajo/master/FetchScheduleEvent.java  |    40 +
 .../org/apache/tajo/master/FragmentPair.java    |    73 +
 .../tajo/master/FragmentScheduleAlgorithm.java  |    38 +
 .../FragmentScheduleAlgorithmFactory.java       |    68 +
 .../org/apache/tajo/master/GlobalEngine.java    |   738 ++
 .../master/GreedyFragmentScheduleAlgorithm.java |   421 +
 .../apache/tajo/master/LazyTaskScheduler.java   |   522 +
 .../apache/tajo/master/ScheduledFetches.java    |    48 +
 .../apache/tajo/master/TajoAsyncDispatcher.java |   232 +
 .../apache/tajo/master/TajoContainerProxy.java  |   178 +
 .../java/org/apache/tajo/master/TajoMaster.java |   579 +
 .../tajo/master/TajoMasterClientService.java    |   754 ++
 .../apache/tajo/master/TajoMasterService.java   |   172 +
 .../tajo/master/TaskRunnerGroupEvent.java       |    51 +
 .../apache/tajo/master/TaskRunnerLauncher.java  |    25 +
 .../tajo/master/TaskSchedulerContext.java       |    65 +
 .../tajo/master/TaskSchedulerFactory.java       |    69 +
 .../java/org/apache/tajo/master/TaskState.java  |    23 +
 .../apache/tajo/master/YarnContainerProxy.java  |   414 +
 .../tajo/master/YarnTaskRunnerLauncherImpl.java |   200 +
 .../apache/tajo/master/cluster/ServerName.java  |   123 +
 .../master/event/ContainerAllocationEvent.java  |    77 +
 .../event/ContainerAllocatorEventType.java      |    26 +
 .../tajo/master/event/ContainerEvent.java       |    37 +
 .../master/event/FragmentScheduleEvent.java     |    61 +
 .../event/GrouppedContainerAllocatorEvent.java  |    45 +
 .../tajo/master/event/LocalTaskEvent.java       |    45 +
 .../tajo/master/event/LocalTaskEventType.java   |    23 +
 .../tajo/master/event/QueryCompletedEvent.java  |    42 +
 .../event/QueryDiagnosticsUpdateEvent.java      |    34 +
 .../apache/tajo/master/event/QueryEvent.java    |    36 +
 .../tajo/master/event/QueryEventType.java       |    36 +
 .../event/QueryMasterQueryCompletedEvent.java   |    39 +
 .../tajo/master/event/QueryStartEvent.java      |    74 +
 .../tajo/master/event/QuerySubQueryEvent.java   |    35 +
 .../event/QueryUnitAttemptScheduleEvent.java    |    87 +
 .../master/event/SubQueryCompletedEvent.java    |    42 +
 .../event/SubQueryContainerAllocationEvent.java |    38 +
 .../event/SubQueryDiagnosticsUpdateEvent.java   |    34 +
 .../apache/tajo/master/event/SubQueryEvent.java |    35 +
 .../tajo/master/event/SubQueryEventType.java    |    43 +
 .../tajo/master/event/SubQueryTaskEvent.java    |    43 +
 .../master/event/TaskAttemptAssignedEvent.java  |    48 +
 .../tajo/master/event/TaskAttemptEvent.java     |    36 +
 .../tajo/master/event/TaskAttemptEventType.java |    54 +
 .../master/event/TaskAttemptScheduleEvent.java  |    37 +
 .../event/TaskAttemptStatusUpdateEvent.java     |    36 +
 .../tajo/master/event/TaskCompletionEvent.java  |    35 +
 .../org/apache/tajo/master/event/TaskEvent.java |    35 +
 .../apache/tajo/master/event/TaskEventType.java |    38 +
 .../tajo/master/event/TaskFatalErrorEvent.java  |    41 +
 .../tajo/master/event/TaskRequestEvent.java     |    59 +
 .../tajo/master/event/TaskSchedulerEvent.java   |    41 +
 .../tajo/master/event/TaskTAttemptEvent.java    |    34 +
 .../master/metrics/CatalogMetricsGaugeSet.java  |    56 +
 .../metrics/WorkerResourceMetricsGaugeSet.java  |    74 +
 .../apache/tajo/master/querymaster/Query.java   |   702 ++
 .../master/querymaster/QueryInProgress.java     |   294 +
 .../tajo/master/querymaster/QueryInfo.java      |   125 +
 .../tajo/master/querymaster/QueryJobEvent.java  |    44 +
 .../master/querymaster/QueryJobManager.java     |   195 +
 .../tajo/master/querymaster/QueryMaster.java    |   504 +
 .../querymaster/QueryMasterManagerService.java  |   239 +
 .../master/querymaster/QueryMasterRunner.java   |   149 +
 .../master/querymaster/QueryMasterTask.java     |   556 +
 .../tajo/master/querymaster/QueryUnit.java      |   658 +
 .../master/querymaster/QueryUnitAttempt.java    |   442 +
 .../tajo/master/querymaster/Repartitioner.java  |   653 +
 .../tajo/master/querymaster/SubQuery.java       |  1125 ++
 .../tajo/master/querymaster/SubQueryState.java  |    30 +
 .../apache/tajo/master/rm/TajoRMContext.java    |    84 +
 .../tajo/master/rm/TajoResourceTracker.java     |   253 +
 .../tajo/master/rm/TajoWorkerContainer.java     |   100 +
 .../tajo/master/rm/TajoWorkerContainerId.java   |    93 +
 .../master/rm/TajoWorkerResourceManager.java    |   537 +
 .../java/org/apache/tajo/master/rm/Worker.java  |   296 +
 .../org/apache/tajo/master/rm/WorkerEvent.java  |    37 +
 .../apache/tajo/master/rm/WorkerEventType.java  |    30 +
 .../tajo/master/rm/WorkerLivelinessMonitor.java |    56 +
 .../tajo/master/rm/WorkerReconnectEvent.java    |    35 +
 .../apache/tajo/master/rm/WorkerResource.java   |   235 +
 .../tajo/master/rm/WorkerResourceManager.java   |   110 +
 .../org/apache/tajo/master/rm/WorkerState.java  |    44 +
 .../tajo/master/rm/WorkerStatusEvent.java       |    54 +
 .../master/rm/YarnRMContainerAllocator.java     |   237 +
 .../tajo/master/rm/YarnTajoResourceManager.java |   349 +
 .../master/session/InvalidSessionException.java |    25 +
 .../session/NoSuchSessionVariableException.java |    25 +
 .../org/apache/tajo/master/session/Session.java |   124 +
 .../tajo/master/session/SessionConstants.java   |    23 +
 .../tajo/master/session/SessionEvent.java       |    34 +
 .../tajo/master/session/SessionEventType.java   |    24 +
 .../session/SessionLivelinessMonitor.java       |    53 +
 .../tajo/master/session/SessionManager.java     |   139 +
 .../org/apache/tajo/net/CachedDNSResolver.java  |    64 +
 .../apache/tajo/util/ApplicationIdUtils.java    |    44 +
 .../java/org/apache/tajo/util/ClassUtil.java    |   155 +
 .../java/org/apache/tajo/util/GeoIPUtil.java    |    46 +
 .../java/org/apache/tajo/util/IndexUtil.java    |   148 +
 .../main/java/org/apache/tajo/util/JSPUtil.java |   209 +
 .../util/metrics/GroupNameMetricsFilter.java    |    43 +
 .../tajo/util/metrics/LogEventGaugeSet.java     |    64 +
 .../tajo/util/metrics/MetricsFilterList.java    |    43 +
 .../tajo/util/metrics/RegexpMetricsFilter.java  |    51 +
 .../tajo/util/metrics/TajoLogEventCounter.java  |    86 +
 .../apache/tajo/util/metrics/TajoMetrics.java   |   133 +
 .../tajo/util/metrics/TajoSystemMetrics.java    |   213 +
 .../util/metrics/reporter/GangliaReporter.java  |   258 +
 .../reporter/MetricsConsoleReporter.java        |    80 +
 .../MetricsConsoleScheduledReporter.java        |    32 +
 .../reporter/MetricsFileScheduledReporter.java  |    58 +
 .../MetricsStreamScheduledReporter.java         |   179 +
 .../util/metrics/reporter/NullReporter.java     |    31 +
 .../metrics/reporter/TajoMetricsReporter.java   |   232 +
 .../reporter/TajoMetricsScheduledReporter.java  |   206 +
 .../java/org/apache/tajo/webapp/HttpServer.java |   447 +
 .../tajo/webapp/QueryExecutorServlet.java       |   376 +
 .../apache/tajo/webapp/StaticHttpServer.java    |    69 +
 .../tajo/worker/AbstractResourceAllocator.java  |    54 +
 .../org/apache/tajo/worker/DeletionService.java |   115 +
 .../java/org/apache/tajo/worker/Fetcher.java    |   256 +
 .../apache/tajo/worker/InterDataRetriever.java  |   113 +
 .../tajo/worker/PartitionRetrieverHandler.java  |    44 +
 .../tajo/worker/RangeRetrieverHandler.java      |   163 +
 .../apache/tajo/worker/ResourceAllocator.java   |    29 +
 .../org/apache/tajo/worker/TajoQueryEngine.java |    49 +
 .../tajo/worker/TajoResourceAllocator.java      |   339 +
 .../java/org/apache/tajo/worker/TajoWorker.java |   583 +
 .../tajo/worker/TajoWorkerClientService.java    |   221 +
 .../tajo/worker/TajoWorkerManagerService.java   |   152 +
 .../main/java/org/apache/tajo/worker/Task.java  |   762 ++
 .../apache/tajo/worker/TaskAttemptContext.java  |   270 +
 .../org/apache/tajo/worker/TaskHistory.java     |   198 +
 .../java/org/apache/tajo/worker/TaskRunner.java |   431 +
 .../apache/tajo/worker/TaskRunnerManager.java   |   238 +
 .../tajo/worker/WorkerHeartbeatService.java     |   289 +
 .../tajo/worker/YarnResourceAllocator.java      |   117 +
 .../FileAccessForbiddenException.java           |    40 +
 .../tajo/worker/dataserver/HttpDataServer.java  |    87 +
 .../dataserver/HttpDataServerHandler.java       |   199 +
 .../HttpDataServerPipelineFactory.java          |    55 +
 .../apache/tajo/worker/dataserver/HttpUtil.java |    69 +
 .../retriever/AdvancedDataRetriever.java        |   128 +
 .../dataserver/retriever/DataRetriever.java     |    29 +
 .../retriever/DirectoryRetriever.java           |    56 +
 .../worker/dataserver/retriever/FileChunk.java  |    51 +
 .../dataserver/retriever/RetrieverHandler.java  |    33 +
 tajo-core/src/main/proto/InternalTypes.proto    |    32 +
 .../src/main/proto/QueryMasterProtocol.proto    |    42 +
 .../main/proto/ResourceTrackerProtocol.proto    |    40 +
 .../src/main/proto/TajoMasterProtocol.proto     |   154 +
 .../src/main/proto/TajoWorkerProtocol.proto     |   258 +
 .../src/main/resources/catalog-default.xml      |    32 +
 tajo-core/src/main/resources/log4j.properties   |    28 +
 tajo-core/src/main/resources/tajo-default.xml   |    50 +
 .../src/main/resources/tajo-metrics.properties  |    75 +
 .../webapps/admin/WEB-INF/jetty-web.xml         |    23 +
 .../resources/webapps/admin/WEB-INF/web.xml     |    27 +
 .../resources/webapps/admin/catalogview.jsp     |   166 +
 .../main/resources/webapps/admin/cluster.jsp    |   216 +
 .../src/main/resources/webapps/admin/conf.jsp   |    56 +
 .../src/main/resources/webapps/admin/env.jsp    |    67 +
 .../main/resources/webapps/admin/functions.jsp  |    77 +
 .../src/main/resources/webapps/admin/getCSV.jsp |    31 +
 .../src/main/resources/webapps/admin/header.jsp |    32 +
 .../src/main/resources/webapps/admin/index.jsp  |   184 +
 .../src/main/resources/webapps/admin/query.jsp  |   137 +
 .../resources/webapps/admin/query_executor.jsp  |   312 +
 .../src/main/resources/webapps/admin/thread.jsp |    45 +
 .../resources/webapps/static/img/logo_tajo.gif  |   Bin 0 -> 3025 bytes
 .../main/resources/webapps/static/img/on.jpg    |   Bin 0 -> 636 bytes
 .../resources/webapps/static/img/tajo_logo.png  |   Bin 0 -> 7592 bytes
 .../resources/webapps/static/img/tajochar.jpg   |   Bin 0 -> 325871 bytes
 .../static/img/tajochar_catalog_small.jpg       |   Bin 0 -> 13275 bytes
 .../webapps/static/img/tajochar_little.jpg      |   Bin 0 -> 27712 bytes
 .../webapps/static/img/tajochar_middle.jpg      |   Bin 0 -> 129938 bytes
 .../static/img/tajochar_queries_small.jpg       |   Bin 0 -> 5184 bytes
 .../webapps/static/img/tajochar_small.jpg       |   Bin 0 -> 65024 bytes
 .../webapps/static/img/tajochar_title.jpg       |   Bin 0 -> 40249 bytes
 .../webapps/static/img/tajochar_title_small.jpg |   Bin 0 -> 10308 bytes
 .../static/img/tajochar_worker_small.jpg        |   Bin 0 -> 13329 bytes
 .../webapps/static/js/jquery-ui.min.js          |     5 +
 .../main/resources/webapps/static/js/jquery.js  |     2 +
 .../static/js/jquery.jsPlumb-1.3.16-all-min.js  |     1 +
 .../static/js/jquery.jsPlumb-1.3.16-all.js      | 10561 +++++++++++++++++
 .../main/resources/webapps/static/queryplan.css |    55 +
 .../src/main/resources/webapps/static/style.css |    74 +
 .../resources/webapps/worker/WEB-INF/web.xml    |    27 +
 .../src/main/resources/webapps/worker/conf.jsp  |    55 +
 .../src/main/resources/webapps/worker/env.jsp   |    69 +
 .../main/resources/webapps/worker/header.jsp    |    37 +
 .../src/main/resources/webapps/worker/index.jsp |   150 +
 .../resources/webapps/worker/querydetail.jsp    |   105 +
 .../main/resources/webapps/worker/queryplan.jsp |   237 +
 .../resources/webapps/worker/querytasks.jsp     |   232 +
 .../main/resources/webapps/worker/queryunit.jsp |   171 +
 .../resources/webapps/worker/taskcontainers.jsp |    87 +
 .../resources/webapps/worker/taskdetail.jsp     |   127 +
 .../src/main/resources/webapps/worker/tasks.jsp |    94 +
 .../main/resources/webapps/worker/thread.jsp    |    45 +
 tajo-core/src/test/java/log4j.properties        |    28 +
 .../org/apache/tajo/BackendTestingUtil.java     |    76 +
 .../java/org/apache/tajo/IntegrationTest.java   |    22 +
 .../apache/tajo/LocalTajoTestingUtility.java    |   136 +
 .../org/apache/tajo/MiniTajoYarnCluster.java    |   175 +
 .../java/org/apache/tajo/QueryTestCaseBase.java |   539 +
 .../org/apache/tajo/TajoTestingCluster.java     |   630 +
 .../org/apache/tajo/TestQueryIdFactory.java     |    58 +
 .../test/java/org/apache/tajo/TestTajoIds.java  |   168 +
 .../test/java/org/apache/tajo/TpchTestBase.java |   117 +
 .../org/apache/tajo/benchmark/TestTPCH.java     |    56 +
 .../tajo/cli/TestExecExternalShellCommand.java  |    46 +
 .../org/apache/tajo/cli/TestHdfsCommand.java    |    46 +
 .../org/apache/tajo/cli/TestSimpleParser.java   |   179 +
 .../org/apache/tajo/client/TestDDLBuilder.java  |   133 +
 .../org/apache/tajo/client/TestTajoClient.java  |   666 ++
 .../org/apache/tajo/cluster/TestServerName.java |   102 +
 .../apache/tajo/engine/eval/ExprTestBase.java   |   210 +
 .../apache/tajo/engine/eval/TestEvalTree.java   |   323 +
 .../tajo/engine/eval/TestEvalTreeUtil.java      |   359 +
 .../apache/tajo/engine/eval/TestPredicates.java |   396 +
 .../tajo/engine/eval/TestSQLDateTimeTypes.java  |    53 +
 .../tajo/engine/eval/TestSQLExpression.java     |   206 +
 .../tajo/engine/function/TestAggFunction.java   |    64 +
 .../engine/function/TestBuiltinFunctions.java   |   123 +
 .../function/TestConditionalExpressions.java    |    82 +
 .../engine/function/TestDateTimeFunctions.java  |   253 +
 .../engine/function/TestGeneralFunction.java    |    49 +
 .../tajo/engine/function/TestMathFunctions.java |   428 +
 .../function/TestPatternMatchingPredicates.java |   140 +
 .../TestStringOperatorsAndFunctions.java        |   610 +
 .../tajo/engine/parser/TestHiveQLAnalyzer.java  |   271 +
 .../tajo/engine/parser/TestSQLAnalyzer.java     |   473 +
 .../tajo/engine/planner/TestLogicalNode.java    |    74 +
 .../engine/planner/TestLogicalOptimizer.java    |   263 +
 .../tajo/engine/planner/TestLogicalPlan.java    |   125 +
 .../tajo/engine/planner/TestLogicalPlanner.java |  1136 ++
 .../tajo/engine/planner/TestPlannerUtil.java    |   321 +
 .../engine/planner/TestSimpleDirectedGraph.java |    79 +
 .../planner/TestSimpleUndirectedGraph.java      |    96 +
 .../planner/TestUniformRangePartition.java      |   442 +
 .../planner/global/TestBroadcastJoinPlan.java   |   400 +
 .../engine/planner/global/TestMasterPlan.java   |    53 +
 .../planner/physical/TestBNLJoinExec.java       |   217 +
 .../planner/physical/TestBSTIndexExec.java      |   210 +
 .../planner/physical/TestExternalSortExec.java  |   184 +
 .../physical/TestFullOuterHashJoinExec.java     |   403 +
 .../physical/TestFullOuterMergeJoinExec.java    |   536 +
 .../planner/physical/TestHashAntiJoinExec.java  |   209 +
 .../planner/physical/TestHashJoinExec.java      |   275 +
 .../planner/physical/TestHashPartitioner.java   |    84 +
 .../planner/physical/TestHashSemiJoinExec.java  |   214 +
 .../physical/TestLeftOuterHashJoinExec.java     |   463 +
 .../physical/TestLeftOuterNLJoinExec.java       |   474 +
 .../planner/physical/TestMergeJoinExec.java     |   193 +
 .../engine/planner/physical/TestNLJoinExec.java |   209 +
 .../planner/physical/TestPhysicalPlanner.java   |  1023 ++
 .../physical/TestProgressExternalSortExec.java  |   233 +
 .../physical/TestRightOuterHashJoinExec.java    |   350 +
 .../physical/TestRightOuterMergeJoinExec.java   |   520 +
 .../engine/planner/physical/TestSortExec.java   |   166 +
 .../tajo/engine/query/TestAlterTable.java       |    52 +
 .../tajo/engine/query/TestAlterTablespace.java  |    59 +
 .../apache/tajo/engine/query/TestCTASQuery.java |   239 +
 .../tajo/engine/query/TestCaseByCases.java      |    70 +
 .../tajo/engine/query/TestCreateDatabase.java   |    75 +
 .../tajo/engine/query/TestCreateTable.java      |   360 +
 .../apache/tajo/engine/query/TestDropTable.java |    38 +
 .../tajo/engine/query/TestGroupByQuery.java     |   246 +
 .../tajo/engine/query/TestInsertQuery.java      |   262 +
 .../tajo/engine/query/TestJoinBroadcast.java    |   377 +
 .../query/TestJoinOnPartitionedTables.java      |    56 +
 .../apache/tajo/engine/query/TestJoinQuery.java |   325 +
 .../apache/tajo/engine/query/TestNetTypes.java  |   102 +
 .../tajo/engine/query/TestNullValues.java       |   162 +
 .../tajo/engine/query/TestSelectQuery.java      |   303 +
 .../apache/tajo/engine/query/TestSortQuery.java |   146 +
 .../tajo/engine/query/TestTablePartitions.java  |   439 +
 .../tajo/engine/query/TestTableSubQuery.java    |    60 +
 .../tajo/engine/query/TestUnionQuery.java       |   141 +
 .../apache/tajo/engine/util/TestTupleCache.java |    89 +
 .../apache/tajo/engine/util/TestTupleUtil.java  |   171 +
 .../org/apache/tajo/jdbc/TestResultSet.java     |   127 +
 .../tajo/jdbc/TestTajoDatabaseMetaData.java     |   355 +
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java |   386 +
 .../tajo/master/TestExecutionBlockCursor.java   |   123 +
 .../apache/tajo/master/TestGlobalPlanner.java   |   205 +
 .../apache/tajo/master/TestRepartitioner.java   |    77 +
 .../querymaster/TestQueryUnitStatusUpdate.java  |   175 +
 .../tajo/master/rm/TestTajoResourceManager.java |   378 +
 .../apache/tajo/storage/TestFileFragment.java   |    93 +
 .../org/apache/tajo/storage/TestRowFile.java    |   145 +
 .../java/org/apache/tajo/util/TestJSPUtil.java  |    84 +
 .../tajo/util/metrics/TestMetricsFilter.java    |    52 +
 .../tajo/util/metrics/TestSystemMetrics.java    |   133 +
 .../org/apache/tajo/worker/TestFetcher.java     |    94 +
 .../tajo/worker/TestRangeRetrieverHandler.java  |   373 +
 .../worker/dataserver/TestHttpDataServer.java   |   172 +
 .../tajo/worker/dataserver/TestHttpUtil.java    |    49 +
 .../resources/dataset/TestAlterTable/table1.tbl |     3 +
 .../dataset/TestCreateTable/table1/table1.tbl   |     3 +
 .../dataset/TestCreateTable/table2/table2.tbl   |     3 +
 .../resources/dataset/TestDropTable/table1.tbl  |     3 +
 .../customer_large/customer.tbl                 |   100 +
 .../lineitem_large/lineitem.tbl                 |    97 +
 .../dataset/TestJoinQuery/table1/table1.tbl     |     5 +
 .../dataset/TestJoinQuery/table2/table2.tbl     |     4 +
 .../dataset/TestNetTypes/table1/table1.tbl      |     5 +
 .../dataset/TestNetTypes/table2/table2.tbl      |     5 +
 .../dataset/TestSortQuery/table1/table1.tbl     |     5 +
 .../dataset/TestSortQuery/table2/table2.tbl     |    24 +
 tajo-core/src/test/resources/log4j.properties   |    28 +
 .../org/apache/tajo/client/TestTajoDump.java    |    44 +
 .../alter_table_add_new_column_ddl.sql          |     1 +
 .../alter_table_rename_column_ddl.sql           |     1 +
 .../alter_table_rename_table_ddl.sql            |     1 +
 .../queries/TestAlterTable/table1_ddl.sql       |     1 +
 .../TestBuiltinFunctions/testAvgDouble.sql      |     1 +
 .../queries/TestBuiltinFunctions/testAvgInt.sql |     1 +
 .../TestBuiltinFunctions/testAvgLong.sql        |     1 +
 .../queries/TestBuiltinFunctions/testCount.sql  |     1 +
 .../TestBuiltinFunctions/testMaxLong.sql        |     1 +
 .../TestBuiltinFunctions/testMaxString.sql      |     1 +
 .../TestBuiltinFunctions/testMinLong.sql        |     1 +
 .../TestBuiltinFunctions/testMinString.sql      |     1 +
 .../queries/TestBuiltinFunctions/testRandom.sql |     1 +
 .../TestBuiltinFunctions/testSplitPart.sql      |     1 +
 .../testSplitPartByString.sql                   |     1 +
 .../testSplitPartNested.sql                     |     1 +
 .../queries/TestCTASQuery/CtasWithGroupby.sql   |     2 +
 .../queries/TestCTASQuery/CtasWithLimit.sql     |    13 +
 .../TestCTASQuery/CtasWithManagedTable.sql      |    12 +
 .../queries/TestCTASQuery/CtasWithOptions.sql   |    13 +
 .../queries/TestCTASQuery/CtasWithOrderby.sql   |    11 +
 .../queries/TestCTASQuery/CtasWithStoreType.sql |    12 +
 .../queries/TestCTASQuery/CtasWithUnion.sql     |    35 +
 .../resources/queries/TestCTASQuery/check1.sql  |     1 +
 .../resources/queries/TestCTASQuery/check2.sql  |     1 +
 .../testCtasWithColumnedPartition.sql           |     2 +
 .../TestCTASQuery/testCtasWithGroupby.sql       |     1 +
 .../queries/TestCTASQuery/testCtasWithLimit.sql |     1 +
 .../TestCTASQuery/testCtasWithOptions.sql       |     1 +
 .../TestCTASQuery/testCtasWithOrderby.sql       |     1 +
 .../TestCTASQuery/testCtasWithStoreType.sql     |     1 +
 .../queries/TestCTASQuery/testCtasWithUnion.sql |     1 +
 .../testCtasWithoutTableDefinition.sql          |     2 +
 .../queries/TestCaseByCases/testTAJO415Case.sql |    36 +
 .../queries/TestCaseByCases/testTAJO418Case.sql |    29 +
 .../queries/TestCaseByCases/testTAJO619Case.sql |     4 +
 .../queries/TestCaseByCases/testTAJO718Case.sql |    10 +
 .../queries/TestCaseByCases/testTAJO739Case.sql |     5 +
 .../create_table_various_types.sql              |    50 +
 .../create_table_various_types_for_hcatalog.sql |    50 +
 .../quoted_identifier_mixed_chars_1.sql         |     1 +
 .../quoted_identifier_mixed_chars_2.sql         |     1 +
 .../quoted_identifier_mixed_chars_3.sql         |    22 +
 .../quoted_identifier_mixed_chars_ddl_1.sql     |     1 +
 .../quoted_identifier_mixed_chars_ddl_2.sql     |     1 +
 .../quoted_identifier_non_ascii_1.sql           |     1 +
 .../quoted_identifier_non_ascii_2.sql           |     1 +
 .../quoted_identifier_non_ascii_3.sql           |     1 +
 .../quoted_identifier_non_ascii_ddl.sql         |     1 +
 .../queries/TestCreateTable/table1_ddl.sql      |     1 +
 .../queries/TestDropTable/drop_table_ddl.sql    |     1 +
 .../queries/TestDropTable/table1_ddl.sql        |     1 +
 .../TestGroupByQuery/testComplexParameter.sql   |     1 +
 .../TestGroupByQuery/testComplexParameter2.sql  |     1 +
 .../testComplexParameterWithSubQuery.sql        |     6 +
 .../testDistinctAggregation1.sql                |     1 +
 .../testDistinctAggregation2.sql                |     1 +
 .../testDistinctAggregation3.sql                |     1 +
 .../testDistinctAggregation4.sql                |     1 +
 .../testDistinctAggregation5.sql                |     1 +
 .../testDistinctAggregation6.sql                |     9 +
 .../testDistinctAggregationWithHaving1.sql      |     1 +
 .../testDistinctAggregationWithUnion1.sql       |    21 +
 .../queries/TestGroupByQuery/testGroupBy.sql    |     1 +
 .../queries/TestGroupByQuery/testGroupBy2.sql   |     1 +
 .../queries/TestGroupByQuery/testGroupBy3.sql   |     1 +
 .../queries/TestGroupByQuery/testGroupBy4.sql   |     9 +
 .../queries/TestGroupByQuery/testGroupBy5.sql   |     1 +
 .../TestGroupByQuery/testGroupByNested1.sql     |     8 +
 .../TestGroupByQuery/testGroupByNested2.sql     |    10 +
 .../testGroupByWithConstantKeys1.sql            |     1 +
 .../testGroupByWithExpressionKeys1.sql          |     1 +
 .../testGroupByWithExpressionKeys2.sql          |     1 +
 .../testGroupByWithSameExprs1.sql               |     6 +
 .../testGroupByWithSameExprs2.sql               |     6 +
 .../testHavingWithAggFunction.sql               |     1 +
 .../testHavingWithNamedTarget.sql               |     1 +
 .../create_customer_large_ddl.sql               |     7 +
 .../create_lineitem_large_ddl.sql               |     7 +
 .../queries/TestJoinBroadcast/oj_table1_ddl.sql |     6 +
 .../queries/TestJoinBroadcast/oj_table2_ddl.sql |     6 +
 .../testBroadcastBasicJoin.sql                  |    11 +
 .../TestJoinBroadcast/testBroadcastSubquery.sql |    11 +
 .../testBroadcastSubquery2.sql                  |    19 +
 .../testBroadcastTwoPartJoin.sql                |    15 +
 .../queries/TestJoinBroadcast/testCrossJoin.sql |     1 +
 .../testCrossJoinAndCaseWhen.sql                |    18 +
 .../testCrossJoinWithAsterisk1.sql              |     1 +
 .../testCrossJoinWithAsterisk2.sql              |     1 +
 .../testCrossJoinWithAsterisk3.sql              |     1 +
 .../testCrossJoinWithAsterisk4.sql              |     1 +
 .../testCrossJoinWithEmptyTable1.sql            |     8 +
 .../TestJoinBroadcast/testFullOuterJoin1.sql    |     8 +
 .../testFullOuterJoinWithEmptyTable1.sql        |     8 +
 .../testInnerJoinWithEmptyTable.sql             |     8 +
 .../testJoinCoReferredEvals1.sql                |    11 +
 .../testJoinCoReferredEvalsWithSameExprs1.sql   |    14 +
 .../testJoinCoReferredEvalsWithSameExprs2.sql   |    22 +
 .../testJoinOnMultipleDatabases.sql             |    25 +
 .../TestJoinBroadcast/testLeftOuterJoin1.sql    |     9 +
 .../testLeftOuterJoinWithConstantExpr1.sql      |     9 +
 .../testLeftOuterJoinWithConstantExpr2.sql      |     9 +
 .../testLeftOuterJoinWithConstantExpr3.sql      |    17 +
 .../testLeftOuterJoinWithEmptyTable1.sql        |     9 +
 .../testLeftOuterJoinWithEmptyTable2.sql        |     9 +
 .../testLeftOuterJoinWithEmptyTable3.sql        |    14 +
 .../testLeftOuterJoinWithEmptyTable4.sql        |    17 +
 .../testOuterJoinAndCaseWhen1.sql               |    12 +
 .../TestJoinBroadcast/testRightOuterJoin1.sql   |     8 +
 .../testRightOuterJoinWithEmptyTable1.sql       |     8 +
 .../TestJoinBroadcast/testTPCHQ2Join.sql        |    25 +
 .../TestJoinBroadcast/testWhereClauseJoin1.sql  |    10 +
 .../TestJoinBroadcast/testWhereClauseJoin2.sql  |     8 +
 .../TestJoinBroadcast/testWhereClauseJoin3.sql  |     9 +
 .../TestJoinBroadcast/testWhereClauseJoin4.sql  |     8 +
 .../TestJoinBroadcast/testWhereClauseJoin5.sql  |    15 +
 .../TestJoinBroadcast/testWhereClauseJoin6.sql  |    19 +
 .../customer_ddl.sql                            |     9 +
 .../insert_into_customer.sql                    |    11 +
 .../selfJoinOfPartitionedTable.sql              |     9 +
 .../testNoProjectionJoinQual.sql                |     1 +
 .../testPartialFilterPushDown.sql               |     9 +
 .../testPartitionTableJoinSmallTable.sql        |    11 +
 .../queries/TestJoinQuery/oj_table1_ddl.sql     |     6 +
 .../queries/TestJoinQuery/oj_table2_ddl.sql     |     6 +
 .../queries/TestJoinQuery/testCrossJoin.sql     |     1 +
 .../TestJoinQuery/testCrossJoinAndCaseWhen.sql  |    18 +
 .../testCrossJoinWithAsterisk1.sql              |     1 +
 .../testCrossJoinWithAsterisk2.sql              |     1 +
 .../testCrossJoinWithAsterisk3.sql              |     1 +
 .../testCrossJoinWithAsterisk4.sql              |     1 +
 .../testCrossJoinWithEmptyTable1.sql            |     8 +
 .../TestJoinQuery/testFullOuterJoin1.sql        |     8 +
 .../testFullOuterJoinWithEmptyTable1.sql        |     8 +
 .../testInnerJoinWithEmptyTable.sql             |     8 +
 .../TestJoinQuery/testJoinCoReferredEvals1.sql  |    11 +
 .../testJoinCoReferredEvalsWithSameExprs1.sql   |    14 +
 .../testJoinCoReferredEvalsWithSameExprs2.sql   |    22 +
 .../testJoinOnMultipleDatabases.sql             |    25 +
 .../TestJoinQuery/testLeftOuterJoin1.sql        |     9 +
 .../testLeftOuterJoinWithConstantExpr1.sql      |     9 +
 .../testLeftOuterJoinWithConstantExpr2.sql      |     9 +
 .../testLeftOuterJoinWithConstantExpr3.sql      |    17 +
 .../testLeftOuterJoinWithEmptyTable1.sql        |     9 +
 .../testLeftOuterJoinWithEmptyTable2.sql        |     9 +
 .../testLeftOuterJoinWithEmptyTable3.sql        |    14 +
 .../testLeftOuterJoinWithEmptyTable4.sql        |    17 +
 .../TestJoinQuery/testOuterJoinAndCaseWhen1.sql |    12 +
 .../TestJoinQuery/testRightOuterJoin1.sql       |     8 +
 .../testRightOuterJoinWithEmptyTable1.sql       |     8 +
 .../queries/TestJoinQuery/testTPCHQ2Join.sql    |    25 +
 .../TestJoinQuery/testWhereClauseJoin1.sql      |    10 +
 .../TestJoinQuery/testWhereClauseJoin2.sql      |     8 +
 .../TestJoinQuery/testWhereClauseJoin3.sql      |     9 +
 .../TestJoinQuery/testWhereClauseJoin4.sql      |     8 +
 .../TestJoinQuery/testWhereClauseJoin5.sql      |    15 +
 .../TestJoinQuery/testWhereClauseJoin6.sql      |    19 +
 .../queries/TestNetTypes/table1_ddl.sql         |     4 +
 .../queries/TestNetTypes/table2_ddl.sql         |     4 +
 .../queries/TestNetTypes/testGroupby.sql        |     8 +
 .../queries/TestNetTypes/testGroupby2.sql       |     9 +
 .../resources/queries/TestNetTypes/testJoin.sql |     1 +
 .../queries/TestNetTypes/testSelect.sql         |     1 +
 .../resources/queries/TestNetTypes/testSort.sql |     1 +
 .../queries/TestNetTypes/testSort2.sql          |     1 +
 .../queries/TestQueryUnitStatusUpdate/case1.sql |     1 +
 .../queries/TestQueryUnitStatusUpdate/case2.sql |     5 +
 .../queries/TestQueryUnitStatusUpdate/case3.sql |    11 +
 .../queries/TestSelectQuery/testCaseWhen.sql    |    11 +
 .../TestSelectQuery/testCaseWhenWithoutElse.sql |    10 +
 .../TestSelectQuery/testCreateAfterSelect.sql   |     1 +
 .../TestSelectQuery/testDatabaseRef1.sql        |     1 +
 .../TestSelectQuery/testDatabaseRef2.sql        |     1 +
 .../TestSelectQuery/testDatabaseRef3.sql        |     1 +
 .../TestSelectQuery/testExplainSelect.sql       |     1 +
 .../queries/TestSelectQuery/testInClause.sql    |     1 +
 .../queries/TestSelectQuery/testInStrClause.sql |     1 +
 .../queries/TestSelectQuery/testLikeClause.sql  |     1 +
 .../queries/TestSelectQuery/testLimit.sql       |     1 +
 .../TestSelectQuery/testNonFromSelect1.sql      |     1 +
 .../queries/TestSelectQuery/testNotEqual.sql    |     1 +
 .../queries/TestSelectQuery/testNotInClause.sql |     1 +
 .../TestSelectQuery/testNotInStrClause.sql      |     1 +
 .../TestSelectQuery/testRealValueCompare.sql    |     1 +
 .../queries/TestSelectQuery/testSelect.sql      |     1 +
 .../queries/TestSelectQuery/testSelect2.sql     |     1 +
 .../queries/TestSelectQuery/testSelect3.sql     |     1 +
 .../TestSelectQuery/testSelectAsterik.sql       |     1 +
 .../TestSelectQuery/testSelectAsterisk1.sql     |     1 +
 .../TestSelectQuery/testSelectAsterisk2.sql     |     1 +
 .../TestSelectQuery/testSelectAsterisk3.sql     |     1 +
 .../TestSelectQuery/testSelectAsterisk4.sql     |     1 +
 .../TestSelectQuery/testSelectAsterisk5.sql     |     1 +
 .../TestSelectQuery/testSelectColumnAlias1.sql  |     1 +
 .../TestSelectQuery/testSelectDistinct.sql      |     7 +
 ...tSelectSameConstantsWithDifferentAliases.sql |     1 +
 .../testSelectSameExprsWithDifferentAliases.sql |     1 +
 .../queries/TestSelectQuery/testSimpleQuery.sql |     1 +
 .../testSimpleQueryWithLimit.sql                |     1 +
 .../TestSelectQuery/testStringCompare.sql       |     1 +
 .../queries/TestSelectQuery/testWhereCond1.sql  |     1 +
 .../TestSelectQuery/testWhereCondWithAlias1.sql |     1 +
 .../TestSelectQuery/testWhereCondWithAlias2.sql |     1 +
 .../create_table_with_asc_desc_keys.sql         |     1 +
 .../create_table_with_date_ddl.sql              |    10 +
 .../queries/TestSortQuery/testAsterisk.sql      |     1 +
 .../queries/TestSortQuery/testSort.sql          |     1 +
 .../TestSortQuery/testSortAfterGroupby.sql      |     1 +
 .../testSortAfterGroupbyWithAlias.sql           |     1 +
 .../queries/TestSortQuery/testSortDesc.sql      |     1 +
 .../TestSortQuery/testSortWithAlias1.sql        |     1 +
 .../TestSortQuery/testSortWithAlias2.sql        |     9 +
 .../TestSortQuery/testSortWithAlias3.sql        |    11 +
 .../testSortWithAliasButOriginalName.sql        |     1 +
 .../TestSortQuery/testSortWithAscDescKeys.sql   |     1 +
 .../queries/TestSortQuery/testSortWithDate.sql  |     1 +
 .../queries/TestSortQuery/testSortWithExpr1.sql |     1 +
 .../queries/TestSortQuery/testSortWithExpr2.sql |     1 +
 .../queries/TestSortQuery/testTopK.sql          |     1 +
 .../queries/TestTPCH/testQ1OrderBy.sql          |    12 +
 .../queries/TestTPCH/testQ2FourJoins.sql        |    18 +
 .../queries/TestTPCH/testTPCH14Expr.sql         |    12 +
 .../queries/TestTablePartitions/case1.sql       |    16 +
 .../queries/TestTablePartitions/case2.sql       |    31 +
 .../queries/TestTablePartitions/case3.sql       |     8 +
 .../TestTableSubQuery/testGroupBySubQuery.sql   |     1 +
 .../TestTableSubQuery/testJoinSubQuery.sql      |     5 +
 .../TestTableSubQuery/testJoinSubQuery2.sql     |     4 +
 .../TestTableSubQuery/testTableSubquery1.sql    |     1 +
 .../queries/TestUnionQuery/testUnion1.sql       |    19 +
 .../queries/TestUnionQuery/testUnion10.sql      |   158 +
 .../queries/TestUnionQuery/testUnion2.sql       |    18 +
 .../queries/TestUnionQuery/testUnion3.sql       |    19 +
 .../queries/TestUnionQuery/testUnion4.sql       |    16 +
 .../queries/TestUnionQuery/testUnion5.sql       |    20 +
 .../queries/TestUnionQuery/testUnion6.sql       |    15 +
 .../queries/TestUnionQuery/testUnion7.sql       |    18 +
 .../queries/TestUnionQuery/testUnion8.sql       |    24 +
 .../queries/TestUnionQuery/testUnion9.sql       |    29 +
 .../testUnionWithSameAliasNames.sql             |    25 +
 .../resources/queries/default/asterisk_1.sql    |     1 +
 .../resources/queries/default/asterisk_2.sql    |     1 +
 .../resources/queries/default/asterisk_3.sql    |     1 +
 .../resources/queries/default/asterisk_4.sql    |     1 +
 .../queries/default/complex_union_1.sql         |    29 +
 .../queries/default/complex_union_2.sql         |    35 +
 .../create_partitioned_table_as_select.sql      |    17 +
 .../queries/default/create_table_1.hiveql       |     1 +
 .../queries/default/create_table_1.sql          |     1 +
 .../queries/default/create_table_10.sql         |     1 +
 .../queries/default/create_table_11.hiveql      |     3 +
 .../queries/default/create_table_11.sql         |     3 +
 .../queries/default/create_table_12.hiveql      |     4 +
 .../queries/default/create_table_12.sql         |     3 +
 .../queries/default/create_table_2.hiveql       |     1 +
 .../queries/default/create_table_2.sql          |     1 +
 .../queries/default/create_table_3.sql          |     1 +
 .../queries/default/create_table_4.sql          |     1 +
 .../queries/default/create_table_5.sql          |     1 +
 .../queries/default/create_table_6.sql          |     1 +
 .../queries/default/create_table_7.sql          |     1 +
 .../queries/default/create_table_8.sql          |    48 +
 .../queries/default/create_table_9.sql          |     1 +
 .../create_table_partition_by_column.sql        |     4 +
 .../create_table_partition_by_hash_1.sql        |     3 +
 .../create_table_partition_by_hash_2.sql        |     7 +
 .../default/create_table_partition_by_list.sql  |     8 +
 .../default/create_table_partition_by_range.sql |     9 +
 .../resources/queries/default/drop_table.sql    |     1 +
 .../queries/default/exists_predicate_1.sql      |     1 +
 .../queries/default/exists_predicate_2.sql      |     1 +
 .../resources/queries/default/groupby_1.sql     |     1 +
 .../resources/queries/default/groupby_2.sql     |     1 +
 .../resources/queries/default/groupby_3.sql     |     1 +
 .../resources/queries/default/groupby_4.sql     |     1 +
 .../resources/queries/default/groupby_5.sql     |     1 +
 .../resources/queries/default/in_subquery_1.sql |     1 +
 .../resources/queries/default/in_subquery_2.sql |     1 +
 .../queries/default/insert_into_select_1.sql    |     1 +
 .../queries/default/insert_into_select_2.sql    |     1 +
 .../queries/default/insert_into_select_3.sql    |     1 +
 .../default/insert_overwrite_into_select_1.sql  |     1 +
 .../insert_overwrite_into_select_2.hiveql       |     1 +
 .../default/insert_overwrite_into_select_2.sql  |     1 +
 .../default/insert_overwrite_into_select_3.sql  |     1 +
 .../test/resources/queries/default/join_1.sql   |     1 +
 .../test/resources/queries/default/join_10.sql  |     1 +
 .../test/resources/queries/default/join_11.sql  |     1 +
 .../test/resources/queries/default/join_12.sql  |     1 +
 .../test/resources/queries/default/join_13.sql  |    13 +
 .../test/resources/queries/default/join_14.sql  |     1 +
 .../resources/queries/default/join_15.hiveql    |     1 +
 .../test/resources/queries/default/join_15.sql  |     1 +
 .../test/resources/queries/default/join_2.sql   |     1 +
 .../test/resources/queries/default/join_3.sql   |     1 +
 .../test/resources/queries/default/join_4.sql   |     1 +
 .../test/resources/queries/default/join_5.sql   |     1 +
 .../test/resources/queries/default/join_6.sql   |     1 +
 .../test/resources/queries/default/join_7.sql   |     1 +
 .../test/resources/queries/default/join_8.sql   |    13 +
 .../test/resources/queries/default/join_9.sql   |     5 +
 .../test/resources/queries/default/select_1.sql |     1 +
 .../resources/queries/default/select_10.hiveql  |     5 +
 .../resources/queries/default/select_10.sql     |     5 +
 .../resources/queries/default/select_11.hiveql  |     4 +
 .../resources/queries/default/select_11.sql     |     6 +
 .../resources/queries/default/select_12.hiveql  |     3 +
 .../resources/queries/default/select_13.hiveql  |     2 +
 .../resources/queries/default/select_13.sql     |     2 +
 .../resources/queries/default/select_14.sql     |     2 +
 .../resources/queries/default/select_15.hiveql  |     1 +
 .../resources/queries/default/select_15.sql     |     1 +
 .../test/resources/queries/default/select_2.sql |     1 +
 .../test/resources/queries/default/select_3.sql |     1 +
 .../test/resources/queries/default/select_4.sql |     1 +
 .../test/resources/queries/default/select_5.sql |     1 +
 .../test/resources/queries/default/select_6.sql |     1 +
 .../test/resources/queries/default/select_7.sql |     1 +
 .../test/resources/queries/default/select_8.sql |     1 +
 .../resources/queries/default/select_9.hiveql   |     4 +
 .../test/resources/queries/default/select_9.sql |     4 +
 .../test/resources/queries/default/set_1.sql    |     1 +
 .../test/resources/queries/default/set_2.sql    |     1 +
 .../test/resources/queries/default/set_3.sql    |     1 +
 .../test/resources/queries/default/set_4.sql    |    36 +
 .../queries/default/table_subquery1.sql         |    19 +
 .../queries/default/table_subquery2.sql         |    21 +
 .../resources/queries/default/union_1.hiveql    |    14 +
 .../TestBuiltinFunctions/testAvgDouble.result   |     5 +
 .../TestBuiltinFunctions/testAvgInt.result      |     3 +
 .../TestBuiltinFunctions/testAvgLong.result     |     3 +
 .../TestBuiltinFunctions/testCount.result       |     3 +
 .../TestBuiltinFunctions/testMaxLong.result     |     3 +
 .../TestBuiltinFunctions/testMaxString.result   |     3 +
 .../TestBuiltinFunctions/testMinLong.result     |     3 +
 .../TestBuiltinFunctions/testMinString.result   |     3 +
 .../TestBuiltinFunctions/testSplitPart.result   |     7 +
 .../testSplitPartByString.result                |     7 +
 .../testSplitPartNested.result                  |     7 +
 .../TestCTASQuery/testCtasWithGroupby.result    |     7 +
 .../TestCTASQuery/testCtasWithLimit.result      |     5 +
 .../TestCTASQuery/testCtasWithOptions.result    |     5 +
 .../TestCTASQuery/testCtasWithOrderby.result    |     7 +
 .../TestCTASQuery/testCtasWithStoreType.result  |     5 +
 .../TestCTASQuery/testCtasWithUnion.result      |     8 +
 .../TestCaseByCases/testTAJO415Case.result      |     7 +
 .../TestCaseByCases/testTAJO418Case.result      |     3 +
 .../TestCaseByCases/testTAJO619Case.result      |     3 +
 .../TestCaseByCases/testTAJO718Case.result      |     5 +
 .../TestCaseByCases/testTAJO739Case.result      |     7 +
 .../quoted_identifier_mixed_chars_1.result      |     5 +
 .../quoted_identifier_mixed_chars_2.result      |     5 +
 .../quoted_identifier_mixed_chars_3.result      |     8 +
 .../quoted_identifier_non_ascii_1.result        |     5 +
 .../quoted_identifier_non_ascii_2.result        |     5 +
 .../quoted_identifier_non_ascii_3.result        |     5 +
 .../testComplexParameter.result                 |     3 +
 .../testComplexParameter2.result                |     3 +
 .../testComplexParameterWithSubQuery.result     |     3 +
 .../testDistinctAggregation1.result             |     5 +
 .../testDistinctAggregation2.result             |     5 +
 .../testDistinctAggregation3.result             |     3 +
 .../testDistinctAggregation4.result             |     4 +
 .../testDistinctAggregation5.result             |     4 +
 .../testDistinctAggregation6.result             |     5 +
 .../testDistinctAggregationWithHaving1.result   |     3 +
 .../testDistinctAggregationWithUnion1.result    |     4 +
 .../results/TestGroupByQuery/testGroupBy.result |     3 +
 .../TestGroupByQuery/testGroupBy2.result        |     4 +
 .../TestGroupByQuery/testGroupBy3.result        |     5 +
 .../TestGroupByQuery/testGroupBy4.result        |     5 +
 .../TestGroupByQuery/testGroupBy5.result        |     5 +
 .../TestGroupByQuery/testGroupByNested1.result  |     6 +
 .../TestGroupByQuery/testGroupByNested2.result  |     6 +
 .../testGroupByWithExpressionKeys1.result       |     5 +
 .../testGroupByWithExpressionKeys2.result       |     5 +
 .../testGroupByWithSameExprs1.result            |     6 +
 .../testGroupByWithSameExprs2.result            |     6 +
 .../testHavingWithAggFunction.result            |     4 +
 .../testHavingWithNamedTarget.result            |     5 +
 .../testBroadcastBasicJoin.result               |    99 +
 .../testBroadcastSubquery.result                |    15 +
 .../testBroadcastSubquery2.result               |     3 +
 .../testBroadcastTwoPartJoin.result             |    15 +
 .../TestJoinBroadcast/testCrossJoin.result      |   127 +
 .../testCrossJoinAndCaseWhen.result             |    27 +
 .../testCrossJoinWithAsterisk1.result           |    27 +
 .../testCrossJoinWithAsterisk2.result           |    27 +
 .../testCrossJoinWithAsterisk3.result           |    27 +
 .../testCrossJoinWithAsterisk4.result           |    27 +
 .../testCrossJoinWithEmptyTable1.result         |     2 +
 .../TestJoinBroadcast/testFullOuterJoin1.result |     7 +
 .../testFullOuterJoinWithEmptyTable1.result     |     7 +
 .../testInnerJoinWithEmptyTable.result          |     2 +
 .../testJoinCoReferredEvals1.result             |    27 +
 ...testJoinCoReferredEvalsWithSameExprs1.result |    22 +
 ...testJoinCoReferredEvalsWithSameExprs2.result |    22 +
 .../testJoinOnMultipleDatabases.result          |     5 +
 .../TestJoinBroadcast/testLeftOuterJoin1.result |     7 +
 .../testLeftOuterJoinWithConstantExpr1.result   |     7 +
 .../testLeftOuterJoinWithConstantExpr2.result   |     7 +
 .../testLeftOuterJoinWithConstantExpr3.result   |     7 +
 .../testLeftOuterJoinWithEmptyTable1.result     |     7 +
 .../testLeftOuterJoinWithEmptyTable2.result     |     7 +
 .../testLeftOuterJoinWithEmptyTable3.result     |     7 +
 .../testLeftOuterJoinWithEmptyTable4.result     |     4 +
 .../testOuterJoinAndCaseWhen1.result            |     7 +
 .../testRightOuterJoin1.result                  |     7 +
 .../testRightOuterJoinWithEmptyTable1.result    |     7 +
 .../TestJoinBroadcast/testTPCHQ2Join.result     |     5 +
 .../testWhereClauseJoin1.result                 |    27 +
 .../testWhereClauseJoin2.result                 |    27 +
 .../testWhereClauseJoin3.result                 |    27 +
 .../testWhereClauseJoin4.result                 |    27 +
 .../testWhereClauseJoin5.result                 |     5 +
 .../selfJoinOfPartitionedTable.result           |     7 +
 .../testNoProjectionJoinQual.result             |     3 +
 .../testPartialFilterPushDown.result            |     3 +
 .../testPartitionTableJoinSmallTable.result     |     7 +
 .../results/TestJoinQuery/testCrossJoin.result  |   127 +
 .../testCrossJoinAndCaseWhen.result             |    27 +
 .../testCrossJoinWithAsterisk1.result           |    27 +
 .../testCrossJoinWithAsterisk2.result           |    27 +
 .../testCrossJoinWithAsterisk3.result           |    27 +
 .../testCrossJoinWithAsterisk4.result           |    27 +
 .../testCrossJoinWithEmptyTable1.result         |     2 +
 .../TestJoinQuery/testFullOuterJoin1.result     |     7 +
 .../testFullOuterJoinWithEmptyTable1.result     |     7 +
 .../testInnerJoinWithEmptyTable.result          |     2 +
 .../testJoinCoReferredEvals1.result             |    27 +
 ...testJoinCoReferredEvalsWithSameExprs1.result |    22 +
 ...testJoinCoReferredEvalsWithSameExprs2.result |    22 +
 .../testJoinOnMultipleDatabases.result          |     5 +
 .../TestJoinQuery/testLeftOuterJoin1.result     |     7 +
 .../testLeftOuterJoinWithConstantExpr1.result   |     7 +
 .../testLeftOuterJoinWithConstantExpr2.result   |     7 +
 .../testLeftOuterJoinWithConstantExpr3.result   |     7 +
 .../testLeftOuterJoinWithEmptyTable1.result     |     7 +
 .../testLeftOuterJoinWithEmptyTable2.result     |     7 +
 .../testLeftOuterJoinWithEmptyTable3.result     |     7 +
 .../testLeftOuterJoinWithEmptyTable4.result     |     4 +
 .../testOuterJoinAndCaseWhen1.result            |     7 +
 .../TestJoinQuery/testRightOuterJoin1.result    |     7 +
 .../testRightOuterJoinWithEmptyTable1.result    |     7 +
 .../results/TestJoinQuery/testTPCHQ2Join.result |     5 +
 .../TestJoinQuery/testWhereClauseJoin1.result   |    27 +
 .../TestJoinQuery/testWhereClauseJoin2.result   |    27 +
 .../TestJoinQuery/testWhereClauseJoin3.result   |    27 +
 .../TestJoinQuery/testWhereClauseJoin4.result   |    27 +
 .../TestJoinQuery/testWhereClauseJoin5.result   |     5 +
 .../results/TestNetTypes/testGroupby.result     |     7 +
 .../results/TestNetTypes/testGroupby2.result    |     5 +
 .../results/TestNetTypes/testJoin.result        |     6 +
 .../results/TestNetTypes/testSelect.result      |     7 +
 .../results/TestNetTypes/testSort.result        |     7 +
 .../results/TestNetTypes/testSort2.result       |     6 +
 .../results/TestSelectQuery/testCaseWhen.result |     7 +
 .../testCaseWhenWithoutElse.result              |     7 +
 .../TestSelectQuery/testDatabaseRef.result      |     7 +
 .../TestSelectQuery/testExplainSelect.result    |     6 +
 .../results/TestSelectQuery/testInClause.result |     5 +
 .../TestSelectQuery/testInStrClause.result      |     4 +
 .../TestSelectQuery/testLikeClause.result       |     9 +
 .../results/TestSelectQuery/testLimit.result    |     5 +
 .../TestSelectQuery/testNonFromSelect1.result   |     3 +
 .../results/TestSelectQuery/testNotEqual.result |     5 +
 .../TestSelectQuery/testNotInClause.result      |     4 +
 .../TestSelectQuery/testNotInStrClause.result   |     4 +
 .../TestSelectQuery/testRealValueCompare.result |     3 +
 .../results/TestSelectQuery/testSelect.result   |     7 +
 .../results/TestSelectQuery/testSelect2.result  |     7 +
 .../results/TestSelectQuery/testSelect3.result  |     7 +
 .../TestSelectQuery/testSelectAsterisk1.result  |     7 +
 .../TestSelectQuery/testSelectAsterisk2.result  |     3 +
 .../TestSelectQuery/testSelectAsterisk3.result  |     3 +
 .../TestSelectQuery/testSelectAsterisk4.result  |     7 +
 .../TestSelectQuery/testSelectAsterisk5.result  |     3 +
 .../testSelectColumnAlias1.result               |     7 +
 .../TestSelectQuery/testSelectDistinct.result   |     7 +
 ...lectSameConstantsWithDifferentAliases.result |     7 +
 ...stSelectSameExprsWithDifferentAliases.result |     7 +
 .../TestSelectQuery/testSimpleQuery.result      |     7 +
 .../testSimpleQueryWithLimit.result             |     5 +
 .../TestSelectQuery/testStringCompare.result    |     5 +
 .../results/TestSelectQuery/testUnion1.result   |    10 +
 .../results/TestSelectQuery/testUnion2.result   |    12 +
 .../TestSelectQuery/testWhereCond1.result       |     3 +
 .../testWhereCondWithAlias1.result              |     4 +
 .../testWhereCondWithAlias2.result              |     4 +
 .../results/TestSortQuery/testAsterisk.result   |     7 +
 .../results/TestSortQuery/testSort.result       |     7 +
 .../TestSortQuery/testSortAfterGroupby.result   |     5 +
 .../testSortAfterGroupbyWithAlias.result        |     5 +
 .../results/TestSortQuery/testSortDesc.result   |     7 +
 .../TestSortQuery/testSortWithAlias1.result     |     7 +
 .../TestSortQuery/testSortWithAlias2.result     |     5 +
 .../TestSortQuery/testSortWithAlias3.result     |     7 +
 .../testSortWithAliasButOriginalName.result     |     7 +
 .../testSortWithAscDescKeys.result              |    26 +
 .../TestSortQuery/testSortWithDate.result       |     7 +
 .../TestSortQuery/testSortWithExpr1.result      |     7 +
 .../TestSortQuery/testSortWithExpr2.result      |     7 +
 .../results/TestSortQuery/testTopK.result       |     5 +
 .../results/TestTPCH/testQ1OrderBy.result       |     4 +
 .../results/TestTPCH/testQ2FourJoins.result     |     3 +
 .../results/TestTPCH/testTPCH14Expr.result      |     3 +
 .../results/TestTablePartitions/case1.result    |     4 +
 .../results/TestTablePartitions/case2.result    |     6 +
 .../results/TestTablePartitions/case3.result    |     5 +
 .../testGroupBySubQuery.result                  |     3 +
 .../TestTableSubQuery/testJoinSubQuery.result   |     4 +
 .../TestTableSubQuery/testJoinSubQuery2.result  |     4 +
 .../TestTableSubQuery/testTableSubquery1.result |     7 +
 .../TestTajoDatabaseMetaData/getTables1.result  |     4 +
 .../TestTajoDatabaseMetaData/getTables2.result  |     4 +
 .../results/TestTajoDump/testDump1.result       |    17 +
 .../results/TestUnionQuery/testUnion1.result    |    10 +
 .../results/TestUnionQuery/testUnion10.result   |    22 +
 .../results/TestUnionQuery/testUnion2.result    |    12 +
 .../results/TestUnionQuery/testUnion3.result    |     4 +
 .../results/TestUnionQuery/testUnion4.result    |     3 +
 .../results/TestUnionQuery/testUnion5.result    |     3 +
 .../results/TestUnionQuery/testUnion6.result    |     3 +
 .../results/TestUnionQuery/testUnion7.result    |    12 +
 .../results/TestUnionQuery/testUnion8.result    |     3 +
 .../results/TestUnionQuery/testUnion9.result    |     7 +
 .../testUnionWithSameAliasNames.result          |    12 +
 .../testBuildDDLForBaseTable.result             |     5 +
 .../testBuildDDLForExternalTable.result         |     5 +
 .../testBuildDDLQuotedTableName1.result         |     5 +
 .../testBuildDDLQuotedTableName2.result         |     5 +
 tajo-core/src/test/resources/yarn-site.xml      |    25 +
 tajo-core/src/test/tpch/customer.tbl            |     5 +
 tajo-core/src/test/tpch/empty_orders.tbl        |     0
 tajo-core/src/test/tpch/lineitem.tbl            |     5 +
 tajo-core/src/test/tpch/nation.tbl              |    25 +
 tajo-core/src/test/tpch/orders.tbl              |     3 +
 tajo-core/src/test/tpch/part.tbl                |     3 +
 tajo-core/src/test/tpch/partsupp.tbl            |     3 +
 tajo-core/src/test/tpch/region.tbl              |     5 +
 tajo-core/src/test/tpch/supplier.tbl            |     3 +
 .../benchmark/simple/groupby1.sql               |     6 -
 .../benchmark/simple/groupby2.sql               |     6 -
 .../benchmark/simple/selection1.sql             |     5 -
 .../benchmark/simple/selection2.sql             |     5 -
 .../benchmark/simple/selection3.sql             |     7 -
 .../benchmark/tpch/customer.schema              |     9 -
 .../benchmark/tpch/lineitem.schema              |    19 -
 .../benchmark/tpch/nation.schema                |     6 -
 .../benchmark/tpch/orders.schema                |    11 -
 .../benchmark/tpch/part.schema                  |    11 -
 .../benchmark/tpch/partsupp.schema              |     8 -
 .../tajo-core-backend/benchmark/tpch/q1.sql     |    21 -
 .../tajo-core-backend/benchmark/tpch/q10.sql    |    31 -
 .../tajo-core-backend/benchmark/tpch/q11.sql    |    27 -
 .../tajo-core-backend/benchmark/tpch/q12.sql    |    24 -
 .../tajo-core-backend/benchmark/tpch/q13.sql    |    18 -
 .../tajo-core-backend/benchmark/tpch/q14.sql    |    13 -
 .../tajo-core-backend/benchmark/tpch/q15.sql    |    32 -
 .../tajo-core-backend/benchmark/tpch/q16.sql    |    30 -
 .../tajo-core-backend/benchmark/tpch/q17.sql    |    17 -
 .../tajo-core-backend/benchmark/tpch/q18.sql    |    33 -
 .../tajo-core-backend/benchmark/tpch/q19.sql    |    35 -
 .../tajo-core-backend/benchmark/tpch/q2.sql     |    38 -
 .../tajo-core-backend/benchmark/tpch/q20.sql    |    36 -
 .../tajo-core-backend/benchmark/tpch/q21.sql    |    39 -
 .../tajo-core-backend/benchmark/tpch/q22.sql    |    36 -
 .../tajo-core-backend/benchmark/tpch/q3.sql     |    22 -
 .../tajo-core-backend/benchmark/tpch/q4.sql     |    21 -
 .../tajo-core-backend/benchmark/tpch/q5.sql     |    24 -
 .../tajo-core-backend/benchmark/tpch/q6.sql     |    10 -
 .../tajo-core-backend/benchmark/tpch/q7.sql     |    38 -
 .../tajo-core-backend/benchmark/tpch/q8.sql     |    37 -
 .../tajo-core-backend/benchmark/tpch/q9.sql     |    31 -
 .../benchmark/tpch/region.schema                |     5 -
 .../benchmark/tpch/supplier.schema              |     9 -
 tajo-core/tajo-core-backend/pom.xml             |   683 --
 .../apache/tajo/engine/parser/HiveQLLexer.g4    |   390 -
 .../apache/tajo/engine/parser/HiveQLParser.g4   |  2067 ----
 .../org/apache/tajo/engine/parser/SQLLexer.g4   |   479 -
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  1398 ---
 .../src/main/java/log4j.properties              |    30 -
 .../org/apache/tajo/benchmark/BenchmarkSet.java |   120 -
 .../java/org/apache/tajo/benchmark/Driver.java  |    46 -
 .../org/apache/tajo/benchmark/SimpleQuery.java  |    29 -
 .../java/org/apache/tajo/benchmark/TPCH.java    |   200 -
 .../eval/AggregationFunctionCallEval.java       |    92 -
 .../tajo/engine/eval/AlgebraicException.java    |    41 -
 .../apache/tajo/engine/eval/AlgebraicUtil.java  |   398 -
 .../tajo/engine/eval/BasicEvalNodeVisitor.java  |   341 -
 .../tajo/engine/eval/BetweenPredicateEval.java  |   219 -
 .../org/apache/tajo/engine/eval/BinaryEval.java |   205 -
 .../apache/tajo/engine/eval/CaseWhenEval.java   |   208 -
 .../org/apache/tajo/engine/eval/CastEval.java   |    87 -
 .../org/apache/tajo/engine/eval/ConstEval.java  |    99 -
 .../org/apache/tajo/engine/eval/EvalNode.java   |   118 -
 .../tajo/engine/eval/EvalNodeVisitor.java       |    24 -
 .../tajo/engine/eval/EvalNodeVisitor2.java      |    71 -
 .../tajo/engine/eval/EvalTreeFactory.java       |    32 -
 .../apache/tajo/engine/eval/EvalTreeUtil.java   |   350 -
 .../org/apache/tajo/engine/eval/EvalType.java   |    83 -
 .../org/apache/tajo/engine/eval/FieldEval.java  |   114 -
 .../apache/tajo/engine/eval/FunctionEval.java   |   144 -
 .../tajo/engine/eval/GeneralFunctionEval.java   |    81 -
 .../org/apache/tajo/engine/eval/InEval.java     |    86 -
 .../tajo/engine/eval/InvalidCastException.java  |    36 -
 .../tajo/engine/eval/InvalidEvalException.java  |    36 -
 .../org/apache/tajo/engine/eval/IsNullEval.java |    84 -
 .../tajo/engine/eval/LikePredicateEval.java     |    50 -
 .../org/apache/tajo/engine/eval/NotEval.java    |    92 -
 .../tajo/engine/eval/PartialBinaryExpr.java     |    70 -
 .../engine/eval/PatternMatchPredicateEval.java  |    82 -
 .../tajo/engine/eval/RegexPredicateEval.java    |    53 -
 .../tajo/engine/eval/RowConstantEval.java       |    79 -
 .../org/apache/tajo/engine/eval/SignedEval.java |    99 -
 .../engine/eval/SimilarToPredicateEval.java     |    43 -
 .../exception/AmbiguousFieldException.java      |    30 -
 .../exception/IllegalQueryStatusException.java  |    38 -
 .../engine/exception/InvalidQueryException.java |    35 -
 .../engine/exception/NoSuchColumnException.java |    25 -
 .../exception/RangeOverflowException.java       |    28 -
 .../tajo/engine/exception/VerifyException.java  |    27 -
 .../tajo/engine/function/AggFunction.java       |    58 -
 .../tajo/engine/function/FunctionContext.java   |    22 -
 .../tajo/engine/function/GeneralFunction.java   |    56 -
 .../engine/function/annotation/Description.java |    50 -
 .../function/annotation/ParamOptionTypes.java   |    29 -
 .../engine/function/annotation/ParamTypes.java  |    30 -
 .../tajo/engine/function/builtin/AvgDouble.java |    97 -
 .../tajo/engine/function/builtin/AvgFloat.java  |    46 -
 .../tajo/engine/function/builtin/AvgInt.java    |    47 -
 .../tajo/engine/function/builtin/AvgLong.java   |    96 -
 .../tajo/engine/function/builtin/Coalesce.java  |    44 -
 .../engine/function/builtin/CoalesceDouble.java |    43 -
 .../engine/function/builtin/CoalesceLong.java   |    44 -
 .../engine/function/builtin/CoalesceString.java |    44 -
 .../tajo/engine/function/builtin/CountRows.java |    84 -
 .../engine/function/builtin/CountValue.java     |    55 -
 .../function/builtin/CountValueDistinct.java    |    90 -
 .../tajo/engine/function/builtin/Date.java      |    62 -
 .../tajo/engine/function/builtin/MaxDouble.java |    78 -
 .../tajo/engine/function/builtin/MaxFloat.java  |    76 -
 .../tajo/engine/function/builtin/MaxInt.java    |    77 -
 .../tajo/engine/function/builtin/MaxLong.java   |    77 -
 .../tajo/engine/function/builtin/MaxString.java |    81 -
 .../tajo/engine/function/builtin/MinDouble.java |    77 -
 .../tajo/engine/function/builtin/MinFloat.java  |    78 -
 .../tajo/engine/function/builtin/MinInt.java    |    77 -
 .../tajo/engine/function/builtin/MinLong.java   |    78 -
 .../tajo/engine/function/builtin/MinString.java |    82 -
 .../tajo/engine/function/builtin/RandomInt.java |    57 -
 .../tajo/engine/function/builtin/Sleep.java     |    52 -
 .../tajo/engine/function/builtin/SumDouble.java |    82 -
 .../function/builtin/SumDoubleDistinct.java     |    98 -
 .../tajo/engine/function/builtin/SumFloat.java  |    80 -
 .../function/builtin/SumFloatDistinct.java      |    98 -
 .../tajo/engine/function/builtin/SumInt.java    |    82 -
 .../engine/function/builtin/SumIntDistinct.java |    98 -
 .../tajo/engine/function/builtin/SumLong.java   |    82 -
 .../function/builtin/SumLongDistinct.java       |    98 -
 .../tajo/engine/function/builtin/Today.java     |    46 -
 .../function/datetime/DatePartFromDate.java     |   194 -
 .../function/datetime/DatePartFromTime.java     |   140 -
 .../datetime/DatePartFromTimestamp.java         |   259 -
 .../datetime/DateTimePartFromUnixTimeStamp.java |   141 -
 .../function/datetime/ToCharTimestamp.java      |    79 -
 .../engine/function/datetime/ToTimestamp.java   |    55 -
 .../function/geoip/GeoIPCountryInet4.java       |    55 -
 .../engine/function/geoip/GeoIPCountryText.java |    56 -
 .../function/geoip/GeoIPInCountryInet4.java     |    61 -
 .../function/geoip/GeoIPInCountryText.java      |    60 -
 .../tajo/engine/function/math/AbsDouble.java    |    61 -
 .../tajo/engine/function/math/AbsFloat.java     |    61 -
 .../tajo/engine/function/math/AbsInt.java       |    61 -
 .../tajo/engine/function/math/AbsLong.java      |    60 -
 .../apache/tajo/engine/function/math/Acos.java  |    61 -
 .../apache/tajo/engine/function/math/Asin.java  |    60 -
 .../apache/tajo/engine/function/math/Atan.java  |    60 -
 .../apache/tajo/engine/function/math/Atan2.java |    62 -
 .../apache/tajo/engine/function/math/Cbrt.java  |    61 -
 .../apache/tajo/engine/function/math/Ceil.java  |    63 -
 .../apache/tajo/engine/function/math/Cos.java   |    60 -
 .../tajo/engine/function/math/Degrees.java      |    61 -
 .../apache/tajo/engine/function/math/Div.java   |    77 -
 .../apache/tajo/engine/function/math/Exp.java   |    61 -
 .../apache/tajo/engine/function/math/Floor.java |    61 -
 .../apache/tajo/engine/function/math/Mod.java   |    77 -
 .../apache/tajo/engine/function/math/Pi.java    |    51 -
 .../apache/tajo/engine/function/math/Pow.java   |    78 -
 .../tajo/engine/function/math/Radians.java      |    62 -
 .../apache/tajo/engine/function/math/Round.java |    71 -
 .../apache/tajo/engine/function/math/Sign.java  |    64 -
 .../apache/tajo/engine/function/math/Sin.java   |    60 -
 .../apache/tajo/engine/function/math/Sqrt.java  |    62 -
 .../apache/tajo/engine/function/math/Tan.java   |    60 -
 .../tajo/engine/function/string/Ascii.java      |    67 -
 .../tajo/engine/function/string/BTrim.java      |    77 -
 .../tajo/engine/function/string/BitLength.java  |    58 -
 .../tajo/engine/function/string/CharLength.java |    59 -
 .../apache/tajo/engine/function/string/Chr.java |    65 -
 .../tajo/engine/function/string/Concat.java     |    73 -
 .../tajo/engine/function/string/Concat_ws.java  |    78 -
 .../tajo/engine/function/string/Decode.java     |    91 -
 .../tajo/engine/function/string/Digest.java     |    87 -
 .../tajo/engine/function/string/Encode.java     |    93 -
 .../tajo/engine/function/string/FindInSet.java  |   116 -
 .../function/string/HexStringConverter.java     |    65 -
 .../tajo/engine/function/string/InitCap.java    |    60 -
 .../tajo/engine/function/string/LTrim.java      |    77 -
 .../tajo/engine/function/string/Left.java       |    84 -
 .../tajo/engine/function/string/Length.java     |    61 -
 .../tajo/engine/function/string/Locate.java     |   106 -
 .../tajo/engine/function/string/Lower.java      |    58 -
 .../tajo/engine/function/string/Lpad.java       |    90 -
 .../apache/tajo/engine/function/string/Md5.java |    65 -
 .../engine/function/string/OctetLength.java     |    61 -
 .../tajo/engine/function/string/QuoteIdent.java |    67 -
 .../tajo/engine/function/string/RTrim.java      |    76 -
 .../engine/function/string/RegexpReplace.java   |   115 -
 .../tajo/engine/function/string/Repeat.java     |    72 -
 .../tajo/engine/function/string/Reverse.java    |    58 -
 .../tajo/engine/function/string/Right.java      |    85 -
 .../tajo/engine/function/string/Rpad.java       |    90 -
 .../tajo/engine/function/string/SplitPart.java  |    73 -
 .../tajo/engine/function/string/StrPos.java     |    73 -
 .../tajo/engine/function/string/StrPosb.java    |   120 -
 .../tajo/engine/function/string/Substr.java     |    94 -
 .../tajo/engine/function/string/ToBin.java      |    65 -
 .../tajo/engine/function/string/ToHex.java      |    76 -
 .../tajo/engine/function/string/Upper.java      |    58 -
 .../apache/tajo/engine/json/CoreGsonHelper.java |    90 -
 .../tajo/engine/json/EvalNodeAdapter.java       |    51 -
 .../tajo/engine/json/LogicalNodeAdapter.java    |    50 -
 .../tajo/engine/parser/HiveQLAnalyzer.java      |  1550 ---
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  1516 ---
 .../tajo/engine/parser/SQLErrorListener.java    |    38 -
 .../tajo/engine/parser/SQLErrorStrategy.java    |    66 -
 .../tajo/engine/parser/SQLParseError.java       |   107 -
 .../tajo/engine/parser/SQLSyntaxError.java      |    51 -
 .../tajo/engine/planner/AlgebraVisitor.java     |   110 -
 .../engine/planner/AlterTablespaceNode.java     |   103 -
 .../tajo/engine/planner/BaseAlgebraVisitor.java |   739 --
 .../engine/planner/BasicLogicalPlanVisitor.java |   319 -
 .../planner/BroadcastJoinPlanVisitor.java       |    86 -
 .../planner/ExplainLogicalPlanVisitor.java      |   236 -
 .../tajo/engine/planner/ExprAnnotator.java      |   631 -
 .../apache/tajo/engine/planner/ExprFinder.java  |    72 -
 .../tajo/engine/planner/ExprNormalizer.java     |   257 -
 .../tajo/engine/planner/ExprsVerifier.java      |   216 -
 .../tajo/engine/planner/GroupElement.java       |    64 -
 .../tajo/engine/planner/LogicalOptimizer.java   |   280 -
 .../apache/tajo/engine/planner/LogicalPlan.java |   782 --
 .../engine/planner/LogicalPlanPreprocessor.java |   442 -
 .../engine/planner/LogicalPlanVerifier.java     |   249 -
 .../tajo/engine/planner/LogicalPlanVisitor.java |    91 -
 .../tajo/engine/planner/LogicalPlanner.java     |  1579 ---
 .../tajo/engine/planner/NamedExprsManager.java  |   369 -
 .../tajo/engine/planner/PhysicalPlanner.java    |    36 -
 .../engine/planner/PhysicalPlannerImpl.java     |  1057 --
 .../planner/PhysicalPlanningException.java      |    31 -
 .../apache/tajo/engine/planner/PlanString.java  |   119 -
 .../apache/tajo/engine/planner/PlannerUtil.java |   762 --
 .../tajo/engine/planner/PlanningException.java  |    29 -
 .../engine/planner/PreLogicalPlanVerifier.java  |   274 -
 .../apache/tajo/engine/planner/Projector.java   |    51 -
 .../engine/planner/RangePartitionAlgorithm.java |   187 -
 .../engine/planner/SimpleAlgebraVisitor.java    |   215 -
 .../org/apache/tajo/engine/planner/Target.java  |   129 -
 .../engine/planner/UniformRangePartition.java   |   451 -
 .../tajo/engine/planner/VerificationState.java  |    44 -
 .../tajo/engine/planner/enforce/Enforcer.java   |   262 -
 .../tajo/engine/planner/global/DataChannel.java |   200 -
 .../engine/planner/global/ExecutionBlock.java   |   127 -
 .../planner/global/ExecutionBlockCursor.java    |    93 -
 .../engine/planner/global/GlobalPlanner.java    |  1187 --
 .../tajo/engine/planner/global/MasterPlan.java  |   268 -
 .../engine/planner/graph/DirectedGraph.java     |    64 -
 .../planner/graph/DirectedGraphCursor.java      |    65 -
 .../planner/graph/DirectedGraphVisitor.java     |    25 -
 .../apache/tajo/engine/planner/graph/Graph.java |    45 -
 .../planner/graph/SimpleDirectedGraph.java      |   270 -
 .../planner/graph/SimpleUndirectedGraph.java    |   102 -
 .../engine/planner/graph/UndirectedGraph.java   |    30 -
 .../engine/planner/logical/AlterTableNode.java  |   134 -
 .../tajo/engine/planner/logical/BinaryNode.java |    77 -
 .../planner/logical/CreateDatabaseNode.java     |    87 -
 .../engine/planner/logical/CreateTableNode.java |   145 -
 .../planner/logical/DropDatabaseNode.java       |    85 -
 .../engine/planner/logical/DropTableNode.java   |    95 -
 .../engine/planner/logical/EvalExprNode.java    |    83 -
 .../tajo/engine/planner/logical/ExceptNode.java |    45 -
 .../engine/planner/logical/GroupbyNode.java     |   198 -
 .../tajo/engine/planner/logical/HavingNode.java |    67 -
 .../engine/planner/logical/IndexScanNode.java   |   122 -
 .../tajo/engine/planner/logical/InsertNode.java |   182 -
 .../engine/planner/logical/IntersectNode.java   |    44 -
 .../tajo/engine/planner/logical/JoinNode.java   |   165 -
 .../tajo/engine/planner/logical/LimitNode.java  |    65 -
 .../engine/planner/logical/LogicalNode.java     |   128 -
 .../planner/logical/LogicalNodeVisitor.java     |    27 -
 .../engine/planner/logical/LogicalRootNode.java |    41 -
 .../tajo/engine/planner/logical/NodeType.java   |    66 -
 .../logical/PartitionedTableScanNode.java       |   155 -
 .../planner/logical/PersistentStoreNode.java    |    90 -
 .../engine/planner/logical/Projectable.java     |    73 -
 .../engine/planner/logical/ProjectionNode.java  |   114 -
 .../engine/planner/logical/RelationNode.java    |    49 -
 .../tajo/engine/planner/logical/ScanNode.java   |   234 -
 .../engine/planner/logical/SelectionNode.java   |    69 -
 .../planner/logical/ShuffleFileWriteNode.java   |   104 -
 .../tajo/engine/planner/logical/SortNode.java   |    94 -
 .../engine/planner/logical/StoreTableNode.java  |   100 -
 .../planner/logical/TableSubQueryNode.java      |   181 -
 .../tajo/engine/planner/logical/UnaryNode.java  |    69 -
 .../tajo/engine/planner/logical/UnionNode.java  |    37 -
 .../tajo/engine/planner/logical/join/Edge.java  |    50 -
 .../planner/logical/join/FoundJoinOrder.java    |    47 -
 .../join/GreedyHeuristicJoinOrderAlgorithm.java |   262 -
 .../engine/planner/logical/join/JoinEdge.java   |    76 -
 .../engine/planner/logical/join/JoinGraph.java  |   115 -
 .../logical/join/JoinOrderAlgorithm.java        |    46 -
 .../planner/physical/AggregationExec.java       |    69 -
 .../engine/planner/physical/BNLJoinExec.java    |   223 -
 .../planner/physical/BSTIndexScanExec.java      |   143 -
 .../physical/BasicPhysicalExecutorVisitor.java  |   265 -
 .../planner/physical/BinaryPhysicalExec.java    |   109 -
 .../planner/physical/ColPartitionStoreExec.java |   100 -
 .../engine/planner/physical/EvalExprExec.java   |    66 -
 .../planner/physical/ExternalSortExec.java      |   782 --
 .../planner/physical/HashAggregateExec.java     |   115 -
 .../HashBasedColPartitionStoreExec.java         |   132 -
 .../planner/physical/HashFullOuterJoinExec.java |   252 -
 .../engine/planner/physical/HashJoinExec.java   |   202 -
 .../planner/physical/HashLeftAntiJoinExec.java  |   109 -
 .../planner/physical/HashLeftOuterJoinExec.java |   212 -
 .../planner/physical/HashLeftSemiJoinExec.java  |   107 -
 .../planner/physical/HashPartitioner.java       |    49 -
 .../physical/HashShuffleFileWriteExec.java      |   160 -
 .../engine/planner/physical/HavingExec.java     |    50 -
 .../planner/physical/JoinTupleComparator.java   |    87 -
 .../tajo/engine/planner/physical/LimitExec.java |    55 -
 .../engine/planner/physical/MemSortExec.java    |    89 -
 .../physical/MergeFullOuterJoinExec.java        |   335 -
 .../engine/planner/physical/MergeJoinExec.java  |   193 -
 .../engine/planner/physical/NLJoinExec.java     |   104 -
 .../planner/physical/NLLeftOuterJoinExec.java   |   122 -
 .../physical/PartitionMergeScanExec.java        |   143 -
 .../engine/planner/physical/Partitioner.java    |    45 -
 .../engine/planner/physical/PhysicalExec.java   |    79 -
 .../physical/PhysicalExecutorVisitor.java       |   107 -
 .../planner/physical/PhysicalPlanUtil.java      |    39 -
 .../physical/PhysicalPlanningException.java     |    31 -
 .../engine/planner/physical/ProjectionExec.java |    69 -
 .../physical/RangeShuffleFileWriteExec.java     |   131 -
 .../physical/RightOuterMergeJoinExec.java       |   346 -
 .../engine/planner/physical/SelectionExec.java  |    49 -
 .../engine/planner/physical/SeqScanExec.java    |   316 -
 .../planner/physical/SortAggregateExec.java     |   125 -
 .../SortBasedColPartitionStoreExec.java         |   149 -
 .../tajo/engine/planner/physical/SortExec.java  |    51 -
 .../engine/planner/physical/StoreTableExec.java |   102 -
 .../engine/planner/physical/TunnelExec.java     |    42 -
 .../planner/physical/UnaryPhysicalExec.java     |    96 -
 .../tajo/engine/planner/physical/UnionExec.java |    62 -
 .../rewrite/BasicQueryRewriteEngine.java        |    70 -
 .../planner/rewrite/FilterPushDownRule.java     |   323 -
 .../rewrite/PartitionedTableRewriter.java       |   373 -
 .../planner/rewrite/ProjectionPushDownRule.java |   966 --
 .../planner/rewrite/QueryRewriteEngine.java     |    32 -
 .../engine/planner/rewrite/RewriteRule.java     |    56 -
 .../apache/tajo/engine/query/QueryContext.java  |   191 -
 .../tajo/engine/query/QueryUnitRequest.java     |    50 -
 .../tajo/engine/query/QueryUnitRequestImpl.java |   328 -
 .../apache/tajo/engine/utils/SchemaUtil.java    |    72 -
 .../apache/tajo/engine/utils/ThreadUtil.java    |   149 -
 .../apache/tajo/engine/utils/TupleCache.java    |   122 -
 .../apache/tajo/engine/utils/TupleCacheKey.java |    60 -
 .../tajo/engine/utils/TupleCacheScanner.java    |   109 -
 .../org/apache/tajo/engine/utils/TupleUtil.java |   231 -
 .../tajo/master/AbstractTaskScheduler.java      |    56 -
 .../org/apache/tajo/master/ContainerProxy.java  |    81 -
 .../DefaultFragmentScheduleAlgorithm.java       |   247 -
 .../tajo/master/DefaultTaskScheduler.java       |   896 --
 .../apache/tajo/master/FetchScheduleEvent.java  |    40 -
 .../org/apache/tajo/master/FragmentPair.java    |    73 -
 .../tajo/master/FragmentScheduleAlgorithm.java  |    38 -
 .../FragmentScheduleAlgorithmFactory.java       |    68 -
 .../org/apache/tajo/master/GlobalEngine.java    |   738 --
 .../master/GreedyFragmentScheduleAlgorithm.java |   421 -
 .../apache/tajo/master/LazyTaskScheduler.java   |   522 -
 .../apache/tajo/master/ScheduledFetches.java    |    47 -
 .../apache/tajo/master/TajoAsyncDispatcher.java |   232 -
 .../apache/tajo/master/TajoContainerProxy.java  |   178 -
 .../java/org/apache/tajo/master/TajoMaster.java |   579 -
 .../tajo/master/TajoMasterClientService.java    |   754 --
 .../apache/tajo/master/TajoMasterService.java   |   172 -
 .../tajo/master/TaskRunnerGroupEvent.java       |    51 -
 .../apache/tajo/master/TaskRunnerLauncher.java  |    25 -
 .../tajo/master/TaskSchedulerContext.java       |    68 -
 .../tajo/master/TaskSchedulerFactory.java       |    69 -
 .../java/org/apache/tajo/master/TaskState.java  |    23 -
 .../apache/tajo/master/YarnContainerProxy.java  |   414 -
 .../tajo/master/YarnTaskRunnerLauncherImpl.java |   200 -
 .../apache/tajo/master/cluster/ServerName.java  |   123 -
 .../master/event/ContainerAllocationEvent.java  |    77 -
 .../event/ContainerAllocatorEventType.java      |    26 -
 .../tajo/master/event/ContainerEvent.java       |    37 -
 .../master/event/FragmentScheduleEvent.java     |    61 -
 .../event/GrouppedContainerAllocatorEvent.java  |    45 -
 .../tajo/master/event/LocalTaskEvent.java       |    45 -
 .../tajo/master/event/LocalTaskEventType.java   |    23 -
 .../tajo/master/event/QueryCompletedEvent.java  |    42 -
 .../event/QueryDiagnosticsUpdateEvent.java      |    34 -
 .../apache/tajo/master/event/QueryEvent.java    |    36 -
 .../tajo/master/event/QueryEventType.java       |    36 -
 .../event/QueryMasterQueryCompletedEvent.java   |    39 -
 .../tajo/master/event/QueryStartEvent.java      |    74 -
 .../tajo/master/event/QuerySubQueryEvent.java   |    35 -
 .../event/QueryUnitAttemptScheduleEvent.java    |    87 -
 .../master/event/SubQueryCompletedEvent.java    |    42 -
 .../event/SubQueryContainerAllocationEvent.java |    38 -
 .../event/SubQueryDiagnosticsUpdateEvent.java   |    34 -
 .../apache/tajo/master/event/SubQueryEvent.java |    35 -
 .../tajo/master/event/SubQueryEventType.java    |    43 -
 .../tajo/master/event/SubQueryTaskEvent.java    |    43 -
 .../master/event/TaskAttemptAssignedEvent.java  |    48 -
 .../tajo/master/event/TaskAttemptEvent.java     |    36 -
 .../tajo/master/event/TaskAttemptEventType.java |    54 -
 .../master/event/TaskAttemptScheduleEvent.java  |    37 -
 .../event/TaskAttemptStatusUpdateEvent.java     |    36 -
 .../tajo/master/event/TaskCompletionEvent.java  |    35 -
 .../org/apache/tajo/master/event/TaskEvent.java |    35 -
 .../apache/tajo/master/event/TaskEventType.java |    38 -
 .../tajo/master/event/TaskFatalErrorEvent.java  |    41 -
 .../tajo/master/event/TaskRequestEvent.java     |    59 -
 .../tajo/master/event/TaskSchedulerEvent.java   |    41 -
 .../tajo/master/event/TaskTAttemptEvent.java    |    34 -
 .../master/metrics/CatalogMetricsGaugeSet.java  |    56 -
 .../metrics/WorkerResourceMetricsGaugeSet.java  |    74 -
 .../apache/tajo/master/querymaster/Query.java   |   702 --
 .../master/querymaster/QueryInProgress.java     |   294 -
 .../tajo/master/querymaster/QueryInfo.java      |   125 -
 .../tajo/master/querymaster/QueryJobEvent.java  |    44 -
 .../master/querymaster/QueryJobManager.java     |   195 -
 .../tajo/master/querymaster/QueryMaster.java    |   504 -
 .../querymaster/QueryMasterManagerService.java  |   239 -
 .../master/querymaster/QueryMasterRunner.java   |   149 -
 .../master/querymaster/QueryMasterTask.java     |   556 -
 .../tajo/master/querymaster/QueryUnit.java      |   658 -
 .../master/querymaster/QueryUnitAttempt.java    |   442 -
 .../tajo/master/querymaster/Repartitioner.java  |   653 -
 .../tajo/master/querymaster/SubQuery.java       |  1125 --
 .../tajo/master/querymaster/SubQueryState.java  |    30 -
 .../apache/tajo/master/rm/TajoRMContext.java    |    84 -
 .../tajo/master/rm/TajoResourceTracker.java     |   253 -
 .../tajo/master/rm/TajoWorkerContainer.java     |   100 -
 .../tajo/master/rm/TajoWorkerContainerId.java   |    93 -
 .../master/rm/TajoWorkerResourceManager.java    |   537 -
 .../java/org/apache/tajo/master/rm/Worker.java  |   296 -
 .../org/apache/tajo/master/rm/WorkerEvent.java  |    37 -
 .../apache/tajo/master/rm/WorkerEventType.java  |    30 -
 .../tajo/master/rm/WorkerLivelinessMonitor.java |    56 -
 .../tajo/master/rm/WorkerReconnectEvent.java    |    35 -
 .../apache/tajo/master/rm/WorkerResource.java   |   235 -
 .../tajo/master/rm/WorkerResourceManager.java   |   110 -
 .../org/apache/tajo/master/rm/WorkerState.java  |    44 -
 .../tajo/master/rm/WorkerStatusEvent.java       |    54 -
 .../master/rm/YarnRMContainerAllocator.java     |   237 -
 .../tajo/master/rm/YarnTajoResourceManager.java |   349 -
 .../master/session/InvalidSessionException.java |    25 -
 .../session/NoSuchSessionVariableException.java |    25 -
 .../org/apache/tajo/master/session/Session.java |   124 -
 .../tajo/master/session/SessionConstants.java   |    23 -
 .../tajo/master/session/SessionEvent.java       |    34 -
 .../tajo/master/session/SessionEventType.java   |    24 -
 .../session/SessionLivelinessMonitor.java       |    53 -
 .../tajo/master/session/SessionManager.java     |   139 -
 .../org/apache/tajo/net/CachedDNSResolver.java  |    64 -
 .../apache/tajo/util/ApplicationIdUtils.java    |    44 -
 .../java/org/apache/tajo/util/ClassUtil.java    |   155 -
 .../java/org/apache/tajo/util/GeoIPUtil.java    |    46 -
 .../java/org/apache/tajo/util/IndexUtil.java    |   148 -
 .../main/java/org/apache/tajo/util/JSPUtil.java |   209 -
 .../util/metrics/GroupNameMetricsFilter.java    |    43 -
 .../tajo/util/metrics/LogEventGaugeSet.java     |    64 -
 .../tajo/util/metrics/MetricsFilterList.java    |    43 -
 .../tajo/util/metrics/RegexpMetricsFilter.java  |    51 -
 .../tajo/util/metrics/TajoLogEventCounter.java  |    86 -
 .../apache/tajo/util/metrics/TajoMetrics.java   |   133 -
 .../tajo/util/metrics/TajoSystemMetrics.java    |   213 -
 .../util/metrics/reporter/GangliaReporter.java  |   258 -
 .../reporter/MetricsConsoleReporter.java        |    80 -
 .../MetricsConsoleScheduledReporter.java        |    32 -
 .../reporter/MetricsFileScheduledReporter.java  |    58 -
 .../MetricsStreamScheduledReporter.java         |   179 -
 .../util/metrics/reporter/NullReporter.java     |    31 -
 .../metrics/reporter/TajoMetricsReporter.java   |   232 -
 .../reporter/TajoMetricsScheduledReporter.java  |   206 -
 .../java/org/apache/tajo/webapp/HttpServer.java |   447 -
 .../tajo/webapp/QueryExecutorServlet.java       |   376 -
 .../apache/tajo/webapp/StaticHttpServer.java    |    69 -
 .../tajo/worker/AbstractResourceAllocator.java  |    54 -
 .../org/apache/tajo/worker/DeletionService.java |   115 -
 .../java/org/apache/tajo/worker/Fetcher.java    |   256 -
 .../apache/tajo/worker/InterDataRetriever.java  |   113 -
 .../tajo/worker/PartitionRetrieverHandler.java  |    44 -
 .../tajo/worker/RangeRetrieverHandler.java      |   163 -
 .../apache/tajo/worker/ResourceAllocator.java   |    29 -
 .../org/apache/tajo/worker/TajoQueryEngine.java |    49 -
 .../tajo/worker/TajoResourceAllocator.java      |   339 -
 .../java/org/apache/tajo/worker/TajoWorker.java |   583 -
 .../tajo/worker/TajoWorkerClientService.java    |   221 -
 .../tajo/worker/TajoWorkerManagerService.java   |   152 -
 .../main/java/org/apache/tajo/worker/Task.java  |   762 --
 .../apache/tajo/worker/TaskAttemptContext.java  |   270 -
 .../org/apache/tajo/worker/TaskHistory.java     |   198 -
 .../java/org/apache/tajo/worker/TaskRunner.java |   431 -
 .../apache/tajo/worker/TaskRunnerManager.java   |   238 -
 .../tajo/worker/WorkerHeartbeatService.java     |   289 -
 .../tajo/worker/YarnResourceAllocator.java      |   117 -
 .../FileAccessForbiddenException.java           |    40 -
 .../tajo/worker/dataserver/HttpDataServer.java  |    87 -
 .../dataserver/HttpDataServerHandler.java       |   199 -
 .../HttpDataServerPipelineFactory.java          |    55 -
 .../apache/tajo/worker/dataserver/HttpUtil.java |    69 -
 .../retriever/AdvancedDataRetriever.java        |   128 -
 .../dataserver/retriever/DataRetriever.java     |    29 -
 .../retriever/DirectoryRetriever.java           |    56 -
 .../worker/dataserver/retriever/FileChunk.java  |    51 -
 .../dataserver/retriever/RetrieverHandler.java  |    33 -
 .../src/main/proto/InternalTypes.proto          |    32 -
 .../src/main/proto/QueryMasterProtocol.proto    |    42 -
 .../main/proto/ResourceTrackerProtocol.proto    |    40 -
 .../src/main/proto/TajoMasterProtocol.proto     |   154 -
 .../src/main/proto/TajoWorkerProtocol.proto     |   258 -
 .../src/main/resources/catalog-default.xml      |    32 -
 .../src/main/resources/log4j.properties         |    28 -
 .../src/main/resources/tajo-default.xml         |    50 -
 .../src/main/resources/tajo-metrics.properties  |    75 -
 .../webapps/admin/WEB-INF/jetty-web.xml         |    23 -
 .../resources/webapps/admin/WEB-INF/web.xml     |    27 -
 .../resources/webapps/admin/catalogview.jsp     |   166 -
 .../main/resources/webapps/admin/cluster.jsp    |   216 -
 .../src/main/resources/webapps/admin/conf.jsp   |    56 -
 .../src/main/resources/webapps/admin/env.jsp    |    67 -
 .../main/resources/webapps/admin/functions.jsp  |    77 -
 .../src/main/resources/webapps/admin/getCSV.jsp |    31 -
 .../src/main/resources/webapps/admin/header.jsp |    32 -
 .../src/main/resources/webapps/admin/index.jsp  |   184 -
 .../src/main/resources/webapps/admin/query.jsp  |   137 -
 .../resources/webapps/admin/query_executor.jsp  |   315 -
 .../src/main/resources/webapps/admin/thread.jsp |    45 -
 .../resources/webapps/static/img/logo_tajo.gif  |   Bin 3025 -> 0 bytes
 .../main/resources/webapps/static/img/on.jpg    |   Bin 636 -> 0 bytes
 .../resources/webapps/static/img/tajo_logo.png  |   Bin 7592 -> 0 bytes
 .../resources/webapps/static/img/tajochar.jpg   |   Bin 325871 -> 0 bytes
 .../static/img/tajochar_catalog_small.jpg       |   Bin 13275 -> 0 bytes
 .../webapps/static/img/tajochar_little.jpg      |   Bin 27712 -> 0 bytes
 .../webapps/static/img/tajochar_middle.jpg      |   Bin 129938 -> 0 bytes
 .../static/img/tajochar_queries_small.jpg       |   Bin 5184 -> 0 bytes
 .../webapps/static/img/tajochar_small.jpg       |   Bin 65024 -> 0 bytes
 .../webapps/static/img/tajochar_title.jpg       |   Bin 40249 -> 0 bytes
 .../webapps/static/img/tajochar_title_small.jpg |   Bin 10308 -> 0 bytes
 .../static/img/tajochar_worker_small.jpg        |   Bin 13329 -> 0 bytes
 .../webapps/static/js/jquery-ui.min.js          |     5 -
 .../main/resources/webapps/static/js/jquery.js  |     2 -
 .../static/js/jquery.jsPlumb-1.3.16-all-min.js  |     1 -
 .../static/js/jquery.jsPlumb-1.3.16-all.js      | 10561 -----------------
 .../main/resources/webapps/static/queryplan.css |    55 -
 .../src/main/resources/webapps/static/style.css |    74 -
 .../resources/webapps/worker/WEB-INF/web.xml    |    27 -
 .../src/main/resources/webapps/worker/conf.jsp  |    55 -
 .../src/main/resources/webapps/worker/env.jsp   |    69 -
 .../main/resources/webapps/worker/header.jsp    |    37 -
 .../src/main/resources/webapps/worker/index.jsp |   150 -
 .../resources/webapps/worker/querydetail.jsp    |   105 -
 .../main/resources/webapps/worker/queryplan.jsp |   237 -
 .../resources/webapps/worker/querytasks.jsp     |   232 -
 .../main/resources/webapps/worker/queryunit.jsp |   171 -
 .../resources/webapps/worker/taskcontainers.jsp |    87 -
 .../resources/webapps/worker/taskdetail.jsp     |   127 -
 .../src/main/resources/webapps/worker/tasks.jsp |    94 -
 .../main/resources/webapps/worker/thread.jsp    |    47 -
 .../src/test/java/log4j.properties              |    28 -
 .../org/apache/tajo/BackendTestingUtil.java     |    76 -
 .../java/org/apache/tajo/IntegrationTest.java   |    22 -
 .../apache/tajo/LocalTajoTestingUtility.java    |   136 -
 .../org/apache/tajo/MiniTajoYarnCluster.java    |   175 -
 .../java/org/apache/tajo/QueryTestCaseBase.java |   539 -
 .../org/apache/tajo/TajoTestingCluster.java     |   630 -
 .../org/apache/tajo/TestQueryIdFactory.java     |    58 -
 .../test/java/org/apache/tajo/TestTajoIds.java  |   168 -
 .../test/java/org/apache/tajo/TpchTestBase.java |   117 -
 .../org/apache/tajo/benchmark/TestTPCH.java     |    56 -
 .../tajo/cli/TestExecExternalShellCommand.java  |    46 -
 .../org/apache/tajo/cli/TestHdfsCommand.java    |    46 -
 .../org/apache/tajo/cli/TestSimpleParser.java   |   179 -
 .../org/apache/tajo/client/TestDDLBuilder.java  |   133 -
 .../org/apache/tajo/client/TestTajoClient.java  |   666 --
 .../org/apache/tajo/cluster/TestServerName.java |   102 -
 .../apache/tajo/engine/eval/ExprTestBase.java   |   210 -
 .../apache/tajo/engine/eval/TestEvalTree.java   |   323 -
 .../tajo/engine/eval/TestEvalTreeUtil.java      |   359 -
 .../apache/tajo/engine/eval/TestPredicates.java |   396 -
 .../tajo/engine/eval/TestSQLDateTimeTypes.java  |    53 -
 .../tajo/engine/eval/TestSQLExpression.java     |   206 -
 .../tajo/engine/function/TestAggFunction.java   |    64 -
 .../engine/function/TestBuiltinFunctions.java   |   123 -
 .../function/TestConditionalExpressions.java    |    82 -
 .../engine/function/TestDateTimeFunctions.java  |   253 -
 .../engine/function/TestGeneralFunction.java    |    49 -
 .../tajo/engine/function/TestMathFunctions.java |   428 -
 .../function/TestPatternMatchingPredicates.java |   140 -
 .../TestStringOperatorsAndFunctions.java        |   610 -
 .../tajo/engine/parser/TestHiveQLAnalyzer.java  |   271 -
 .../tajo/engine/parser/TestSQLAnalyzer.java     |   473 -
 .../tajo/engine/planner/TestLogicalNode.java    |    74 -
 .../engine/planner/TestLogicalOptimizer.java    |   263 -
 .../tajo/engine/planner/TestLogicalPlan.java    |   125 -
 .../tajo/engine/planner/TestLogicalPlanner.java |  1136 --
 .../tajo/engine/planner/TestPlannerUtil.java    |   321 -
 .../engine/planner/TestSimpleDirectedGraph.java |    79 -
 .../planner/TestSimpleUndirectedGraph.java      |    96 -
 .../planner/TestUniformRangePartition.java      |   442 -
 .../planner/global/TestBroadcastJoinPlan.java   |   400 -
 .../engine/planner/global/TestMasterPlan.java   |    53 -
 .../planner/physical/TestBNLJoinExec.java       |   217 -
 .../planner/physical/TestBSTIndexExec.java      |   210 -
 .../planner/physical/TestExternalSortExec.java  |   184 -
 .../physical/TestFullOuterHashJoinExec.java     |   403 -
 .../physical/TestFullOuterMergeJoinExec.java    |   536 -
 .../planner/physical/TestHashAntiJoinExec.java  |   209 -
 .../planner/physical/TestHashJoinExec.java      |   275 -
 .../planner/physical/TestHashPartitioner.java   |    84 -
 .../planner/physical/TestHashSemiJoinExec.java  |   214 -
 .../physical/TestLeftOuterHashJoinExec.java     |   463 -
 .../physical/TestLeftOuterNLJoinExec.java       |   474 -
 .../planner/physical/TestMergeJoinExec.java     |   193 -
 .../engine/planner/physical/TestNLJoinExec.java |   209 -
 .../planner/physical/TestPhysicalPlanner.java   |  1023 --
 .../physical/TestProgressExternalSortExec.java  |   233 -
 .../physical/TestRightOuterHashJoinExec.java    |   350 -
 .../physical/TestRightOuterMergeJoinExec.java   |   520 -
 .../engine/planner/physical/TestSortExec.java   |   166 -
 .../tajo/engine/query/TestAlterTable.java       |    52 -
 .../tajo/engine/query/TestAlterTablespace.java  |    63 -
 .../apache/tajo/engine/query/TestCTASQuery.java |   239 -
 .../tajo/engine/query/TestCaseByCases.java      |    70 -
 .../tajo/engine/query/TestCreateDatabase.java   |    75 -
 .../tajo/engine/query/TestCreateTable.java      |   360 -
 .../apache/tajo/engine/query/TestDropTable.java |    38 -
 .../tajo/engine/query/TestGroupByQuery.java     |   246 -
 .../tajo/engine/query/TestInsertQuery.java      |   262 -
 .../tajo/engine/query/TestJoinBroadcast.java    |   377 -
 .../query/TestJoinOnPartitionedTables.java      |    56 -
 .../apache/tajo/engine/query/TestJoinQuery.java |   325 -
 .../apache/tajo/engine/query/TestNetTypes.java  |   102 -
 .../tajo/engine/query/TestNullValues.java       |   162 -
 .../tajo/engine/query/TestSelectQuery.java      |   303 -
 .../apache/tajo/engine/query/TestSortQuery.java |   146 -
 .../tajo/engine/query/TestTablePartitions.java  |   439 -
 .../tajo/engine/query/TestTableSubQuery.java    |    60 -
 .../tajo/engine/query/TestUnionQuery.java       |   141 -
 .../apache/tajo/engine/util/TestTupleCache.java |    89 -
 .../apache/tajo/engine/util/TestTupleUtil.java  |   171 -
 .../org/apache/tajo/jdbc/TestResultSet.java     |   127 -
 .../tajo/jdbc/TestTajoDatabaseMetaData.java     |   355 -
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java |   386 -
 .../tajo/master/TestExecutionBlockCursor.java   |   123 -
 .../apache/tajo/master/TestGlobalPlanner.java   |   205 -
 .../apache/tajo/master/TestRepartitioner.java   |    77 -
 .../querymaster/TestQueryUnitStatusUpdate.java  |   175 -
 .../tajo/master/rm/TestTajoResourceManager.java |   378 -
 .../apache/tajo/storage/TestFileFragment.java   |    93 -
 .../org/apache/tajo/storage/TestRowFile.java    |   145 -
 .../java/org/apache/tajo/util/TestJSPUtil.java  |    84 -
 .../tajo/util/metrics/TestMetricsFilter.java    |    52 -
 .../tajo/util/metrics/TestSystemMetrics.java    |   133 -
 .../org/apache/tajo/worker/TestFetcher.java     |    94 -
 .../tajo/worker/TestRangeRetrieverHandler.java  |   373 -
 .../worker/dataserver/TestHttpDataServer.java   |   172 -
 .../tajo/worker/dataserver/TestHttpUtil.java    |    49 -
 .../resources/dataset/TestAlterTable/table1.tbl |     3 -
 .../dataset/TestCreateTable/table1/table1.tbl   |     3 -
 .../dataset/TestCreateTable/table2/table2.tbl   |     3 -
 .../resources/dataset/TestDropTable/table1.tbl  |     3 -
 .../customer_large/customer.tbl                 |   100 -
 .../lineitem_large/lineitem.tbl                 |    97 -
 .../dataset/TestJoinQuery/table1/table1.tbl     |     5 -
 .../dataset/TestJoinQuery/table2/table2.tbl     |     4 -
 .../dataset/TestNetTypes/table1/table1.tbl      |     5 -
 .../dataset/TestNetTypes/table2/table2.tbl      |     5 -
 .../dataset/TestSortQuery/table1/table1.tbl     |     5 -
 .../dataset/TestSortQuery/table2/table2.tbl     |    24 -
 .../src/test/resources/log4j.properties         |    28 -
 .../org/apache/tajo/client/TestTajoDump.java    |    44 -
 .../alter_table_add_new_column_ddl.sql          |     1 -
 .../alter_table_rename_column_ddl.sql           |     1 -
 .../alter_table_rename_table_ddl.sql            |     1 -
 .../queries/TestAlterTable/table1_ddl.sql       |     1 -
 .../TestBuiltinFunctions/testAvgDouble.sql      |     1 -
 .../queries/TestBuiltinFunctions/testAvgInt.sql |     1 -
 .../TestBuiltinFunctions/testAvgLong.sql        |     1 -
 .../queries/TestBuiltinFunctions/testCount.sql  |     1 -
 .../TestBuiltinFunctions/testMaxLong.sql        |     1 -
 .../TestBuiltinFunctions/testMaxString.sql      |     1 -
 .../TestBuiltinFunctions/testMinLong.sql        |     1 -
 .../TestBuiltinFunctions/testMinString.sql      |     1 -
 .../queries/TestBuiltinFunctions/testRandom.sql |     1 -
 .../TestBuiltinFunctions/testSplitPart.sql      |     1 -
 .../testSplitPartByString.sql                   |     1 -
 .../testSplitPartNested.sql                     |     1 -
 .../queries/TestCTASQuery/CtasWithGroupby.sql   |     2 -
 .../queries/TestCTASQuery/CtasWithLimit.sql     |    13 -
 .../TestCTASQuery/CtasWithManagedTable.sql      |    12 -
 .../queries/TestCTASQuery/CtasWithOptions.sql   |    13 -
 .../queries/TestCTASQuery/CtasWithOrderby.sql   |    11 -
 .../queries/TestCTASQuery/CtasWithStoreType.sql |    12 -
 .../queries/TestCTASQuery/CtasWithUnion.sql     |    35 -
 .../resources/queries/TestCTASQuery/check1.sql  |     1 -
 .../resources/queries/TestCTASQuery/check2.sql  |     1 -
 .../testCtasWithColumnedPartition.sql           |     2 -
 .../TestCTASQuery/testCtasWithGroupby.sql       |     1 -
 .../queries/TestCTASQuery/testCtasWithLimit.sql |     1 -
 .../TestCTASQuery/testCtasWithOptions.sql       |     1 -
 .../TestCTASQuery/testCtasWithOrderby.sql       |     1 -
 .../TestCTASQuery/testCtasWithStoreType.sql     |     1 -
 .../queries/TestCTASQuery/testCtasWithUnion.sql |     1 -
 .../testCtasWithoutTableDefinition.sql          |     2 -
 .../queries/TestCaseByCases/testTAJO415Case.sql |    36 -
 .../queries/TestCaseByCases/testTAJO418Case.sql |    29 -
 .../queries/TestCaseByCases/testTAJO619Case.sql |     4 -
 .../queries/TestCaseByCases/testTAJO718Case.sql |    10 -
 .../queries/TestCaseByCases/testTAJO739Case.sql |     5 -
 .../create_table_various_types.sql              |    50 -
 .../create_table_various_types_for_hcatalog.sql |    50 -
 .../quoted_identifier_mixed_chars_1.sql         |     1 -
 .../quoted_identifier_mixed_chars_2.sql         |     1 -
 .../quoted_identifier_mixed_chars_3.sql         |    22 -
 .../quoted_identifier_mixed_chars_ddl_1.sql     |     1 -
 .../quoted_identifier_mixed_chars_ddl_2.sql     |     1 -
 .../quoted_identifier_non_ascii_1.sql           |     1 -
 .../quoted_identifier_non_ascii_2.sql           |     1 -
 .../quoted_identifier_non_ascii_3.sql           |     1 -
 .../quoted_identifier_non_ascii_ddl.sql         |     1 -
 .../queries/TestCreateTable/table1_ddl.sql      |     1 -
 .../queries/TestDropTable/drop_table_ddl.sql    |     1 -
 .../queries/TestDropTable/table1_ddl.sql        |     1 -
 .../TestGroupByQuery/testComplexParameter.sql   |     1 -
 .../TestGroupByQuery/testComplexParameter2.sql  |     1 -
 .../testComplexParameterWithSubQuery.sql        |     6 -
 .../testDistinctAggregation1.sql                |     1 -
 .../testDistinctAggregation2.sql                |     1 -
 .../testDistinctAggregation3.sql                |     1 -
 .../testDistinctAggregation4.sql                |     1 -
 .../testDistinctAggregation5.sql                |     1 -
 .../testDistinctAggregation6.sql                |     9 -
 .../testDistinctAggregationWithHaving1.sql      |     1 -
 .../testDistinctAggregationWithUnion1.sql       |    21 -
 .../queries/TestGroupByQuery/testGroupBy.sql    |     1 -
 .../queries/TestGroupByQuery/testGroupBy2.sql   |     1 -
 .../queries/TestGroupByQuery/testGroupBy3.sql   |     1 -
 .../queries/TestGroupByQuery/testGroupBy4.sql   |     9 -
 .../queries/TestGroupByQuery/testGroupBy5.sql   |     1 -
 .../TestGroupByQuery/testGroupByNested1.sql     |     8 -
 .../TestGroupByQuery/testGroupByNested2.sql     |    10 -
 .../testGroupByWithConstantKeys1.sql            |     1 -
 .../testGroupByWithExpressionKeys1.sql          |     1 -
 .../testGroupByWithExpressionKeys2.sql          |     1 -
 .../testGroupByWithSameExprs1.sql               |     6 -
 .../testGroupByWithSameExprs2.sql               |     6 -
 .../testHavingWithAggFunction.sql               |     1 -
 .../testHavingWithNamedTarget.sql               |     1 -
 .../create_customer_large_ddl.sql               |     7 -
 .../create_lineitem_large_ddl.sql               |     7 -
 .../queries/TestJoinBroadcast/oj_table1_ddl.sql |     6 -
 .../queries/TestJoinBroadcast/oj_table2_ddl.sql |     6 -
 .../testBroadcastBasicJoin.sql                  |    11 -
 .../TestJoinBroadcast/testBroadcastSubquery.sql |    11 -
 .../testBroadcastSubquery2.sql                  |    19 -
 .../testBroadcastTwoPartJoin.sql                |    15 -
 .../queries/TestJoinBroadcast/testCrossJoin.sql |     1 -
 .../testCrossJoinAndCaseWhen.sql                |    18 -
 .../testCrossJoinWithAsterisk1.sql              |     1 -
 .../testCrossJoinWithAsterisk2.sql              |     1 -
 .../testCrossJoinWithAsterisk3.sql              |     1 -
 .../testCrossJoinWithAsterisk4.sql              |     1 -
 .../testCrossJoinWithEmptyTable1.sql            |     8 -
 .../TestJoinBroadcast/testFullOuterJoin1.sql    |     8 -
 .../testFullOuterJoinWithEmptyTable1.sql        |     8 -
 .../testInnerJoinWithEmptyTable.sql             |     8 -
 .../testJoinCoReferredEvals1.sql                |    11 -
 .../testJoinCoReferredEvalsWithSameExprs1.sql   |    14 -
 .../testJoinCoReferredEvalsWithSameExprs2.sql   |    22 -
 .../testJoinOnMultipleDatabases.sql             |    25 -
 .../TestJoinBroadcast/testLeftOuterJoin1.sql    |     9 -
 .../testLeftOuterJoinWithConstantExpr1.sql      |     9 -
 .../testLeftOuterJoinWithConstantExpr2.sql      |     9 -
 .../testLeftOuterJoinWithConstantExpr3.sql      |    17 -
 .../testLeftOuterJoinWithEmptyTable1.sql        |     9 -
 .../testLeftOuterJoinWithEmptyTable2.sql        |     9 -
 .../testLeftOuterJoinWithEmptyTable3.sql        |    14 -
 .../testLeftOuterJoinWithEmptyTable4.sql        |    17 -
 .../testOuterJoinAndCaseWhen1.sql               |    12 -
 .../TestJoinBroadcast/testRightOuterJoin1.sql   |     8 -
 .../testRightOuterJoinWithEmptyTable1.sql       |     8 -
 .../TestJoinBroadcast/testTPCHQ2Join.sql        |    25 -
 .../TestJoinBroadcast/testWhereClauseJoin1.sql  |    10 -
 .../TestJoinBroadcast/testWhereClauseJoin2.sql  |     8 -
 .../TestJoinBroadcast/testWhereClauseJoin3.sql  |     9 -
 .../TestJoinBroadcast/testWhereClauseJoin4.sql  |     8 -
 .../TestJoinBroadcast/testWhereClauseJoin5.sql  |    15 -
 .../TestJoinBroadcast/testWhereClauseJoin6.sql  |    19 -
 .../customer_ddl.sql                            |     9 -
 .../insert_into_customer.sql                    |    11 -
 .../selfJoinOfPartitionedTable.sql              |     9 -
 .../testNoProjectionJoinQual.sql                |     1 -
 .../testPartialFilterPushDown.sql               |     9 -
 .../testPartitionTableJoinSmallTable.sql        |    11 -
 .../queries/TestJoinQuery/oj_table1_ddl.sql     |     6 -
 .../queries/TestJoinQuery/oj_table2_ddl.sql     |     6 -
 .../queries/TestJoinQuery/testCrossJoin.sql     |     1 -
 .../TestJoinQuery/testCrossJoinAndCaseWhen.sql  |    18 -
 .../testCrossJoinWithAsterisk1.sql              |     1 -
 .../testCrossJoinWithAsterisk2.sql              |     1 -
 .../testCrossJoinWithAsterisk3.sql              |     1 -
 .../testCrossJoinWithAsterisk4.sql              |     1 -
 .../testCrossJoinWithEmptyTable1.sql            |     8 -
 .../TestJoinQuery/testFullOuterJoin1.sql        |     8 -
 .../testFullOuterJoinWithEmptyTable1.sql        |     8 -
 .../testInnerJoinWithEmptyTable.sql             |     8 -
 .../TestJoinQuery/testJoinCoReferredEvals1.sql  |    11 -
 .../testJoinCoReferredEvalsWithSameExprs1.sql   |    14 -
 .../testJoinCoReferredEvalsWithSameExprs2.sql   |    22 -
 .../testJoinOnMultipleDatabases.sql             |    25 -
 .../TestJoinQuery/testLeftOuterJoin1.sql        |     9 -
 .../testLeftOuterJoinWithConstantExpr1.sql      |     9 -
 .../testLeftOuterJoinWithConstantExpr2.sql      |     9 -
 .../testLeftOuterJoinWithConstantExpr3.sql      |    17 -
 .../testLeftOuterJoinWithEmptyTable1.sql        |     9 -
 .../testLeftOuterJoinWithEmptyTable2.sql        |     9 -
 .../testLeftOuterJoinWithEmptyTable3.sql        |    14 -
 .../testLeftOuterJoinWithEmptyTable4.sql        |    17 -
 .../TestJoinQuery/testOuterJoinAndCaseWhen1.sql |    12 -
 .../TestJoinQuery/testRightOuterJoin1.sql       |     8 -
 .../testRightOuterJoinWithEmptyTable1.sql       |     8 -
 .../queries/TestJoinQuery/testTPCHQ2Join.sql    |    25 -
 .../TestJoinQuery/testWhereClauseJoin1.sql      |    10 -
 .../TestJoinQuery/testWhereClauseJoin2.sql      |     8 -
 .../TestJoinQuery/testWhereClauseJoin3.sql      |     9 -
 .../TestJoinQuery/testWhereClauseJoin4.sql      |     8 -
 .../TestJoinQuery/testWhereClauseJoin5.sql      |    15 -
 .../TestJoinQuery/testWhereClauseJoin6.sql      |    19 -
 .../queries/TestNetTypes/table1_ddl.sql         |     4 -
 .../queries/TestNetTypes/table2_ddl.sql         |     4 -
 .../queries/TestNetTypes/testGroupby.sql        |     8 -
 .../queries/TestNetTypes/testGroupby2.sql       |     9 -
 .../resources/queries/TestNetTypes/testJoin.sql |     1 -
 .../queries/TestNetTypes/testSelect.sql         |     1 -
 .../resources/queries/TestNetTypes/testSort.sql |     1 -
 .../queries/TestNetTypes/testSort2.sql          |     1 -
 .../queries/TestQueryUnitStatusUpdate/case1.sql |     1 -
 .../queries/TestQueryUnitStatusUpdate/case2.sql |     5 -
 .../queries/TestQueryUnitStatusUpdate/case3.sql |    11 -
 .../queries/TestSelectQuery/testCaseWhen.sql    |    11 -
 .../TestSelectQuery/testCaseWhenWithoutElse.sql |    10 -
 .../TestSelectQuery/testCreateAfterSelect.sql   |     1 -
 .../TestSelectQuery/testDatabaseRef1.sql        |     1 -
 .../TestSelectQuery/testDatabaseRef2.sql        |     1 -
 .../TestSelectQuery/testDatabaseRef3.sql        |     1 -
 .../TestSelectQuery/testExplainSelect.sql       |     1 -
 .../queries/TestSelectQuery/testInClause.sql    |     1 -
 .../queries/TestSelectQuery/testInStrClause.sql |     1 -
 .../queries/TestSelectQuery/testLikeClause.sql  |     1 -
 .../queries/TestSelectQuery/testLimit.sql       |     1 -
 .../TestSelectQuery/testNonFromSelect1.sql      |     1 -
 .../queries/TestSelectQuery/testNotEqual.sql    |     1 -
 .../queries/TestSelectQuery/testNotInClause.sql |     1 -
 .../TestSelectQuery/testNotInStrClause.sql      |     1 -
 .../TestSelectQuery/testRealValueCompare.sql    |     1 -
 .../queries/TestSelectQuery/testSelect.sql      |     1 -
 .../queries/TestSelectQuery/testSelect2.sql     |     1 -
 .../queries/TestSelectQuery/testSelect3.sql     |     1 -
 .../TestSelectQuery/testSelectAsterik.sql       |     1 -
 .../TestSelectQuery/testSelectAsterisk1.sql     |     1 -
 .../TestSelectQuery/testSelectAsterisk2.sql     |     1 -
 .../TestSelectQuery/testSelectAsterisk3.sql     |     1 -
 .../TestSelectQuery/testSelectAsterisk4.sql     |     1 -
 .../TestSelectQuery/testSelectAsterisk5.sql     |     1 -
 .../TestSelectQuery/testSelectColumnAlias1.sql  |     1 -
 .../TestSelectQuery/testSelectDistinct.sql      |     7 -
 ...tSelectSameConstantsWithDifferentAliases.sql |     1 -
 .../testSelectSameExprsWithDifferentAliases.sql |     1 -
 .../queries/TestSelectQuery/testSimpleQuery.sql |     1 -
 .../testSimpleQueryWithLimit.sql                |     1 -
 .../TestSelectQuery/testStringCompare.sql       |     1 -
 .../queries/TestSelectQuery/testWhereCond1.sql  |     1 -
 .../TestSelectQuery/testWhereCondWithAlias1.sql |     1 -
 .../TestSelectQuery/testWhereCondWithAlias2.sql |     1 -
 .../create_table_with_asc_desc_keys.sql         |     1 -
 .../create_table_with_date_ddl.sql              |    10 -
 .../queries/TestSortQuery/testAsterisk.sql      |     1 -
 .../queries/TestSortQuery/testSort.sql          |     1 -
 .../TestSortQuery/testSortAfterGroupby.sql      |     1 -
 .../testSortAfterGroupbyWithAlias.sql           |     1 -
 .../queries/TestSortQuery/testSortDesc.sql      |     1 -
 .../TestSortQuery/testSortWithAlias1.sql        |     1 -
 .../TestSortQuery/testSortWithAlias2.sql        |     9 -
 .../TestSortQuery/testSortWithAlias3.sql        |    11 -
 .../testSortWithAliasButOriginalName.sql        |     1 -
 .../TestSortQuery/testSortWithAscDescKeys.sql   |     1 -
 .../queries/TestSortQuery/testSortWithDate.sql  |     1 -
 .../queries/TestSortQuery/testSortWithExpr1.sql |     1 -
 .../queries/TestSortQuery/testSortWithExpr2.sql |     1 -
 .../queries/TestSortQuery/testTopK.sql          |     1 -
 .../queries/TestTPCH/testQ1OrderBy.sql          |    12 -
 .../queries/TestTPCH/testQ2FourJoins.sql        |    18 -
 .../queries/TestTPCH/testTPCH14Expr.sql         |    12 -
 .../queries/TestTablePartitions/case1.sql       |    16 -
 .../queries/TestTablePartitions/case2.sql       |    31 -
 .../queries/TestTablePartitions/case3.sql       |     8 -
 .../TestTableSubQuery/testGroupBySubQuery.sql   |     1 -
 .../TestTableSubQuery/testJoinSubQuery.sql      |     5 -
 .../TestTableSubQuery/testJoinSubQuery2.sql     |     4 -
 .../TestTableSubQuery/testTableSubquery1.sql    |     1 -
 .../queries/TestUnionQuery/testUnion1.sql       |    19 -
 .../queries/TestUnionQuery/testUnion10.sql      |   158 -
 .../queries/TestUnionQuery/testUnion2.sql       |    18 -
 .../queries/TestUnionQuery/testUnion3.sql       |    19 -
 .../queries/TestUnionQuery/testUnion4.sql       |    16 -
 .../queries/TestUnionQuery/testUnion5.sql       |    20 -
 .../queries/TestUnionQuery/testUnion6.sql       |    15 -
 .../queries/TestUnionQuery/testUnion7.sql       |    18 -
 .../queries/TestUnionQuery/testUnion8.sql       |    24 -
 .../queries/TestUnionQuery/testUnion9.sql       |    29 -
 .../testUnionWithSameAliasNames.sql             |    25 -
 .../resources/queries/default/asterisk_1.sql    |     1 -
 .../resources/queries/default/asterisk_2.sql    |     1 -
 .../resources/queries/default/asterisk_3.sql    |     1 -
 .../resources/queries/default/asterisk_4.sql    |     1 -
 .../queries/default/complex_union_1.sql         |    29 -
 .../queries/default/complex_union_2.sql         |    35 -
 .../create_partitioned_table_as_select.sql      |    17 -
 .../queries/default/create_table_1.hiveql       |     1 -
 .../queries/default/create_table_1.sql          |     1 -
 .../queries/default/create_table_10.sql         |     1 -
 .../queries/default/create_table_11.hiveql      |     3 -
 .../queries/default/create_table_11.sql         |     3 -
 .../queries/default/create_table_12.hiveql      |     4 -
 .../queries/default/create_table_12.sql         |     3 -
 .../queries/default/create_table_2.hiveql       |     1 -
 .../queries/default/create_table_2.sql          |     1 -
 .../queries/default/create_table_3.sql          |     1 -
 .../queries/default/create_table_4.sql          |     1 -
 .../queries/default/create_table_5.sql          |     1 -
 .../queries/default/create_table_6.sql          |     1 -
 .../queries/default/create_table_7.sql          |     1 -
 .../queries/default/create_table_8.sql          |    48 -
 .../queries/default/create_table_9.sql          |     1 -
 .../create_table_partition_by_column.sql        |     4 -
 .../create_table_partition_by_hash_1.sql        |     3 -
 .../create_table_partition_by_hash_2.sql        |     7 -
 .../default/create_table_partition_by_list.sql  |     8 -
 .../default/create_table_partition_by_range.sql |     9 -
 .../resources/queries/default/drop_table.sql    |     1 -
 .../queries/default/exists_predicate_1.sql      |     1 -
 .../queries/default/exists_predicate_2.sql      |     1 -
 .../resources/queries/default/groupby_1.sql     |     1 -
 .../resources/queries/default/groupby_2.sql     |     1 -
 .../resources/queries/default/groupby_3.sql     |     1 -
 .../resources/queries/default/groupby_4.sql     |     1 -
 .../resources/queries/default/groupby_5.sql     |     1 -
 .../resources/queries/default/in_subquery_1.sql |     1 -
 .../resources/queries/default/in_subquery_2.sql |     1 -
 .../queries/default/insert_into_select_1.sql    |     1 -
 .../queries/default/insert_into_select_2.sql    |     1 -
 .../queries/default/insert_into_select_3.sql    |     1 -
 .../default/insert_overwrite_into_select_1.sql  |     1 -
 .../insert_overwrite_into_select_2.hiveql       |     1 -
 .../default/insert_overwrite_into_select_2.sql  |     1 -
 .../default/insert_overwrite_into_select_3.sql  |     1 -
 .../test/resources/queries/default/join_1.sql   |     1 -
 .../test/resources/queries/default/join_10.sql  |     1 -
 .../test/resources/queries/default/join_11.sql  |     1 -
 .../test/resources/queries/default/join_12.sql  |     1 -
 .../test/resources/queries/default/join_13.sql  |    13 -
 .../test/resources/queries/default/join_14.sql  |     1 -
 .../resources/queries/default/join_15.hiveql    |     1 -
 .../test/resources/queries/default/join_15.sql  |     1 -
 .../test/resources/queries/default/join_2.sql   |     1 -
 .../test/resources/queries/default/join_3.sql   |     1 -
 .../test/resources/queries/default/join_4.sql   |     1 -
 .../test/resources/queries/default/join_5.sql   |     1 -
 .../test/resources/queries/default/join_6.sql   |     1 -
 .../test/resources/queries/default/join_7.sql   |     1 -
 .../test/resources/queries/default/join_8.sql   |    13 -
 .../test/resources/queries/default/join_9.sql   |     5 -
 .../test/resources/queries/default/select_1.sql |     1 -
 .../resources/queries/default/select_10.hiveql  |     5 -
 .../resources/queries/default/select_10.sql     |     5 -
 .../resources/queries/default/select_11.hiveql  |     4 -
 .../resources/queries/default/select_11.sql     |     6 -
 .../resources/queries/default/select_12.hiveql  |     3 -
 .../resources/queries/default/select_13.hiveql  |     2 -
 .../resources/queries/default/select_13.sql     |     2 -
 .../resources/queries/default/select_14.sql     |     2 -
 .../resources/queries/default/select_15.hiveql  |     1 -
 .../resources/queries/default/select_15.sql     |     1 -
 .../test/resources/queries/default/select_2.sql |     1 -
 .../test/resources/queries/default/select_3.sql |     1 -
 .../test/resources/queries/default/select_4.sql |     1 -
 .../test/resources/queries/default/select_5.sql |     1 -
 .../test/resources/queries/default/select_6.sql |     1 -
 .../test/resources/queries/default/select_7.sql |     1 -
 .../test/resources/queries/default/select_8.sql |     1 -
 .../resources/queries/default/select_9.hiveql   |     4 -
 .../test/resources/queries/default/select_9.sql |     4 -
 .../test/resources/queries/default/set_1.sql    |     1 -
 .../test/resources/queries/default/set_2.sql    |     1 -
 .../test/resources/queries/default/set_3.sql    |     1 -
 .../test/resources/queries/default/set_4.sql    |    36 -
 .../queries/default/table_subquery1.sql         |    19 -
 .../queries/default/table_subquery2.sql         |    21 -
 .../resources/queries/default/union_1.hiveql    |    14 -
 .../TestBuiltinFunctions/testAvgDouble.result   |     5 -
 .../TestBuiltinFunctions/testAvgInt.result      |     3 -
 .../TestBuiltinFunctions/testAvgLong.result     |     3 -
 .../TestBuiltinFunctions/testCount.result       |     3 -
 .../TestBuiltinFunctions/testMaxLong.result     |     3 -
 .../TestBuiltinFunctions/testMaxString.result   |     3 -
 .../TestBuiltinFunctions/testMinLong.result     |     3 -
 .../TestBuiltinFunctions/testMinString.result   |     3 -
 .../TestBuiltinFunctions/testSplitPart.result   |     7 -
 .../testSplitPartByString.result                |     7 -
 .../testSplitPartNested.result                  |     7 -
 .../TestCTASQuery/testCtasWithGroupby.result    |     7 -
 .../TestCTASQuery/testCtasWithLimit.result      |     5 -
 .../TestCTASQuery/testCtasWithOptions.result    |     5 -
 .../TestCTASQuery/testCtasWithOrderby.result    |     7 -
 .../TestCTASQuery/testCtasWithStoreType.result  |     5 -
 .../TestCTASQuery/testCtasWithUnion.result      |     8 -
 .../TestCaseByCases/testTAJO415Case.result      |     7 -
 .../TestCaseByCases/testTAJO418Case.result      |     3 -
 .../TestCaseByCases/testTAJO619Case.result      |     3 -
 .../TestCaseByCases/testTAJO718Case.result      |     5 -
 .../TestCaseByCases/testTAJO739Case.result      |     7 -
 .../quoted_identifier_mixed_chars_1.result      |     5 -
 .../quoted_identifier_mixed_chars_2.result      |     5 -
 .../quoted_identifier_mixed_chars_3.result      |     8 -
 .../quoted_identifier_non_ascii_1.result        |     5 -
 .../quoted_identifier_non_ascii_2.result        |     5 -
 .../quoted_identifier_non_ascii_3.result        |     5 -
 .../testComplexParameter.result                 |     3 -
 .../testComplexParameter2.result                |     3 -
 .../testComplexParameterWithSubQuery.result     |     3 -
 .../testDistinctAggregation1.result             |     5 -
 .../testDistinctAggregation2.result             |     5 -
 .../testDistinctAggregation3.result             |     3 -
 .../testDistinctAggregation4.result             |     4 -
 .../testDistinctAggregation5.result             |     4 -
 .../testDistinctAggregation6.result             |     5 -
 .../testDistinctAggregationWithHaving1.result   |     3 -
 .../testDistinctAggregationWithUnion1.result    |     4 -
 .../results/TestGroupByQuery/testGroupBy.result |     3 -
 .../TestGroupByQuery/testGroupBy2.result        |     4 -
 .../TestGroupByQuery/testGroupBy3.result        |     5 -
 .../TestGroupByQuery/testGroupBy4.result        |     5 -
 .../TestGroupByQuery/testGroupBy5.result        |     5 -
 .../TestGroupByQuery/testGroupByNested1.result  |     6 -
 .../TestGroupByQuery/testGroupByNested2.result  |     6 -
 .../testGroupByWithExpressionKeys1.result       |     5 -
 .../testGroupByWithExpressionKeys2.result       |     5 -
 .../testGroupByWithSameExprs1.result            |     6 -
 .../testGroupByWithSameExprs2.result            |     6 -
 .../testHavingWithAggFunction.result            |     4 -
 .../testHavingWithNamedTarget.result            |     5 -
 .../testBroadcastBasicJoin.result               |    99 -
 .../testBroadcastSubquery.result                |    15 -
 .../testBroadcastSubquery2.result               |     3 -
 .../testBroadcastTwoPartJoin.result             |    15 -
 .../TestJoinBroadcast/testCrossJoin.result      |   127 -
 .../testCrossJoinAndCaseWhen.result             |    27 -
 .../testCrossJoinWithAsterisk1.result           |    27 -
 .../testCrossJoinWithAsterisk2.result           |    27 -
 .../testCrossJoinWithAsterisk3.result           |    27 -
 .../testCrossJoinWithAsterisk4.result           |    27 -
 .../testCrossJoinWithEmptyTable1.result         |     2 -
 .../TestJoinBroadcast/testFullOuterJoin1.result |     7 -
 .../testFullOuterJoinWithEmptyTable1.result     |     7 -
 .../testInnerJoinWithEmptyTable.result          |     2 -
 .../testJoinCoReferredEvals1.result             |    27 -
 ...testJoinCoReferredEvalsWithSameExprs1.result |    22 -
 ...testJoinCoReferredEvalsWithSameExprs2.result |    22 -
 .../testJoinOnMultipleDatabases.result          |     5 -
 .../TestJoinBroadcast/testLeftOuterJoin1.result |     7 -
 .../testLeftOuterJoinWithConstantExpr1.result   |     7 -
 .../testLeftOuterJoinWithConstantExpr2.result   |     7 -
 .../testLeftOuterJoinWithConstantExpr3.result   |     7 -
 .../testLeftOuterJoinWithEmptyTable1.result     |     7 -
 .../testLeftOuterJoinWithEmptyTable2.result     |     7 -
 .../testLeftOuterJoinWithEmptyTable3.result     |     7 -
 .../testLeftOuterJoinWithEmptyTable4.result     |     4 -
 .../testOuterJoinAndCaseWhen1.result            |     7 -
 .../testRightOuterJoin1.result                  |     7 -
 .../testRightOuterJoinWithEmptyTable1.result    |     7 -
 .../TestJoinBroadcast/testTPCHQ2Join.result     |     5 -
 .../testWhereClauseJoin1.result                 |    27 -
 .../testWhereClauseJoin2.result                 |    27 -
 .../testWhereClauseJoin3.result                 |    27 -
 .../testWhereClauseJoin4.result                 |    27 -
 .../testWhereClauseJoin5.result                 |     5 -
 .../selfJoinOfPartitionedTable.result           |     7 -
 .../testNoProjectionJoinQual.result             |     3 -
 .../testPartialFilterPushDown.result            |     3 -
 .../testPartitionTableJoinSmallTable.result     |     7 -
 .../results/TestJoinQuery/testCrossJoin.result  |   127 -
 .../testCrossJoinAndCaseWhen.result             |    27 -
 .../testCrossJoinWithAsterisk1.result           |    27 -
 .../testCrossJoinWithAsterisk2.result           |    27 -
 .../testCrossJoinWithAsterisk3.result           |    27 -
 .../testCrossJoinWithAsterisk4.result           |    27 -
 .../testCrossJoinWithEmptyTable1.result         |     2 -
 .../TestJoinQuery/testFullOuterJoin1.result     |     7 -
 .../testFullOuterJoinWithEmptyTable1.result     |     7 -
 .../testInnerJoinWithEmptyTable.result          |     2 -
 .../testJoinCoReferredEvals1.result             |    27 -
 ...testJoinCoReferredEvalsWithSameExprs1.result |    22 -
 ...testJoinCoReferredEvalsWithSameExprs2.result |    22 -
 .../testJoinOnMultipleDatabases.result          |     5 -
 .../TestJoinQuery/testLeftOuterJoin1.result     |     7 -
 .../testLeftOuterJoinWithConstantExpr1.result   |     7 -
 .../testLeftOuterJoinWithConstantExpr2.result   |     7 -
 .../testLeftOuterJoinWithConstantExpr3.result   |     7 -
 .../testLeftOuterJoinWithEmptyTable1.result     |     7 -
 .../testLeftOuterJoinWithEmptyTable2.result     |     7 -
 .../testLeftOuterJoinWithEmptyTable3.result     |     7 -
 .../testLeftOuterJoinWithEmptyTable4.result     |     4 -
 .../testOuterJoinAndCaseWhen1.result            |     7 -
 .../TestJoinQuery/testRightOuterJoin1.result    |     7 -
 .../testRightOuterJoinWithEmptyTable1.result    |     7 -
 .../results/TestJoinQuery/testTPCHQ2Join.result |     5 -
 .../TestJoinQuery/testWhereClauseJoin1.result   |    27 -
 .../TestJoinQuery/testWhereClauseJoin2.result   |    27 -
 .../TestJoinQuery/testWhereClauseJoin3.result   |    27 -
 .../TestJoinQuery/testWhereClauseJoin4.result   |    27 -
 .../TestJoinQuery/testWhereClauseJoin5.result   |     5 -
 .../results/TestNetTypes/testGroupby.result     |     7 -
 .../results/TestNetTypes/testGroupby2.result    |     5 -
 .../results/TestNetTypes/testJoin.result        |     6 -
 .../results/TestNetTypes/testSelect.result      |     7 -
 .../results/TestNetTypes/testSort.result        |     7 -
 .../results/TestNetTypes/testSort2.result       |     6 -
 .../results/TestSelectQuery/testCaseWhen.result |     7 -
 .../testCaseWhenWithoutElse.result              |     7 -
 .../TestSelectQuery/testDatabaseRef.result      |     7 -
 .../TestSelectQuery/testExplainSelect.result    |     6 -
 .../results/TestSelectQuery/testInClause.result |     5 -
 .../TestSelectQuery/testInStrClause.result      |     4 -
 .../TestSelectQuery/testLikeClause.result       |     9 -
 .../results/TestSelectQuery/testLimit.result    |     5 -
 .../TestSelectQuery/testNonFromSelect1.result   |     3 -
 .../results/TestSelectQuery/testNotEqual.result |     5 -
 .../TestSelectQuery/testNotInClause.result      |     4 -
 .../TestSelectQuery/testNotInStrClause.result   |     4 -
 .../TestSelectQuery/testRealValueCompare.result |     3 -
 .../results/TestSelectQuery/testSelect.result   |     7 -
 .../results/TestSelectQuery/testSelect2.result  |     7 -
 .../results/TestSelectQuery/testSelect3.result  |     7 -
 .../TestSelectQuery/testSelectAsterisk1.result  |     7 -
 .../TestSelectQuery/testSelectAsterisk2.result  |     3 -
 .../TestSelectQuery/testSelectAsterisk3.result  |     3 -
 .../TestSelectQuery/testSelectAsterisk4.result  |     7 -
 .../TestSelectQuery/testSelectAsterisk5.result  |     3 -
 .../testSelectColumnAlias1.result               |     7 -
 .../TestSelectQuery/testSelectDistinct.result   |     7 -
 ...lectSameConstantsWithDifferentAliases.result |     7 -
 ...stSelectSameExprsWithDifferentAliases.result |     7 -
 .../TestSelectQuery/testSimpleQuery.result      |     7 -
 .../testSimpleQueryWithLimit.result             |     5 -
 .../TestSelectQuery/testStringCompare.result    |     5 -
 .../results/TestSelectQuery/testUnion1.result   |    10 -
 .../results/TestSelectQuery/testUnion2.result   |    12 -
 .../TestSelectQuery/testWhereCond1.result       |     3 -
 .../testWhereCondWithAlias1.result              |     4 -
 .../testWhereCondWithAlias2.result              |     4 -
 .../results/TestSortQuery/testAsterisk.result   |     7 -
 .../results/TestSortQuery/testSort.result       |     7 -
 .../TestSortQuery/testSortAfterGroupby.result   |     5 -
 .../testSortAfterGroupbyWithAlias.result        |     5 -
 .../results/TestSortQuery/testSortDesc.result   |     7 -
 .../TestSortQuery/testSortWithAlias1.result     |     7 -
 .../TestSortQuery/testSortWithAlias2.result     |     5 -
 .../TestSortQuery/testSortWithAlias3.result     |     7 -
 .../testSortWithAliasButOriginalName.result     |     7 -
 .../testSortWithAscDescKeys.result              |    26 -
 .../TestSortQuery/testSortWithDate.result       |     7 -
 .../TestSortQuery/testSortWithExpr1.result      |     7 -
 .../TestSortQuery/testSortWithExpr2.result      |     7 -
 .../results/TestSortQuery/testTopK.result       |     5 -
 .../results/TestTPCH/testQ1OrderBy.result       |     4 -
 .../results/TestTPCH/testQ2FourJoins.result     |     3 -
 .../results/TestTPCH/testTPCH14Expr.result      |     3 -
 .../results/TestTablePartitions/case1.result    |     4 -
 .../results/TestTablePartitions/case2.result    |     6 -
 .../results/TestTablePartitions/case3.result    |     5 -
 .../testGroupBySubQuery.result                  |     3 -
 .../TestTableSubQuery/testJoinSubQuery.result   |     4 -
 .../TestTableSubQuery/testJoinSubQuery2.result  |     4 -
 .../TestTableSubQuery/testTableSubquery1.result |     7 -
 .../TestTajoDatabaseMetaData/getTables1.result  |     4 -
 .../TestTajoDatabaseMetaData/getTables2.result  |     4 -
 .../results/TestTajoDump/testDump1.result       |    17 -
 .../results/TestUnionQuery/testUnion1.result    |    10 -
 .../results/TestUnionQuery/testUnion10.result   |    22 -
 .../results/TestUnionQuery/testUnion2.result    |    12 -
 .../results/TestUnionQuery/testUnion3.result    |     4 -
 .../results/TestUnionQuery/testUnion4.result    |     3 -
 .../results/TestUnionQuery/testUnion5.result    |     3 -
 .../results/TestUnionQuery/testUnion6.result    |     3 -
 .../results/TestUnionQuery/testUnion7.result    |    12 -
 .../results/TestUnionQuery/testUnion8.result    |     3 -
 .../results/TestUnionQuery/testUnion9.result    |     7 -
 .../testUnionWithSameAliasNames.result          |    12 -
 .../testBuildDDLForBaseTable.result             |     5 -
 .../testBuildDDLForExternalTable.result         |     5 -
 .../testBuildDDLQuotedTableName1.result         |     5 -
 .../testBuildDDLQuotedTableName2.result         |     5 -
 .../src/test/resources/yarn-site.xml            |    25 -
 .../src/test/tpch/customer.tbl                  |     5 -
 .../src/test/tpch/empty_orders.tbl              |     0
 .../src/test/tpch/lineitem.tbl                  |     5 -
 .../tajo-core-backend/src/test/tpch/nation.tbl  |    25 -
 .../tajo-core-backend/src/test/tpch/orders.tbl  |     3 -
 .../tajo-core-backend/src/test/tpch/part.tbl    |     3 -
 .../src/test/tpch/partsupp.tbl                  |     3 -
 .../tajo-core-backend/src/test/tpch/region.tbl  |     5 -
 .../src/test/tpch/supplier.tbl                  |     3 -
 tajo-core/tajo-core-pullserver/pom.xml          |   156 -
 .../pullserver/FadvisedFileRegionWrapper.java   |    35 -
 .../pullserver/listener/FileCloseListener.java  |    40 -
 .../pullserver/FadvisedFileRegionWrapper.java   |    34 -
 .../pullserver/listener/FileCloseListener.java  |    42 -
 .../pullserver/FadvisedFileRegionWrapper.java   |    36 -
 .../pullserver/listener/FileCloseListener.java  |    42 -
 .../FileAccessForbiddenException.java           |    40 -
 .../tajo/pullserver/HttpDataServerHandler.java  |   245 -
 .../HttpDataServerPipelineFactory.java          |    56 -
 .../org/apache/tajo/pullserver/HttpUtil.java    |    69 -
 .../tajo/pullserver/PullServerAuxService.java   |   657 -
 .../tajo/pullserver/TajoPullServerService.java  |   648 -
 .../retriever/AdvancedDataRetriever.java        |   126 -
 .../pullserver/retriever/DataRetriever.java     |    29 -
 .../retriever/DirectoryRetriever.java           |    56 -
 .../tajo/pullserver/retriever/FileChunk.java    |    51 -
 .../pullserver/retriever/RetrieverHandler.java  |    33 -
 .../java/org/apache/tajo/storage/Tuple.java     |    70 -
 tajo-dist/pom.xml                               |     2 +-
 tajo-project/pom.xml                            |    15 +-
 .../org/apache/tajo/storage/TestStorages.java   |     5 +-
 tajo-yarn-pullserver/pom.xml                    |   156 +
 .../pullserver/FadvisedFileRegionWrapper.java   |    35 +
 .../pullserver/listener/FileCloseListener.java  |    40 +
 .../pullserver/FadvisedFileRegionWrapper.java   |    34 +
 .../pullserver/listener/FileCloseListener.java  |    42 +
 .../pullserver/FadvisedFileRegionWrapper.java   |    36 +
 .../pullserver/listener/FileCloseListener.java  |    42 +
 .../FileAccessForbiddenException.java           |    40 +
 .../tajo/pullserver/HttpDataServerHandler.java  |   245 +
 .../HttpDataServerPipelineFactory.java          |    56 +
 .../org/apache/tajo/pullserver/HttpUtil.java    |    69 +
 .../tajo/pullserver/PullServerAuxService.java   |   656 +
 .../tajo/pullserver/TajoPullServerService.java  |   648 +
 .../retriever/AdvancedDataRetriever.java        |   126 +
 .../pullserver/retriever/DataRetriever.java     |    29 +
 .../retriever/DirectoryRetriever.java           |    56 +
 .../tajo/pullserver/retriever/FileChunk.java    |    51 +
 .../pullserver/retriever/RetrieverHandler.java  |    33 +
 .../java/org/apache/tajo/storage/Tuple.java     |    70 +
 2438 files changed, 116240 insertions(+), 116437 deletions(-)
----------------------------------------------------------------------



[46/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/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
new file mode 100644
index 0000000..3125e30
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java
@@ -0,0 +1,30 @@
+/**
+ * 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/6594ac1c/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
new file mode 100644
index 0000000..66e9ee7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.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/6594ac1c/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
new file mode 100644
index 0000000..fba691d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java
@@ -0,0 +1,35 @@
+/**
+ * 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/6594ac1c/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
new file mode 100644
index 0000000..70191a9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/exception/NoSuchColumnException.java
@@ -0,0 +1,25 @@
+/*
+ * 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 + "'");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/exception/RangeOverflowException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/exception/RangeOverflowException.java b/tajo-core/src/main/java/org/apache/tajo/engine/exception/RangeOverflowException.java
new file mode 100644
index 0000000..409d6ed
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/exception/RangeOverflowException.java
@@ -0,0 +1,28 @@
+/**
+ * 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;
+
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleRange;
+
+public class RangeOverflowException extends RuntimeException {
+  public RangeOverflowException(TupleRange range, Tuple overflowValue, long inc) {
+    super("Overflow Error: tried to increase " + inc + " to " + overflowValue + ", but the range " + range);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/exception/VerifyException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/exception/VerifyException.java b/tajo-core/src/main/java/org/apache/tajo/engine/exception/VerifyException.java
new file mode 100644
index 0000000..beda787
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/exception/VerifyException.java
@@ -0,0 +1,27 @@
+/**
+ * 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;
+
+import org.apache.tajo.engine.planner.PlanningException;
+
+public class VerifyException extends PlanningException {
+  public VerifyException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/AggFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/AggFunction.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/AggFunction.java
new file mode 100644
index 0000000..a5a2583
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/AggFunction.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.Function;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Tuple;
+
+public abstract class AggFunction<T extends Datum> extends Function<T> {
+
+  public AggFunction(Column[] definedArgs) {
+    super(definedArgs);
+  }
+
+  public abstract FunctionContext newContext();
+
+  public abstract void eval(FunctionContext ctx, Tuple params);
+
+  public void merge(FunctionContext ctx, Tuple part) {
+    eval(ctx, part);
+  }
+
+  public abstract Datum getPartialResult(FunctionContext ctx);
+
+  public abstract DataType getPartialResultType();
+
+  public abstract T terminate(FunctionContext ctx);
+
+  @Override
+  public String toJson() {
+    return CatalogGsonHelper.toJson(this, AggFunction.class);
+  }
+
+  @Override
+  public CatalogProtos.FunctionType getFunctionType() {
+    return CatalogProtos.FunctionType.AGGREGATION;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionContext.java
new file mode 100644
index 0000000..076c8ec
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionContext.java
@@ -0,0 +1,22 @@
+/**
+ * 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.function;
+
+public interface FunctionContext {
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/GeneralFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/GeneralFunction.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/GeneralFunction.java
new file mode 100644
index 0000000..f7c31a1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/GeneralFunction.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.Function;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.eval.FunctionEval;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.Tuple;
+
+public abstract class GeneralFunction extends Function implements GsonObject {
+  public GeneralFunction(Column[] definedArgs) {
+    super(definedArgs);
+  }
+
+  /**
+   * This method gives hints to an actual function instance.
+   */
+  public void init(FunctionEval.ParamType [] paramTypes) {}
+
+  public abstract Datum eval(Tuple params);
+
+	public enum Type {
+	  AGG,
+	  GENERAL
+	}
+
+  @Override
+  public String toJson() {
+    return CatalogGsonHelper.toJson(this, GeneralFunction.class);
+  }
+
+  @Override
+  public CatalogProtos.FunctionType getFunctionType() {
+    return CatalogProtos.FunctionType.GENERAL;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/Description.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/Description.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/Description.java
new file mode 100644
index 0000000..bc6c7a2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/Description.java
@@ -0,0 +1,50 @@
+/**
+ * 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.function.annotation;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Description.
+ *
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface Description {
+
+  String description();
+
+  String detail() default "";
+
+  String example();
+
+  String functionName();
+
+  String[] synonyms() default {};
+
+  Type returnType();
+
+  ParamTypes[] paramTypes() default @ParamTypes;
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamOptionTypes.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamOptionTypes.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamOptionTypes.java
new file mode 100644
index 0000000..0ef7f4c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamOptionTypes.java
@@ -0,0 +1,29 @@
+/**
+ * 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.function.annotation;
+
+import org.apache.tajo.common.TajoDataTypes;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+@Retention(RetentionPolicy.RUNTIME)
+public @interface ParamOptionTypes {
+  TajoDataTypes.Type[] paramOptionTypes() default {};
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamTypes.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamTypes.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamTypes.java
new file mode 100644
index 0000000..ed2545c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/annotation/ParamTypes.java
@@ -0,0 +1,30 @@
+/**
+ * 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.function.annotation;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+@Retention(RetentionPolicy.RUNTIME)
+public @interface ParamTypes {
+  Type[] paramTypes() default { Type.NULL_TYPE };
+  ParamOptionTypes[] paramOptionTypes() default {};
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
new file mode 100644
index 0000000..df5cc80
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
@@ -0,0 +1,97 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.InternalTypes.AvgDoubleProto;
+
+@Description(
+  functionName = "avg",
+  description = "The mean of a set of numbers.",
+  example = "> SELECT avg(expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+)
+public class AvgDouble extends AggFunction {
+  public AvgDouble() {
+    super(new Column[] {
+        new Column("expr", 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;
+    ProtobufDatum datum = (ProtobufDatum) part.get(0);
+    AvgDoubleProto proto = (AvgDoubleProto) datum.get();
+    avgCtx.sum += proto.getSum();
+    avgCtx.count += proto.getCount();
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    AvgContext avgCtx = (AvgContext) ctx;
+    AvgDoubleProto.Builder builder = AvgDoubleProto.newBuilder();
+    builder.setSum(avgCtx.sum);
+    builder.setCount(avgCtx.count);
+    return new ProtobufDatum(builder.build());
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newDataType(Type.PROTOBUF, AvgDoubleProto.class.getName());
+  }
+
+  @Override
+  public Datum terminate(FunctionContext ctx) {
+    AvgContext avgCtx = (AvgContext) ctx;
+    return DatumFactory.createFloat8(avgCtx.sum / avgCtx.count);
+  }
+
+  protected class AvgContext implements FunctionContext {
+    double sum;
+    long count;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
new file mode 100644
index 0000000..4a30036
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
@@ -0,0 +1,46 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "avg",
+  description = "The mean of a set of numbers.",
+  example = "> SELECT avg(expr);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4})}
+)
+public class AvgFloat extends AvgDouble {
+
+  public AvgFloat() {
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    AvgContext avgCtx = (AvgContext) ctx;
+    avgCtx.sum += params.get(0).asFloat4();
+    avgCtx.count++;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
new file mode 100644
index 0000000..3c59f13
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
@@ -0,0 +1,47 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "avg",
+  description = "the mean of a set of numbers.",
+  example = "> SELECT avg(expr);",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})}
+)
+public class AvgInt extends AvgLong {
+
+  public AvgInt() {
+    super();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    AvgContext avgCtx = (AvgContext) ctx;
+    avgCtx.sum += params.get(0).asInt4();
+    avgCtx.count++;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
new file mode 100644
index 0000000..5bb5ff9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
@@ -0,0 +1,96 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.InternalTypes.AvgLongProto;
+
+@Description(
+  functionName = "avg",
+  description = "the mean of a set of numbers",
+  example = "> SELECT avg(expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
+)
+public class AvgLong extends AggFunction<Float8Datum> {
+
+  public AvgLong() {
+    super(new Column[] {
+        new Column("expr", 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;
+    ProtobufDatum datum = (ProtobufDatum) part.get(0);
+    AvgLongProto proto = (AvgLongProto) datum.get();
+    avgCtx.sum += proto.getSum();
+    avgCtx.count += proto.getCount();
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    AvgContext avgCtx = (AvgContext) ctx;
+    AvgLongProto.Builder builder = AvgLongProto.newBuilder();
+    builder.setSum(avgCtx.sum);
+    builder.setCount(avgCtx.count);
+    return new ProtobufDatum(builder.build());
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newDataType(Type.PROTOBUF, AvgLongProto.class.getName());
+  }
+
+  @Override
+  public Float8Datum terminate(FunctionContext ctx) {
+    AvgContext avgCtx = (AvgContext) ctx;
+    return DatumFactory.createFloat8((double) avgCtx.sum / avgCtx.count);
+  }
+
+  protected class AvgContext implements FunctionContext {
+    long sum;
+    long count;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Coalesce.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Coalesce.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Coalesce.java
new file mode 100644
index 0000000..98abb02
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Coalesce.java
@@ -0,0 +1,44 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+public abstract class Coalesce extends GeneralFunction {
+  public Coalesce(Column[] definedArgs) {
+    super(definedArgs);
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    int paramSize = params.size();
+    for (int i = 0; i < paramSize; i++) {
+      Datum datum = params.get(i);
+      if (datum instanceof NullDatum) {
+        continue;
+      }
+      return datum;
+    }
+    return NullDatum.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceDouble.java
new file mode 100644
index 0000000..3e94150
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceDouble.java
@@ -0,0 +1,43 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "coalesce",
+    description = "Returns the first of its arguments that is not null.",
+    detail = "Like a CASE expression, COALESCE only evaluates the arguments that are needed to determine the result; " +
+        "that is, arguments to the right of the first non-null argument are not evaluated",
+    example = "> SELECT coalesce(null, null, 10.0);\n"
+        + "10.0",
+    returnType = TajoDataTypes.Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8, TajoDataTypes.Type.FLOAT8_ARRAY})}
+)
+public class CoalesceDouble extends Coalesce {
+  public CoalesceDouble() {
+    super(new Column[] {
+        new Column("column", TajoDataTypes.Type.FLOAT8),
+        new Column("params", TajoDataTypes.Type.FLOAT8_ARRAY),
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceLong.java
new file mode 100644
index 0000000..5d55255
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceLong.java
@@ -0,0 +1,44 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "coalesce",
+    description = "Returns the first of its arguments that is not null.",
+    detail = "Like a CASE expression, COALESCE only evaluates the arguments that are needed to determine the result; " +
+        "that is, arguments to the right of the first non-null argument are not evaluated",
+    example = "> SELECT coalesce(null, null, 10);\n"
+        + "10",
+    returnType = TajoDataTypes.Type.INT8,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT8_ARRAY})}
+)
+public class CoalesceLong extends Coalesce {
+
+  public CoalesceLong() {
+    super(new Column[] {
+        new Column("column", TajoDataTypes.Type.INT8),
+        new Column("params", TajoDataTypes.Type.INT8_ARRAY),
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceString.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceString.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceString.java
new file mode 100644
index 0000000..50e4786
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CoalesceString.java
@@ -0,0 +1,44 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "coalesce",
+    description = "Returns the first of its arguments that is not null.",
+    detail = "Like a CASE expression, COALESCE only evaluates the arguments that are needed to determine the result; " +
+        "that is, arguments to the right of the first non-null argument are not evaluated",
+    example = "> SELECT coalesce(null, null, 'default');\n"
+        + "default",
+    returnType = TajoDataTypes.Type.TEXT,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT_ARRAY})}
+)
+public class CoalesceString extends Coalesce {
+
+  public CoalesceString() {
+    super(new Column[] {
+        new Column("column", TajoDataTypes.Type.TEXT),
+        new Column("params", TajoDataTypes.Type.TEXT_ARRAY),
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.java
new file mode 100644
index 0000000..aa1d941
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.java
@@ -0,0 +1,84 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "count",
+  description = "the total number of retrieved rows",
+  example = "> SELECT count(*);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {})}
+)
+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.newSimpleDataType(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/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.java
new file mode 100644
index 0000000..12d8892
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.java
@@ -0,0 +1,55 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Count(column) function
+ */
+@Description(
+  functionName = "count",
+  description = "The number of retrieved rows for "
+          + "which the supplied expressions are non-NULL",
+  example = "> SELECT count(expr);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.ANY})}
+)
+public final class CountValue extends CountRows {
+
+  public CountValue() {
+    super(new Column[] {
+        new Column("expr", 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/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValueDistinct.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValueDistinct.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValueDistinct.java
new file mode 100644
index 0000000..c0f5ec3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/CountValueDistinct.java
@@ -0,0 +1,90 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Count(distinct column) function
+ */
+@Description(
+  functionName = "count",
+  description = " The number of rows for "
+          + "which the supplied expressions are unique and non-NULL.",
+  example = "> SELECT count(expr);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.ANY})}
+)
+public final class CountValueDistinct extends CountRows {
+
+  public CountValueDistinct() {
+    super(new Column[] {
+        new Column("expr", Type.ANY)
+    });
+  }
+
+  @Override
+  public void eval(FunctionContext context, Tuple params) {
+  }
+
+  @Override
+  public void merge(FunctionContext context, Tuple part) {
+    CountDistinctValueContext distinctContext = (CountDistinctValueContext) context;
+    Datum value = part.get(0);
+    if ((distinctContext.latest == null || (!distinctContext.latest.equals(value)) && !(value instanceof NullDatum))) {
+      distinctContext.latest = value;
+      distinctContext.count++;
+    }
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createInt8(((CountDistinctValueContext) ctx).count);
+  }
+
+  @Override
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((CountDistinctValueContext) ctx).count);
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new CountDistinctValueContext();
+  }
+
+  private class CountDistinctValueContext implements FunctionContext {
+    long count = 0;
+    Datum latest = null;
+  }
+
+  @Override
+  public CatalogProtos.FunctionType getFunctionType() {
+    return CatalogProtos.FunctionType.DISTINCT_AGGREGATION;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Date.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Date.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Date.java
new file mode 100644
index 0000000..7666e89
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Date.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.builtin;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+@Description(
+  functionName = "date",
+  description = "Extracts the date part of the date or datetime expression expr.",
+  example = "> SELECT date(expr);",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})}
+)
+public class Date extends GeneralFunction {
+  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("expr", TEXT)});
+  }
+
+  @Override
+  public Int8Datum 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/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
new file mode 100644
index 0000000..5c9ceb1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
@@ -0,0 +1,78 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "max",
+  description = "the maximum value of expr",
+  example = "> SELECT max(expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+)
+public class MaxDouble extends AggFunction<Float8Datum> {
+
+  public MaxDouble() {
+    super(new Column[] {
+        new Column("expr", 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.newSimpleDataType(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/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
new file mode 100644
index 0000000..067026d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
@@ -0,0 +1,76 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "max",
+  description = "the maximum value of expr",
+  example = "> SELECT max(expr);",
+  returnType = Type.FLOAT4,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+)
+public class MaxFloat extends AggFunction<Datum> {
+  public MaxFloat() {
+    super(new Column[] {
+        new Column("expr", 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.newSimpleDataType(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/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
new file mode 100644
index 0000000..a329390
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "max",
+  description = "the maximum value of expr",
+  example = "> SELECT max(expr);",
+  returnType = Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+)
+public class MaxInt extends AggFunction<Datum> {
+
+  public MaxInt() {
+    super(new Column[] {
+        new Column("expr", 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.newSimpleDataType(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/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
new file mode 100644
index 0000000..71bab1a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "max",
+  description = "the maximum value of expr",
+  example = "> SELECT max(expr);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
+)
+public class MaxLong extends AggFunction<Int8Datum> {
+  public MaxLong() {
+    super(new Column[] {
+        new Column("expr", 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.newSimpleDataType(Type.INT8);
+  }
+
+  @Override
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((MaxContext) ctx).max);
+  }
+
+  private class MaxContext implements FunctionContext {
+    long max;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java
new file mode 100644
index 0000000..439462f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java
@@ -0,0 +1,81 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+    functionName = "max",
+    description = "the maximum value of expr",
+    example = "> SELECT max(expr);",
+    returnType = TajoDataTypes.Type.TEXT,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class MaxString  extends AggFunction<Datum> {
+
+  public MaxString() {
+    super(new Column[] {
+        new Column("expr", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new MaxContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    MaxContext maxCtx = (MaxContext) ctx;
+    if (maxCtx.max == null) {
+      maxCtx.max = params.get(0).asChars();
+    } else if (params.get(0).asChars().compareTo(maxCtx.max) > 0) {
+      maxCtx.max = params.get(0).asChars();
+    }
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createText(((MaxContext) ctx).max);
+  }
+
+  @Override
+  public TajoDataTypes.DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(TajoDataTypes.Type.TEXT);
+  }
+
+  @Override
+  public TextDatum terminate(FunctionContext ctx) {
+    return DatumFactory.createText(((MaxContext) ctx).max);
+  }
+
+  private class MaxContext implements FunctionContext {
+    String max;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
new file mode 100644
index 0000000..7b9e0ab
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "min",
+  description = "the minimum value of expr",
+  example = "> SELECT min(expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+)
+public class MinDouble extends AggFunction<Datum> {
+
+  public MinDouble() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new MinContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    MinContext minCtx = (MinContext) ctx;
+    minCtx.min = Math.min(minCtx.min, params.get(0).asFloat8());
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((MinContext) ctx).min);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+  }
+
+  @Override
+  public Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((MinContext) ctx).min);
+  }
+
+  private class MinContext implements FunctionContext {
+    double min = Double.MAX_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
new file mode 100644
index 0000000..95a97be
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
@@ -0,0 +1,78 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float4Datum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "min",
+  description = "the minimum value of expr",
+  example = "> SELECT min(expr);",
+  returnType = Type.FLOAT4,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+)
+public class MinFloat extends AggFunction<Float4Datum> {
+
+  public MinFloat() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT4)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new MinContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    MinContext minCtx = (MinContext) ctx;
+    minCtx.min = Math.min(minCtx.min, params.get(0).asFloat4());
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createFloat4(((MinContext) ctx).min);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.FLOAT4);
+  }
+
+  @Override
+  public Float4Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat4(((MinContext) ctx).min);
+  }
+
+  private class MinContext implements FunctionContext {
+    float min = Float.MAX_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
new file mode 100644
index 0000000..24b379a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
@@ -0,0 +1,77 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "min",
+  description = "the minimum value of expr",
+  example = "> SELECT min(expr);",
+  returnType = Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+)
+public class MinInt extends AggFunction<Datum> {
+
+  public MinInt() {
+    super(new Column[] {
+        new Column("expr", Type.INT8)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new MinContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    MinContext minCtx = (MinContext) ctx;
+    minCtx.min = Math.min(minCtx.min, params.get(0).asInt4());
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createInt4(((MinContext) ctx).min);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.INT4);
+  }
+
+  @Override
+  public Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt4(((MinContext) ctx).min);
+  }
+
+  private class MinContext implements FunctionContext {
+    int min = Integer.MAX_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
new file mode 100644
index 0000000..ad9ebfa
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
@@ -0,0 +1,78 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "min",
+  description = "the minimum value of expr",
+  example = "> SELECT min(expr);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
+)
+public class MinLong extends AggFunction<Datum> {
+
+  public MinLong() {
+    super(new Column[] {
+        new Column("expr", Type.INT8)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new MinContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    MinContext minCtx = (MinContext)ctx;
+    minCtx.min = Math.min(minCtx.min, params.get(0).asInt8());
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createInt8(((MinContext) ctx).min);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.INT8);
+  }
+
+  @Override
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((MinContext) ctx).min);
+  }
+
+  private class MinContext implements FunctionContext {
+    long min = Long.MAX_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
new file mode 100644
index 0000000..cbf214e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
@@ -0,0 +1,82 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "min",
+  description = "the minimum value of expr",
+  example = "> SELECT min(expr);",
+  returnType = Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {Type.TEXT})}
+)
+public class MinString extends AggFunction<Datum> {
+
+  public MinString() {
+    super(new Column[] {
+        new Column("expr", Type.TEXT)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new MinContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    MinContext minCtx = (MinContext) ctx;
+    if (minCtx.min == null) {
+      minCtx.min = params.get(0).asChars();
+    } else if (params.get(0).asChars().compareTo(minCtx.min) < 0) {
+      minCtx.min = params.get(0).asChars();
+    }
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createText(((MinContext) ctx).min);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.TEXT);
+  }
+
+  @Override
+  public TextDatum terminate(FunctionContext ctx) {
+    return DatumFactory.createText(((MinContext) ctx).min);
+  }
+
+  private class MinContext implements FunctionContext {
+    String min;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java
new file mode 100644
index 0000000..bdcf485
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import java.util.Random;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.INT4;
+
+@Description(
+  functionName = "random",
+  description = "A pseudorandom number",
+  example = "> SELECT random(10);\n"
+          + "4",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})}
+)
+public class RandomInt extends GeneralFunction {
+  private Random random;
+
+  public RandomInt() {
+    super(new Column[] {
+        new Column("n", INT4)
+    });
+    random = new Random(System.nanoTime());
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    return DatumFactory.createInt4(random.nextInt(params.get(0).asInt4()));
+  }
+
+}


[07/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
new file mode 100644
index 0000000..fa13011
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -0,0 +1,610 @@
+/**
+ * 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.function;
+
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+
+public class TestStringOperatorsAndFunctions extends ExprTestBase {
+
+  @Test
+  public void testConcatenateOnLiteral() throws IOException {
+    testSimpleEval("select ('abc' || 'def') col1 ", new String[]{"abcdef"});
+    testSimpleEval("select 'abc' || 'def' as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select 1 || 'def' as col1 ", new String[]{"1def"});
+    testSimpleEval("select 'abc' || 2 as col1 ", new String[]{"abc2"});
+  }
+
+  @Test
+  public void testConcatenateOnExpressions() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", INT4);
+    schema.addColumn("col3", FLOAT8);
+
+    testSimpleEval("select (1+3) || 2 as col1 ", new String[]{"42"});
+
+    testEval(schema, "table1", "abc,2,3.14", "select col1 || col2 || col3 from table1", new String[]{"abc23.14"});
+    testEval(schema, "table1", "abc,2,3.14", "select col1 || '---' || col3 from table1", new String[]{"abc---3.14"});
+  }
+
+  @Test
+  public void testFunctionCallIngoreCases() throws IOException {
+    testSimpleEval("select ltrim(' trim') ", new String[]{"trim"});
+    testSimpleEval("select LTRIM(' trim') ", new String[]{"trim"});
+    testSimpleEval("select lTRim(' trim') ", new String[]{"trim"});
+    testSimpleEval("select ltrIM(' trim') ", new String[]{"trim"});
+  }
+
+  @Test
+  public void testLTrim() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+
+    testSimpleEval("select ltrim(' trim') ", new String[]{"trim"});
+    testSimpleEval("select ltrim('xxtrim', 'xx') ", new String[]{"trim"});
+
+    testSimpleEval("select trim(leading 'xx' from 'xxtrim') ", new String[]{"trim"});
+    testSimpleEval("select trim(leading from '  trim') ", new String[]{"trim"});
+    testSimpleEval("select trim('  trim') ", new String[]{"trim"});
+
+    testEval(schema, "table1", "  trim,abc", "select ltrim(col1) from table1", new String[]{"trim"});
+    testEval(schema, "table1", "xxtrim,abc", "select ltrim(col1, 'xx') from table1", new String[]{"trim"});
+    testEval(schema, "table1", "xxtrim,abc", "select trim(leading 'xx' from col1) from table1", new String[]{"trim"});
+
+    testEval(schema, "table1", "  trim,  abc", "select ltrim(col1) || ltrim(col2) from table1",
+        new String[]{"trimabc"});
+  }
+
+  @Test
+  public void testRTrim() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+
+    testSimpleEval("select rtrim('trim ') ", new String[]{"trim"});
+    testSimpleEval("select rtrim('trimxx', 'xx') ", new String[]{"trim"});
+
+    testSimpleEval("select trim(trailing 'xx' from 'trimxx') ", new String[]{"trim"});
+    testSimpleEval("select trim(trailing from 'trim  ') ", new String[]{"trim"});
+    testSimpleEval("select trim('trim  ') ", new String[]{"trim"});
+
+    testEval(schema, "table1", "trim  ,abc", "select rtrim(col1) from table1", new String[]{"trim"});
+    testEval(schema, "table1", "trimxx,abc", "select rtrim(col1, 'xx') from table1", new String[]{"trim"});
+    testEval(schema, "table1", "trimxx,abc", "select trim(trailing 'xx' from col1) from table1", new String[]{"trim"});
+
+    testEval(schema, "table1", "trim  ,abc  ", "select rtrim(col1) || rtrim(col2) from table1",
+        new String[]{"trimabc"});
+  }
+
+  @Test
+  public void testTrim() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+
+    testSimpleEval("select trim(' trim ') ", new String[]{"trim"});
+    testSimpleEval("select btrim('xxtrimxx', 'xx') ", new String[]{"trim"});
+
+    testSimpleEval("select trim(both 'xx' from 'xxtrimxx') ", new String[]{"trim"});
+    testSimpleEval("select trim(both from '  trim  ') ", new String[]{"trim"});
+    testSimpleEval("select trim('  trim  ') ", new String[]{"trim"});
+
+    testEval(schema, "table1", "  trim  ,abc", "select trim(col1) from table1", new String[]{"trim"});
+    testEval(schema, "table1", "xxtrimxx,abc", "select trim(col1, 'xx') from table1", new String[]{"trim"});
+    testEval(schema, "table1", "xxtrimxx,abc", "select trim(both 'xx' from col1) from table1", new String[]{"trim"});
+
+    testEval(schema, "table1", "  trim  ,xxabcxx", "select trim(col1) || trim(col2,'xx') from table1",
+        new String[]{"trimabc"});
+  }
+
+  @Test
+  public void testRegexReplace() throws IOException {
+    testSimpleEval("select regexp_replace('abcdef','bc','--') as col1 ", new String[]{"a--def"});
+
+    // null test
+    testSimpleEval("select regexp_replace(null, 'bc', '--') as col1 ", new String[]{""});
+    testSimpleEval("select regexp_replace('abcdef', null, '--') as col1 ", new String[]{""});
+    testSimpleEval("select regexp_replace('abcdef','bc', null) as col1 ", new String[]{""});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+
+    // find matches and replace from column values
+    testEval(schema, "table1", "------,(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from table1",
+        new String[]{"ab--ab"});
+
+    // null test from a table
+    testEval(schema, "table1", ",(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from table1",
+        new String[]{""});
+    testEval(schema, "table1", "------,(^--|--$),", "select regexp_replace(col1, col2, col3) as str from table1",
+        new String[]{""});
+  }
+
+  @Test
+  public void testLeft() throws IOException {
+    testSimpleEval("select left('abcdef',1) as col1 ", new String[]{"a"});
+    testSimpleEval("select left('abcdef',2) as col1 ", new String[]{"ab"});
+    testSimpleEval("select left('abcdef',3) as col1 ", new String[]{"abc"});
+    testSimpleEval("select left('abcdef',4) as col1 ", new String[]{"abcd"});
+    testSimpleEval("select left('abcdef',5) as col1 ", new String[]{"abcde"});
+    testSimpleEval("select left('abcdef',6) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select left('abcdef',7) as col1 ", new String[]{"abcdef"});
+
+    testSimpleEval("select left('abcdef',-1) as col1 ", new String[]{"abcde"});
+    testSimpleEval("select left('abcdef',-2) as col1 ", new String[]{"abcd"});
+    testSimpleEval("select left('abcdef',-3) as col1 ", new String[]{"abc"});
+    testSimpleEval("select left('abcdef',-4) as col1 ", new String[]{"ab"});
+    testSimpleEval("select left('abcdef',-5) as col1 ", new String[]{"a"});
+    testSimpleEval("select left('abcdef',-6) as col1 ", new String[]{""});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", INT4);
+    schema.addColumn("col3", TEXT);
+
+    // for null tests
+    testEval(schema, "table1", ",1,ghi", "select left(col1,1) is null from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc,,ghi", "select left(col1,col2) is null from table1", new String[]{"t"});
+
+    testEval(schema, "table1", "abc,1,ghi", "select left(col1,1) || left(col3,3) from table1", new String[]{"aghi"});
+  }
+
+  @Test
+  public void testRight() throws IOException {
+    testSimpleEval("select right('abcdef',1) as col1 ", new String[]{"f"});
+    testSimpleEval("select right('abcdef',2) as col1 ", new String[]{"ef"});
+    testSimpleEval("select right('abcdef',3) as col1 ", new String[]{"def"});
+    testSimpleEval("select right('abcdef',4) as col1 ", new String[]{"cdef"});
+    testSimpleEval("select right('abcdef',5) as col1 ", new String[]{"bcdef"});
+    testSimpleEval("select right('abcdef',6) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select right('abcdef',7) as col1 ", new String[]{"abcdef"});
+
+    testSimpleEval("select right('abcdef',-1) as col1 ", new String[]{"bcdef"});
+    testSimpleEval("select right('abcdef',-2) as col1 ", new String[]{"cdef"});
+    testSimpleEval("select right('abcdef',-3) as col1 ", new String[]{"def"});
+    testSimpleEval("select right('abcdef',-4) as col1 ", new String[]{"ef"});
+    testSimpleEval("select right('abcdef',-5) as col1 ", new String[]{"f"});
+    testSimpleEval("select right('abcdef',-6) as col1 ", new String[]{""});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", INT4);
+    schema.addColumn("col3", TEXT);
+
+    // for null tests
+    testEval(schema, "table1", ",1,ghi", "select right(col1,1) is null from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc,,ghi", "select right(col1,col2) is null from table1", new String[]{"t"});
+
+    testEval(schema, "table1", "abc,1,ghi", "select right(col1,1) || right(col3,3) from table1", new String[]{"cghi"});
+  }
+
+  @Test
+  public void testReverse() throws IOException {
+    testSimpleEval("select reverse('abcdef') as col1 ", new String[]{"fedcba"});
+    testSimpleEval("select reverse('가') as col1 ", new String[]{"가"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "abc,efg,3.14", "select reverse(col1) || reverse(col2) from table1",
+        new String[]{"cbagfe"});
+  }
+
+  @Test
+  public void testRepeat() throws IOException {
+    testSimpleEval("select repeat('ab',4) as col1 ", new String[]{"abababab"});
+    testSimpleEval("select repeat('가',3) as col1 ", new String[]{"가가가"});
+    testSimpleEval("select repeat('a',2) as col1 ", new String[]{"aa"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "abc,efg,3.14", "select repeat(col1,2) from table1", new String[]{"abcabc"});
+  }
+
+
+  @Test
+  public void testUpper() throws IOException {
+    testSimpleEval("select upper('abcdef') as col1 ", new String[]{"ABCDEF"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "abc,efg,3.14", "select upper(col1), upper(col2) from table1",
+        new String[]{"ABC", "EFG"});
+    testEval(schema, "table1", "abc,efg,3.14", "select upper(col1) || upper(col2) from table1", new String[]{"ABCEFG"});
+  }
+
+  @Test
+  public void testLower() throws IOException {
+    testSimpleEval("select lower('ABCdEF') as col1 ", new String[]{"abcdef"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABC,DEF,3.14", "select lower(col1), lower(col2) from table1",
+        new String[]{"abc", "def"});
+    testEval(schema, "table1", "ABC,DEF,3.14", "select lower(col1) || lower(col2) from table1", new String[]{"abcdef"});
+  }
+
+  @Test
+  public void testCharLength() throws IOException {
+    testSimpleEval("select char_length('123456') as col1 ", new String[]{"6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABC,DEF,3.14", "select character_length(lower(col1) || lower(col2)) from table1",
+        new String[]{"6"});
+  }
+
+  @Test
+  public void testLength() throws IOException {
+    testSimpleEval("select length('123456') as col1 ", new String[]{"6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABC,DEF,3.14", "select length(lower(col1) || lower(col2)) from table1",
+        new String[]{"6"});
+  }
+
+  @Test
+  public void testMd5() throws IOException {
+    testSimpleEval("select md5('1') as col1 ", new String[]{"c4ca4238a0b923820dcc509a6f75849b"});
+    testSimpleEval("select md5('tajo') as col1 ", new String[]{"742721b3a79f71a9491681b8e8a7ce85"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "abc,efg,3.14", "select md5(col1) from table1",
+        new String[]{"900150983cd24fb0d6963f7d28e17f72"});
+  }
+
+  @Test
+  public void testDigest() throws IOException {
+    testSimpleEval("select digest('tajo', 'md2') as col1 ", new String[]{"bf523bce8241982f6bea9af0f7fd37ff"});
+    testSimpleEval("select digest('tajo', 'md5') as col1 ", new String[]{"742721b3a79f71a9491681b8e8a7ce85"});
+    testSimpleEval("select digest('tajo', 'sha1') as col1 ", new String[]{"02b0e20540b89f0b735092bbac8093eb2e3804cf"});
+    testSimpleEval("select digest('tajo', 'sha256') as col1 ", 
+      new String[]{"6440083be076869a9f9d0271a4bf298d98c8aa3ecb49df841895fbcddbb04a70"});
+    testSimpleEval("select digest('tajo', 'sha384') as col1 ", 
+      new String[]{"59ff99b0e274eb3d8e10f221b6b949bfc1244d2a1226c5c720062fb03d82272be633e4a0f2babccffbfdff7cc1cb06fb"});
+    testSimpleEval("select digest('tajo', 'sha512') as col1 ", 
+      new String[]{"ee8ba254d331ddfb1bca9aaf0c4b8c58aea5331928cbd20168c87828afb853b0c096af71ec69a23b669217a1dddd2934edaac33b1296fe526b22abd28a15c4b3"});
+    testSimpleEval("select digest('tajo', 'not') as col1 ", new String[]{""});
+  }
+
+  @Test
+  public void testHex() throws IOException {
+    testSimpleEval("select to_hex(1) as col1 ", new String[]{"1"});
+    testSimpleEval("select to_hex(10) as col1 ", new String[]{"a"});
+    testSimpleEval("select to_hex(1234) as col1 ", new String[]{"4d2"});
+    testSimpleEval("select to_hex(1023456788888888) as col1 ", new String[]{"3a2d41a583d38"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", ",abcdef,3.14", "select to_hex(10) from table1",
+        new String[]{"a"});
+  }
+
+  @Test
+  public void testBin() throws IOException {
+    testSimpleEval("select to_bin(1) as col1 ", new String[]{"1"});
+    testSimpleEval("select to_bin(10) as col1 ", new String[]{"1010"});
+    testSimpleEval("select to_bin(1234) as col1 ", new String[]{"10011010010"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", ",abcdef,3.14", "select to_bin(20) from table1",
+        new String[]{"10100"});
+  }
+
+  @Test
+  public void testOctetLength() throws IOException {
+    testSimpleEval("select octet_length('123456') as col1 ", new String[]{"6"});
+    testSimpleEval("select octet_length('1') as col1 ", new String[]{"1"});
+    testSimpleEval("select octet_length('가') as col1 ", new String[]{"3"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABC,DEF,3.14", "select octet_length(lower(col1) || lower(col2)) from table1",
+        new String[]{"6"});
+  }
+
+  @Test
+  public void testSplitPart() throws IOException {
+    testSimpleEval("select split_part('1386577650.123', '.', 1) as col1 ", new String[]{"1386577650"});
+    testSimpleEval("select split_part('1386577650.123', '.', 2) as col1 ", new String[]{"123"});
+    // If part is larger than the number of string portions, it will returns NULL.
+    testSimpleEval("select split_part('1386577650.123', '.', 3) is null", new String[]{"t"});
+
+    // null handling tests
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "t1", ",.,1", "select split_part(col1, col2, col3::int) is null from t1", new String[]{"t"});
+    testEval(schema, "t1", "1386577650.123,,1", "select split_part(col1, col2, col3::int) from t1",
+        new String[]{"1386577650.123"});
+    testEval(schema, "t1", "1386577650.123,.,", "select split_part(col1, col2, col3::int) is null from t1",
+        new String[]{"t"});
+  }
+
+  @Test
+  public void testSubstr() throws IOException {
+    testSimpleEval("select substr('abcdef', 3, 2) as col1 ", new String[]{"cd"});
+    testSimpleEval("select substr('abcdef', 3) as col1 ", new String[]{"cdef"});
+    testSimpleEval("select substr('abcdef', 1, 1) as col1 ", new String[]{"a"});
+    testSimpleEval("select substr('abcdef', 0, 1) as col1 ", new String[]{""});
+    testSimpleEval("select substr('abcdef', 0, 2) as col1 ", new String[]{"a"});
+    testSimpleEval("select substr('abcdef', 0) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select substr('abcdef', 1, 100) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select substr('abcdef', 0, 100) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select substr('일이삼사오', 2, 2) as col1 ", new String[]{"이삼"});
+    testSimpleEval("select substr('일이삼사오', 3) as col1 ", new String[]{"삼사오"});
+
+    testSimpleEval("select substr('abcdef', -1) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select substr('abcdef', -1, 100) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select substr('abcdef', -1, 3) as col1 ", new String[]{"a"});
+    testSimpleEval("select substr('abcdef', -1, 1) as col1 ", new String[]{""});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", ",abcdef,3.14", "select substr(lower(col2), 2, 3) from table1",
+        new String[]{"bcd"});
+  }
+  
+  @Test
+  public void testLocate() throws IOException {
+    // normal case
+    testSimpleEval("select locate('abcdef', 'a') as col1 ", new String[]{"1"});
+    testSimpleEval("select locate('abcdef', 'a', 0) as col1 ", new String[]{"1"});
+    testSimpleEval("select locate('abcdef', 'a', 1) as col1 ", new String[]{"1"});
+    testSimpleEval("select locate('abcdef', 'z') as col1 ", new String[]{"0"});
+    testSimpleEval("select locate('abcdef', 'z', 1) as col1 ", new String[]{"0"});
+    testSimpleEval("select locate('foobarbar', 'bar') as col1 ", new String[]{"4"});
+    testSimpleEval("select locate('foobarbar', 'bar', 0) as col1 ", new String[]{"4"});
+    testSimpleEval("select locate('foobarbar', 'bar', 1) as col1 ", new String[]{"4"});
+    testSimpleEval("select locate('foobarbar', 'bar', 5) as col1 ", new String[]{"7"});
+    testSimpleEval("select locate('foobarbar', 'bar', 9) as col1 ", new String[]{"0"});
+    testSimpleEval("select locate('가나다라마라마', '라마') as col1 ", new String[]{"4"});
+    testSimpleEval("select locate('가나다라마라마', '라마', 5) as col1 ", new String[]{"6"});
+    // empty string
+    testSimpleEval("select locate('abcdef', '') as col1 ", new String[]{"1"});
+    testSimpleEval("select locate('abcdef', '', 2) as col1 ", new String[]{"2"});
+    // pos = last index of string (expected value(6) is tested on mysql)
+    testSimpleEval("select locate('abcdef', '', 6) as col1 ", new String[]{"6"});
+    // pos = last index + 1 (expected value(7) is tested on mysql)
+    testSimpleEval("select locate('abcdef', '', 7) as col1 ", new String[]{"7"});
+    // pos = greater then last index + 1 (expected value(0) is tested on mysql)
+    testSimpleEval("select locate('abcdef', '', 8) as col1 ", new String[]{"0"});
+    // pos = greater then last index + 1 (expected value(0) is tested on mysql)
+    testSimpleEval("select locate('abcdef', '', 9) as col1 ", new String[]{"0"});
+    testSimpleEval("select locate('가나다라', '', 2) as col1 ", new String[]{"2"});
+    testSimpleEval("select locate('가나다라', '', 4) as col1 ", new String[]{"4"});
+    testSimpleEval("select locate('가나다라', '', 5) as col1 ", new String[]{"5"});
+    testSimpleEval("select locate('가나다라', '', 6) as col1 ", new String[]{"0"});
+    
+    // negative pos    
+    testSimpleEval("select locate('abcdef', 'a', -1) as col1 ", new String[]{"0"});
+    testSimpleEval("select locate('abcdef', 'a', -5) as col1 ", new String[]{"0"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd') from table1", new String[]{"3"});
+    testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd', 1) from table1", new String[]{"3"});
+    testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd', 4) from table1", new String[]{"0"});
+    testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'xy') from table1", new String[]{"0"});
+    // null string
+    testEval(schema, "table1", ",abcdef,3.14", "select locate(col1, 'cd') is null from table1", new String[]{"t"});
+    // nul substring
+    testEval(schema, "table1", ",abcdef,3.14", "select locate('cd', col1) is null from table1", new String[]{"t"});
+  }
+
+  @Test
+  public void testBitLength() throws IOException {
+    testSimpleEval("select bit_length('123456') as col1 ", new String[]{"48"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABC,DEF,3.14", "select bit_length(lower(col1) || lower(col2)) from table1",
+        new String[]{"48"});
+  }
+
+  @Test
+  public void testStrpos() throws IOException {
+    testSimpleEval("select strpos('tajo','jo') as col1 ", new String[]{"3"});
+    testSimpleEval("select strpos('tajo','') as col1 ", new String[]{"1"});
+    testSimpleEval("select strpos('tajo','abcdef') as col1 ", new String[]{"0"});
+    testSimpleEval("select strpos('일이삼사오육','삼사') as col1 ", new String[]{"3"});
+    testSimpleEval("select strpos('일이삼사오육','일이삼') as col1 ", new String[]{"1"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strpos(lower(col1) || lower(col2), 'fh') from table1",
+        new String[]{"6"});
+  }
+
+  @Test
+  public void testStrposb() throws IOException {
+    testSimpleEval("select strposb('tajo','jo') as col1 ", new String[]{"3"});
+    testSimpleEval("select strposb('tajo','') as col1 ", new String[]{"1"});
+    testSimpleEval("select strposb('tajo','abcdef') as col1 ", new String[]{"0"});
+    testSimpleEval("select strposb('일이삼사오육','삼사') as col1 ", new String[]{"7"});    //utf8 1 korean word = 3 chars
+    testSimpleEval("select strposb('일이삼사오육','삼사일') as col1 ", new String[]{"0"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strposb(lower(col1) || lower(col2), 'fh') from table1",
+        new String[]{"6"});
+  }
+
+  @Test
+  public void testInitcap() throws IOException {
+    testSimpleEval("select initcap('hi bro') ", new String[]{"Hi Bro"});
+    testSimpleEval("select initcap('HI BRO') ", new String[]{"Hi Bro"});
+  }
+
+  @Test
+  public void testAscii() throws IOException {
+    testSimpleEval("select ascii('abc') as col1 ", new String[]{"97"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    testEval(schema, "table1", "abc", "select ascii(col1) from table1",
+            new String[]{"97"});
+    testEval(schema, "table1", "12", "select ascii(col1) from table1",
+            new String[]{"49"});
+
+  }
+
+  @Test
+  public void testChr() throws IOException {
+    testSimpleEval("select chr(48) as col1 ", new String[]{"0"});
+    testSimpleEval("select chr(49) as col1 ", new String[]{"1"});
+    testSimpleEval("select chr(50) as col1 ", new String[]{"2"});
+    testSimpleEval("select chr(64) as col1 ", new String[]{"@"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", INT4);
+    testEval(schema, "table1", "65", "select chr(col1) from table1", new String[]{"A"});
+    testEval(schema, "table1", "66", "select chr(col1) from table1", new String[]{"B"});
+    testEval(schema, "table1", "52512", "select chr(col1) from table1", new String[]{"촠"});
+  }
+
+  @Test
+  public void testLpad() throws IOException {
+    testSimpleEval("select lpad('hi', 5, 'xy') ", new String[]{"xyxhi"});
+    testSimpleEval("select LPAD('hello', 7, 'xy') ", new String[]{"xyhello"});
+    testSimpleEval("select LPAD('hello', 3, 'xy') ", new String[]{"hel"});
+    testSimpleEval("select lPAD('hello', 7) ", new String[]{"  hello"});
+    testSimpleEval("select lPAD('가나다라', 3) ", new String[]{"가나다"});
+
+  }
+
+  @Test
+  public void testRpad() throws IOException {
+    testSimpleEval("select rpad('hi', 5, 'xy') ", new String[]{"hixyx"});
+    testSimpleEval("select RPAD('hello', 7, 'xy') ", new String[]{"helloxy"});
+    testSimpleEval("select RPAD('hello', 3, 'xy') ", new String[]{"hel"});
+    testSimpleEval("select rPAD('hello', 7) ", new String[]{"hello  "});
+    testSimpleEval("select rPAD('가나다라', 3) ", new String[]{"가나다"});
+
+  }
+
+  @Test
+  public void testQuote_ident() throws IOException {
+    testSimpleEval("select quote_ident('Foo bar') ", new String[]{"\"Foo bar\""});
+    testSimpleEval("select QUOTE_IDENT('Tajo Function') ", new String[]{"\"Tajo Function\""});
+  }
+
+  @Test
+  public void testEncode() throws IOException {
+    testSimpleEval("select encode('Hello\nworld', 'base64') ", new String[]{"SGVsbG8Kd29ybGQ="});
+    testSimpleEval("select encode('Hello\nworld', 'hex') ",
+        new String[]{"0x480x650x6c0x6c0x6f0x0a0x770x6f0x720x6c0x64"});
+    testSimpleEval("select encode('한글', 'base64') ", new String[]{"7ZWc6riA"});
+    testSimpleEval("select encode('한글', 'hex') ", new String[]{"0xd55c0xae00"});
+    testSimpleEval("select encode('한글\n테스트\t입니다.', 'hex') ",
+        new String[]{"0xd55c0xae000x0a0xd14c0xc2a40xd2b80x090xc7850xb2c80xb2e40x2e"});
+  }
+
+
+  @Test
+  public void testDecode() throws IOException {
+    testSimpleEval("select decode('SGVsbG8Kd29ybGQ=', 'base64') ",
+        new String[]{StringEscapeUtils.escapeJava("Hello\nworld")});
+    testSimpleEval("select decode('0x480x650x6c0x6c0x6f0x0a0x770x6f0x720x6c0x64', 'hex') ",
+        new String[]{StringEscapeUtils.escapeJava("Hello\nworld")});
+    testSimpleEval("select decode('7ZWc6riA', 'base64') ", new String[]{StringEscapeUtils.escapeJava("한글")});
+    testSimpleEval("select decode('0xd55c0xae00', 'hex') ", new String[]{StringEscapeUtils.escapeJava("한글")});
+    testSimpleEval("select decode('0xd55c0xae000x0a0xd14c0xc2a40xd2b80x090xc7850xb2c80xb2e40x2e', 'hex') ",
+        new String[]{StringEscapeUtils.escapeJava("한글\n" + "테스트\t입니다.")});
+  }
+
+  @Test
+  public void testFindInSet() throws IOException {
+    // abnormal cases
+    testSimpleEval("select find_in_set('cr','crt') as col1 ", new String[]{"0"}); // there is no matched string
+    testSimpleEval("select find_in_set('c,r','crt,c,cr,c,def') as col1 ", new String[]{"0"}); // abnormal parameter
+
+    // normal cases
+    testSimpleEval("select find_in_set('crt','crt,c,cr,d,def') as col1 ", new String[]{"1"});
+    testSimpleEval("select find_in_set('c','crt,c,cr,d,def') as col1 ", new String[]{"2"});
+    testSimpleEval("select find_in_set('def','crt,c,cr,d,def') as col1 ", new String[]{"5"});
+    // unicode test
+    testSimpleEval("select find_in_set('딸기','사과,배,옥수수,감자,딸기,수박') as col1 ", new String[]{"5"});
+
+    // null test
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    testEval(schema, "table1", "|crt,c,cr,c,def", "select find_in_set(col1, col2) is null from table1",
+        new String[]{"t"}, '|', true);
+    testEval(schema, "table1", "cr|", "select find_in_set(col1, col2) is null from table1",
+        new String[]{"t"}, '|', true);
+  }
+
+  @Test
+  public void testConcat() throws IOException {
+    testSimpleEval("select concat('333', '22') ", new String[]{"33322"});
+    testSimpleEval("select concat('한글', '22') ", new String[]{"한글22"});
+  }
+
+  @Test
+  public void testConcat_ws() throws IOException {
+    testSimpleEval("select concat_ws(',', '333', '22') ", new String[]{"333,22"});
+    testSimpleEval("select concat_ws(',', '한글', '22') ", new String[]{"한글,22"});
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java b/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java
new file mode 100644
index 0000000..ef21dc3
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java
@@ -0,0 +1,271 @@
+/**
+ * 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.parser;
+
+import com.google.common.base.Preconditions;
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.engine.parser.SQLParser.SqlContext;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.util.FileUtil;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveQLAnalyzer {
+  private static final Log LOG = LogFactory.getLog(TestHiveQLAnalyzer.class.getName());
+  protected static final String BASE_PATH = "src/test/resources/queries/default/";
+
+  public static Expr parseQuery(String sql) {
+    ANTLRInputStream input = new ANTLRInputStream(sql);
+    SQLLexer lexer = new SQLLexer(input);
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    SQLParser parser = new SQLParser(tokens);
+    parser.setBuildParseTree(true);
+    SQLAnalyzer visitor = new SQLAnalyzer();
+    SqlContext context = parser.sql();
+    return visitor.visitSql(context);
+  }
+
+  public static Expr parseHiveQL(String sql) {
+    HiveQLAnalyzer converter = new HiveQLAnalyzer();
+    return converter.parse(sql);
+  }
+
+  public static String getMethodName(int depth) {
+    final StackTraceElement[] ste = Thread.currentThread().getStackTrace();
+    return ste[depth].getMethodName();
+  }
+
+  public static void compareJsonResult(String sqlPath) throws IOException {
+      Preconditions.checkNotNull(sqlPath);
+      compareJsonResult(sqlPath, sqlPath);
+  }
+
+  public static void compareJsonResult(String sqlPath, String hiveqlPath) throws IOException {
+    Preconditions.checkNotNull(sqlPath, hiveqlPath);
+    String sql = FileUtil.readTextFile(new File(BASE_PATH + sqlPath));
+    String hiveQL = FileUtil.readTextFile(new File(BASE_PATH + hiveqlPath));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(hiveQL);
+    assertEquals(expr.toJson(), hiveExpr.toJson());
+  }
+
+  @Test
+  public void testSelect1() throws IOException {
+    compareJsonResult("select_1.sql");
+  }
+
+  @Test
+  public void testSelect3() throws IOException {
+    compareJsonResult("select_3.sql");
+  }
+
+  @Test
+  public void testSelect4() throws IOException {
+    compareJsonResult("select_4.sql");
+  }
+
+  @Test
+  public void testSelect5() throws IOException {
+    compareJsonResult("select_5.sql");
+  }
+
+  @Test
+  public void testSelect7() throws IOException {
+    compareJsonResult("select_7.sql");
+  }
+
+  @Test
+  public void testSelect8() throws IOException {
+    compareJsonResult("select_8.sql");
+  }
+
+  @Test
+  public void testSelect9() throws IOException {
+    compareJsonResult("select_9.sql", "select_9.hiveql");
+  }
+
+  @Test
+  public void testSelect10() throws IOException {
+    compareJsonResult("select_10.sql", "select_10.hiveql");
+  }
+
+  //TODO: support beween condition
+  //@Test
+//  public void testSelect11() throws IOException {
+//    compareJsonResult("select_11.sql", "select_11.hiveql");
+//  }
+
+  @Test
+  public void testSelect12() throws IOException {
+    compareJsonResult("select_12.hiveql");
+  }
+
+  @Test
+  public void testSelect13() throws IOException {
+    compareJsonResult("select_13.sql", "select_13.hiveql");
+  }
+
+  @Test
+  public void testSelect14() throws IOException {
+    compareJsonResult("select_14.sql");
+  }
+
+  @Test
+  public void testSelect15() throws IOException {
+    compareJsonResult("select_15.sql", "select_15.hiveql");
+  }
+
+  @Test
+  public void testAsterisk1() throws IOException {
+    compareJsonResult("asterisk_1.sql");
+  }
+
+  @Test
+  public void testAsterisk2() throws IOException {
+    compareJsonResult("asterisk_2.sql");
+  }
+
+  @Test
+  public void testAsterisk3() throws IOException {
+    compareJsonResult("asterisk_3.sql");
+  }
+
+  @Test
+  public void testAsterisk4() throws IOException {
+    compareJsonResult("asterisk_4.sql");
+  }
+
+  @Test
+  public void testGroupby1() throws IOException {
+    compareJsonResult("groupby_1.sql");
+  }
+
+  @Test
+  public void testGroupby2() throws IOException {
+    compareJsonResult("groupby_2.sql");
+  }
+
+  @Test
+  public void testGroupby3() throws IOException {
+    compareJsonResult("groupby_3.sql");
+  }
+
+  @Test
+  public void testGroupby4() throws IOException {
+    compareJsonResult("groupby_4.sql");
+  }
+
+  @Test
+  public void testGroupby5() throws IOException {
+    compareJsonResult("groupby_5.sql");
+  }
+
+  @Test
+  public void testJoin2() throws IOException {
+    compareJsonResult("join_2.sql");
+  }
+
+  @Test
+  public void testJoin5() throws IOException {
+    compareJsonResult("join_5.sql");
+  }
+
+  @Test
+  public void testJoin6() throws IOException {
+    compareJsonResult("join_6.sql");
+  }
+
+  @Test
+  public void testJoin7() throws IOException {
+    compareJsonResult("join_7.sql");
+  }
+
+    //TODO: support complex join conditions
+    //@Test
+//  public void testJoin9() throws IOException {
+//    compareJsonResult("join_9.sql");
+//  }
+
+  @Test
+  public void testJoin12() throws IOException {
+    compareJsonResult("join_12.sql");
+  }
+
+  @Test
+  public void testJoin13() throws IOException {
+    compareJsonResult("join_13.sql");
+  }
+
+  @Test
+  public void testJoin14() throws IOException {
+    compareJsonResult("join_14.sql");
+  }
+
+  @Test
+  public void testJoin15() throws IOException {
+    compareJsonResult("join_15.sql", "join_15.hiveql");
+  }
+
+  @Test
+  public void testUnion1() throws IOException {
+    compareJsonResult("union_1.hiveql");
+  }
+
+  @Test
+  public void testInsert1() throws IOException {
+    compareJsonResult("insert_into_select_1.sql");
+  }
+
+  @Test
+  public void testInsert2() throws IOException {
+    compareJsonResult("insert_overwrite_into_select_2.sql", "insert_overwrite_into_select_2.hiveql");
+  }
+
+  @Test
+  public void testCreate1() throws IOException {
+    compareJsonResult("create_table_1.sql", "create_table_1.hiveql");
+  }
+
+  @Test
+  public void testCreate2() throws IOException {
+    compareJsonResult("create_table_2.sql", "create_table_2.hiveql");
+  }
+
+  @Test
+  public void testCreate11() throws IOException {
+    compareJsonResult("create_table_11.sql", "create_table_11.hiveql");
+  }
+
+  @Test
+  public void testCreate12() throws IOException {
+    compareJsonResult("create_table_12.sql", "create_table_12.hiveql");
+  }
+
+  @Test
+  public void testDrop() throws IOException {
+    compareJsonResult("drop_table.sql");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java b/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
new file mode 100644
index 0000000..2010502
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
@@ -0,0 +1,473 @@
+/**
+ * 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.parser;
+
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.apache.tajo.algebra.CreateTable;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.algebra.LiteralValue;
+import org.apache.tajo.algebra.OpType;
+import org.apache.tajo.engine.parser.SQLParser.SqlContext;
+import org.apache.tajo.util.FileUtil;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This unit tests uses a number of query files located in tajo/tajo-core/src/test/resources/queries.
+ * So, you must set tajo/tajo-core/ as the working directory.
+ */
+public class TestSQLAnalyzer {
+
+  public static Expr parseQuery(String sql) {
+    ANTLRInputStream input = new ANTLRInputStream(sql);
+    SQLLexer lexer = new SQLLexer(input);
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    SQLParser parser = new SQLParser(tokens);
+    parser.setBuildParseTree(true);
+    SQLAnalyzer visitor = new SQLAnalyzer();
+    SqlContext context = parser.sql();
+    return visitor.visitSql(context);
+  }
+
+
+  @Test
+  public void testSelect1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSelect2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSelect3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_3.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSelect4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_4.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSelect5() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_5.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testAsterisk1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testAsterisk2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testAsterisk3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_3.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testAsterisk4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_4.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testGroupby1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/groupby_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_3.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_4.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin5() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_5.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin6() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_6.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin7() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_7.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin8() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_8.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin9() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_9.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin10() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_10.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testJoin11() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_11.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSet1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/set_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSet2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/set_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSet3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/set_3.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testSet4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/set_4.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testDropTable() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/drop_table.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_3.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_4.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable5() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_5.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable6() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_6.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable7() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_7.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable8() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_8.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable9() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_9.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTable10() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_10.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testCreateTablePartitionByHash1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_partition_by_hash_1.sql"));
+    Expr expr = parseQuery(sql);
+    assertEquals(OpType.CreateTable, expr.getType());
+    CreateTable createTable = (CreateTable) expr;
+    assertTrue(createTable.hasPartition());
+    assertEquals(CreateTable.PartitionType.HASH, createTable.getPartitionMethod().getPartitionType());
+    CreateTable.HashPartition hashPartition = createTable.getPartitionMethod();
+    assertEquals("col1", hashPartition.getColumns()[0].getCanonicalName());
+    assertTrue(hashPartition.hasQuantifier());
+  }
+
+  @Test
+  public void testCreateTablePartitionByHash2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_partition_by_hash_2.sql"));
+    Expr expr = parseQuery(sql);
+    assertEquals(OpType.CreateTable, expr.getType());
+    CreateTable createTable = (CreateTable) expr;
+    assertTrue(createTable.hasPartition());
+    assertEquals(CreateTable.PartitionType.HASH, createTable.getPartitionMethod().getPartitionType());
+    CreateTable.HashPartition hashPartition = createTable.getPartitionMethod();
+    assertEquals("col1", hashPartition.getColumns()[0].getCanonicalName());
+    assertTrue(hashPartition.hasSpecifiers());
+    assertEquals(3, hashPartition.getSpecifiers().size());
+  }
+
+  @Test
+  public void testCreateTablePartitionByRange() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_partition_by_range.sql"));
+    Expr expr = parseQuery(sql);
+    assertEquals(OpType.CreateTable, expr.getType());
+    CreateTable createTable = (CreateTable) expr;
+    assertTrue(createTable.hasPartition());
+    assertEquals(CreateTable.PartitionType.RANGE, createTable.getPartitionMethod().getPartitionType());
+    CreateTable.RangePartition rangePartition = createTable.getPartitionMethod();
+    assertEquals("col1", rangePartition.getColumns()[0].getCanonicalName());
+    assertEquals(3, rangePartition.getSpecifiers().size());
+  }
+
+  @Test
+  public void testCreateTablePartitionByList() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_partition_by_list.sql"));
+    Expr expr = parseQuery(sql);
+    assertEquals(OpType.CreateTable, expr.getType());
+    CreateTable createTable = (CreateTable) expr;
+    assertTrue(createTable.hasPartition());
+    assertEquals(CreateTable.PartitionType.LIST, createTable.getPartitionMethod().getPartitionType());
+    CreateTable.ListPartition listPartition = createTable.getPartitionMethod();
+    assertEquals("col1", listPartition.getColumns()[0].getCanonicalName());
+    assertEquals(2, listPartition.getSpecifiers().size());
+    Iterator<CreateTable.ListPartitionSpecifier> iterator = listPartition.getSpecifiers().iterator();
+    CreateTable.ListPartitionSpecifier specifier = iterator.next();
+    LiteralValue value1 = (LiteralValue) specifier.getValueList().getValues()[0];
+    LiteralValue value2 = (LiteralValue) specifier.getValueList().getValues()[1];
+    assertEquals("Seoul", value1.getValue());
+    assertEquals("서울", value2.getValue());
+
+    specifier = iterator.next();
+    value1 = (LiteralValue) specifier.getValueList().getValues()[0];
+    value2 = (LiteralValue) specifier.getValueList().getValues()[1];
+    assertEquals("Busan", value1.getValue());
+    assertEquals("부산", value2.getValue());
+  }
+
+  @Test
+  public void testCreateTablePartitionByColumn() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_partition_by_column.sql"));
+    Expr expr = parseQuery(sql);
+    assertEquals(OpType.CreateTable, expr.getType());
+    CreateTable createTable = (CreateTable) expr;
+    assertTrue(createTable.hasPartition());
+    assertEquals(CreateTable.PartitionType.COLUMN, createTable.getPartitionMethod().getPartitionType());
+    CreateTable.ColumnPartition columnPartition = createTable.getPartitionMethod();
+    assertEquals(3, columnPartition.getColumns().length);
+    assertEquals("col3", columnPartition.getColumns()[0].getColumnName());
+    assertEquals("col4", columnPartition.getColumns()[1].getColumnName());
+    assertEquals("col5", columnPartition.getColumns()[2].getColumnName());
+  }
+
+  @Test
+  public void testTableSubQuery1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/table_subquery1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testTableSubQuery2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/table_subquery2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInSubquery1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/in_subquery_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInSubquery2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/in_subquery_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testExistsPredicate1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/exists_predicate_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testExistsPredicate2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/exists_predicate_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInsertIntoTable() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_into_select_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInsertIntoLocation() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_into_select_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInsertIntoTable2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_into_select_3.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInsertOverwriteIntoTable() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_overwrite_into_select_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInsertOverwriteIntoLocation() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_overwrite_into_select_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testInsertOverwriteIntoTable2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_overwrite_into_select_3.sql"));
+    parseQuery(sql);
+  }
+
+  static String[] exprs = {
+      "1 + 2", // 0
+      "3 - 4", // 1
+      "5 * 6", // 2
+      "7 / 8", // 3
+      "10 % 2", // 4
+      "1 * 2 > 3 / 4", // 5
+      "1 * 2 < 3 / 4", // 6
+      "1 * 2 = 3 / 4", // 7
+      "1 * 2 != 3 / 4", // 8
+      "1 * 2 <> 3 / 4", // 9
+      "gender in ('male', 'female')", // 10
+      "gender not in ('male', 'female')", // 11
+      "score > 90 and age < 20", // 12
+      "score > 90 and age < 20 and name != 'hyunsik'", // 13
+      "score > 90 or age < 20", // 14
+      "score > 90 or age < 20 and name != 'hyunsik'", // 15
+      "((a+3 > 1) or 1=1) and (3 != (abc + 4) and type in (3,4))", // 16
+      "3", // 17
+      "1.2", // 18
+      "sum(age)", // 19
+      "now()", // 20
+      "not (90 > 100)", // 21
+      "type like '%top'", // 22
+      "type not like 'top%'", // 23
+      "col = 'value'", // 24
+      "col is null", // 25
+      "col is not null", // 26
+      "col = null", // 27
+      "col != null", // 38
+  };
+
+  public static Expr parseExpr(String sql) {
+    ANTLRInputStream input = new ANTLRInputStream(sql);
+    SQLLexer lexer = new SQLLexer(input);
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    SQLParser parser = new SQLParser(tokens);
+    parser.setBuildParseTree(true);
+    SQLAnalyzer visitor = new SQLAnalyzer();
+    SQLParser.Value_expressionContext context = parser.value_expression();
+    return visitor.visitValue_expression(context);
+  }
+
+  @Test
+  public void testExprs() {
+    for (int i = 0; i < exprs.length; i++) {
+      parseExpr(exprs[i]);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalNode.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalNode.java
new file mode 100644
index 0000000..3fe75f0
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalNode.java
@@ -0,0 +1,74 @@
+/**
+ * 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.planner;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestLogicalNode {
+  public static final void testCloneLogicalNode(LogicalNode n1) throws CloneNotSupportedException {
+    LogicalNode copy = (LogicalNode) n1.clone();
+    assertTrue(n1.deepEquals(copy));
+  }
+
+  @Test
+  public void testEquals() {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("age", Type.INT2);
+    GroupbyNode groupbyNode = new GroupbyNode(0);
+    groupbyNode.setGroupingColumns(new Column[]{schema.getColumn(1), schema.getColumn(2)});
+    ScanNode scanNode = new ScanNode(0);
+    scanNode.init(CatalogUtil.newTableDesc("in", schema, CatalogUtil.newTableMeta(StoreType.CSV), new Path("in")));
+
+    GroupbyNode groupbyNode2 = new GroupbyNode(0);
+    groupbyNode2.setGroupingColumns(new Column[]{schema.getColumn(1), schema.getColumn(2)});
+    JoinNode joinNode = new JoinNode(0);
+    ScanNode scanNode2 = new ScanNode(0);
+    scanNode2.init(CatalogUtil.newTableDesc("in2", schema, CatalogUtil.newTableMeta(StoreType.CSV), new Path("in2")));
+
+    groupbyNode.setChild(scanNode);
+    groupbyNode2.setChild(joinNode);
+    joinNode.setLeftChild(scanNode);
+    joinNode.setRightChild(scanNode2);
+
+    assertTrue(groupbyNode.equals(groupbyNode2));
+    assertFalse(groupbyNode.deepEquals(groupbyNode2));
+
+    ScanNode scanNode3 = new ScanNode(0);
+    scanNode3.init(CatalogUtil.newTableDesc("in", schema, CatalogUtil.newTableMeta(StoreType.CSV), new Path("in")));
+    groupbyNode2.setChild(scanNode3);
+
+    assertTrue(groupbyNode.equals(groupbyNode2));
+    assertTrue(groupbyNode.deepEquals(groupbyNode2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
new file mode 100644
index 0000000..5acd512
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
@@ -0,0 +1,263 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.builtin.SumInt;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.*;
+
+public class TestLogicalOptimizer {
+
+  private static TajoTestingCluster util;
+  private static CatalogService catalog;
+  private static SQLAnalyzer sqlAnalyzer;
+  private static LogicalPlanner planner;
+  private static LogicalOptimizer optimizer;
+  private static Session session = LocalTajoTestingUtility.createDummySession();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.startCatalogCluster();
+    catalog = util.getMiniCatalogCluster().getCatalog();
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse");
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+      catalog.createFunction(funcDesc);
+    }
+    
+    Schema schema = new Schema();
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("deptname", Type.TEXT);
+    schema2.addColumn("manager", Type.TEXT);
+
+    Schema schema3 = new Schema();
+    schema3.addColumn("deptname", Type.TEXT);
+    schema3.addColumn("score", Type.INT4);
+    schema3.addColumn("phone", Type.INT4);
+
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    TableDesc people = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta,
+        CommonTestingUtil.getTestDir());
+    catalog.createTable(people);
+
+    TableDesc student =
+        new TableDesc(
+            CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dept"), schema2, StoreType.CSV, new Options(),
+            CommonTestingUtil.getTestDir());
+    catalog.createTable(student);
+
+    TableDesc score =
+        new TableDesc(
+            CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "score"), schema3, StoreType.CSV, new Options(),
+            CommonTestingUtil.getTestDir());
+    catalog.createTable(score);
+
+    FunctionDesc funcDesc = new FunctionDesc("sumtest", SumInt.class, FunctionType.GENERAL,
+        CatalogUtil.newSimpleDataType(Type.INT4),
+        CatalogUtil.newSimpleDataTypeArray(Type.INT4));
+
+    catalog.createFunction(funcDesc);
+    sqlAnalyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+    optimizer = new LogicalOptimizer(util.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+  
+  static String[] QUERIES = {
+    "select name, manager from employee as e, dept as dp where e.deptName = dp.deptName", // 0
+    "select name, empId, deptName from employee where empId > 500", // 1
+    "select name from employee where empId = 100", // 2
+    "select name, max(empId) as final from employee where empId > 50 group by name", // 3
+    "select name, score from employee natural join score", // 4
+    "select name, score from employee join score on employee.deptName = score.deptName", // 5
+  };
+  
+  @Test
+  public final void testProjectionPushWithNaturalJoin() throws PlanningException, CloneNotSupportedException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(QUERIES[4]);
+    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalNode plan = newPlan.getRootBlock().getRoot();
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    TestLogicalNode.testCloneLogicalNode(root);
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+    assertEquals(NodeType.JOIN, projNode.getChild().getType());
+    JoinNode joinNode = projNode.getChild();
+    assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+    
+    LogicalNode optimized = optimizer.optimize(newPlan);
+
+    assertEquals(NodeType.ROOT, optimized.getType());
+    root = (LogicalRootNode) optimized;
+    TestLogicalNode.testCloneLogicalNode(root);
+    assertEquals(NodeType.JOIN, root.getChild().getType());
+    joinNode = root.getChild();
+    assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
+    assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
+  }
+  
+  @Test
+  public final void testProjectionPushWithInnerJoin() throws PlanningException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(QUERIES[5]);
+    LogicalPlan newPlan = planner.createPlan(session, expr);
+    optimizer.optimize(newPlan);
+  }
+  
+  @Test
+  public final void testProjectionPush() throws CloneNotSupportedException, PlanningException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(QUERIES[2]);
+    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalNode plan = newPlan.getRootBlock().getRoot();
+    
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    TestLogicalNode.testCloneLogicalNode(root);
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
+    SelectionNode selNode = projNode.getChild();
+    assertEquals(NodeType.SCAN, selNode.getChild().getType());
+
+    LogicalNode optimized = optimizer.optimize(newPlan);
+    assertEquals(NodeType.ROOT, optimized.getType());
+    root = (LogicalRootNode) optimized;
+    TestLogicalNode.testCloneLogicalNode(root);
+    assertEquals(NodeType.SCAN, root.getChild().getType());
+  }
+  
+  @Test
+  public final void testOptimizeWithGroupBy() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[3]);
+    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalNode plan = newPlan.getRootBlock().getRoot();
+        
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    TestLogicalNode.testCloneLogicalNode(root);
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+    assertEquals(NodeType.GROUP_BY, projNode.getChild().getType());
+    GroupbyNode groupbyNode = projNode.getChild();
+    assertEquals(NodeType.SELECTION, groupbyNode.getChild().getType());
+    SelectionNode selNode = groupbyNode.getChild();
+    assertEquals(NodeType.SCAN, selNode.getChild().getType());
+    
+    LogicalNode optimized = optimizer.optimize(newPlan);
+    assertEquals(NodeType.ROOT, optimized.getType());
+    root = (LogicalRootNode) optimized;
+    TestLogicalNode.testCloneLogicalNode(root);
+    assertEquals(NodeType.GROUP_BY, root.getChild().getType());
+    groupbyNode = root.getChild();
+    assertEquals(NodeType.SCAN, groupbyNode.getChild().getType());
+  }
+
+  @Test
+  public final void testPushable() throws CloneNotSupportedException, PlanningException {
+    // two relations
+    Expr expr = sqlAnalyzer.parse(QUERIES[0]);
+    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalNode plan = newPlan.getRootBlock().getRoot();
+    
+    assertEquals(NodeType.ROOT, plan.getType());
+    LogicalRootNode root = (LogicalRootNode) plan;
+    TestLogicalNode.testCloneLogicalNode(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+
+    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
+    SelectionNode selNode = projNode.getChild();
+    
+    assertEquals(NodeType.JOIN, selNode.getChild().getType());
+    JoinNode joinNode = selNode.getChild();
+    assertFalse(joinNode.hasJoinQual());
+    
+    // Test for Pushable
+    assertTrue(LogicalPlanner.checkIfBeEvaluatedAtJoin(newPlan.getRootBlock(), selNode.getQual(), joinNode, false));
+    
+    // Optimized plan
+    LogicalNode optimized = optimizer.optimize(newPlan);
+    assertEquals(NodeType.ROOT, optimized.getType());
+    root = (LogicalRootNode) optimized;
+    
+    assertEquals(NodeType.JOIN, root.getChild().getType());
+    joinNode = root.getChild();
+    assertTrue(joinNode.hasJoinQual());
+    
+    // Scan Pushable Test
+    expr = sqlAnalyzer.parse(QUERIES[1]);
+    newPlan = planner.createPlan(session, expr);
+    plan = newPlan.getRootBlock().getRoot();
+    
+    assertEquals(NodeType.ROOT, plan.getType());
+    root = (LogicalRootNode) plan;
+    TestLogicalNode.testCloneLogicalNode(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    projNode = root.getChild();
+
+    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
+    selNode = projNode.getChild();
+    
+    assertEquals(NodeType.SCAN, selNode.getChild().getType());
+    ScanNode scanNode = selNode.getChild();
+    // Test for Join Node
+    assertTrue(LogicalPlanner.checkIfBeEvaluatedAtRelation(newPlan.getRootBlock(), selNode.getQual(), scanNode));
+  }
+
+  @Test
+  public final void testInsertInto() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(TestLogicalPlanner.insertStatements[0]);
+    LogicalPlan newPlan = planner.createPlan(session, expr);
+    optimizer.optimize(newPlan);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
new file mode 100644
index 0000000..2f1e0f9
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner;
+
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.engine.planner.LogicalPlan.BlockType;
+import static org.junit.Assert.*;
+
+public class TestLogicalPlan {
+  private static TajoTestingCluster util;
+  private static TPCH tpch;
+  private static CatalogService catalog;
+  private static SQLAnalyzer sqlAnalyzer = new SQLAnalyzer();
+  private static LogicalPlanner planner;
+  private static LogicalOptimizer optimizer;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    util = new TajoTestingCluster();
+    util.startCatalogCluster();
+    catalog = util.getMiniCatalogCluster().getCatalog();
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, CommonTestingUtil.getTestDir().toUri().toString());
+    catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+      catalog.createFunction(funcDesc);
+    }
+
+    // TPC-H Schema for Complex Queries
+    String [] tpchTables = {
+        "part", "supplier", "partsupp", "nation", "region", "lineitem", "customer", "orders"
+    };
+    int [] tableVolumns = {
+        100, 200, 50, 5, 5, 800, 300, 100
+    };
+    tpch = new TPCH();
+    tpch.loadSchemas();
+    tpch.loadOutSchema();
+
+    for (int i = 0; i < tpchTables.length; i++) {
+      TableMeta m = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV);
+      TableStats stats = new TableStats();
+      stats.setNumBytes(tableVolumns[i]);
+      TableDesc d = CatalogUtil.newTableDesc(tpchTables[i], tpch.getSchema(tpchTables[i]), m,
+          CommonTestingUtil.getTestDir());
+      d.setStats(stats);
+      catalog.createTable(d);
+    }
+    planner = new LogicalPlanner(catalog);
+    optimizer = new LogicalOptimizer(util.getConfiguration());
+  }
+
+  public static void tearDown() {
+    util.shutdownCatalogCluster();
+  }
+
+  @Test
+  public final void testQueryBlockGraph() {
+    LogicalPlan plan = new LogicalPlan(LocalTajoTestingUtility.createDummySession().getCurrentDatabase(), planner);
+    LogicalPlan.QueryBlock root = plan.newAndGetBlock(LogicalPlan.ROOT_BLOCK);
+    LogicalPlan.QueryBlock new1 = plan.newQueryBlock();
+    LogicalPlan.QueryBlock new2 = plan.newQueryBlock();
+
+    plan.getQueryBlockGraph().addEdge(new1.getName(), root.getName(),
+        new LogicalPlan.BlockEdge(new1, root, BlockType.TableSubQuery));
+    plan.getQueryBlockGraph().addEdge(new2.getName(), root.getName(),
+        new LogicalPlan.BlockEdge(new2, root, BlockType.TableSubQuery));
+
+    SimpleDirectedGraph<String, LogicalPlan.BlockEdge> graph = plan.getQueryBlockGraph();
+    assertEquals(2, graph.getChildCount(root.getName()));
+
+    assertEquals(root.getName(), graph.getParent(new1.getName(), 0));
+    assertEquals(root.getName(), graph.getParent(new2.getName(), 0));
+
+    assertTrue(graph.isRoot(root.getName()));
+    assertFalse(graph.isRoot(new1.getName()));
+    assertFalse(graph.isRoot(new2.getName()));
+
+    assertFalse(graph.isLeaf(root.getName()));
+    assertTrue(graph.isLeaf(new1.getName()));
+    assertTrue(graph.isLeaf(new2.getName()));
+
+    Set<LogicalPlan.QueryBlock> result = new HashSet<LogicalPlan.QueryBlock>();
+    result.add(new1);
+    result.add(new2);
+
+    Set<LogicalPlan.QueryBlock> childs = new HashSet<LogicalPlan.QueryBlock>(plan.getChildBlocks(root));
+    assertEquals(result, childs);
+
+    assertEquals(root, plan.getParentBlock(new1));
+    assertEquals(root, plan.getParentBlock(new2));
+  }
+}


[24/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
new file mode 100644
index 0000000..a8f5b31
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -0,0 +1,702 @@
+/**
+ * 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.master.querymaster;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoProtos.QueryState;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.CreateTableNode;
+import org.apache.tajo.engine.planner.logical.InsertNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class Query implements EventHandler<QueryEvent> {
+  private static final Log LOG = LogFactory.getLog(Query.class);
+
+  // Facilities for Query
+  private final TajoConf systemConf;
+  private final Clock clock;
+  private String queryStr;
+  private Map<ExecutionBlockId, SubQuery> subqueries;
+  private final EventHandler eventHandler;
+  private final MasterPlan plan;
+  private final AbstractStorageManager sm;
+  QueryMasterTask.QueryMasterTaskContext context;
+  private ExecutionBlockCursor cursor;
+
+  // Query Status
+  private final QueryId id;
+  private long appSubmitTime;
+  private long startTime;
+  private long finishTime;
+  private TableDesc resultDesc;
+  private int completedSubQueryCount = 0;
+  private int successedSubQueryCount = 0;
+  private int killedSubQueryCount = 0;
+  private int failedSubQueryCount = 0;
+  private int erroredSubQueryCount = 0;
+  private final List<String> diagnostics = new ArrayList<String>();
+
+  // Internal Variables
+  private final Lock readLock;
+  private final Lock writeLock;
+  private int priority = 100;
+
+  // State Machine
+  private final StateMachine<QueryState, QueryEventType, QueryEvent> stateMachine;
+
+  // Transition Handler
+  private static final SingleArcTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+  private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
+  private static final SubQueryCompletedTransition SUBQUERY_COMPLETED_TRANSITION = new SubQueryCompletedTransition();
+  private static final QueryCompletedTransition QUERY_COMPLETED_TRANSITION = new QueryCompletedTransition();
+
+  protected static final StateMachineFactory
+      <Query,QueryState,QueryEventType,QueryEvent> stateMachineFactory =
+      new StateMachineFactory<Query, QueryState, QueryEventType, QueryEvent>
+          (QueryState.QUERY_NEW)
+
+          // Transitions from NEW state
+          .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_RUNNING,
+              QueryEventType.START,
+              new StartTransition())
+          .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_NEW,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_KILLED,
+              QueryEventType.KILL,
+              new KillNewQueryTransition())
+          .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from RUNNING state
+          .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+              QueryEventType.SUBQUERY_COMPLETED,
+              SUBQUERY_COMPLETED_TRANSITION)
+          .addTransition(QueryState.QUERY_RUNNING,
+              EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
+                  QueryState.QUERY_ERROR),
+              QueryEventType.QUERY_COMPLETED,
+              QUERY_COMPLETED_TRANSITION)
+          .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_KILL_WAIT,
+              QueryEventType.KILL,
+              new KillSubQueriesTransition())
+          .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from QUERY_SUCCEEDED state
+          .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          // ignore-able transitions
+          .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+              QueryEventType.SUBQUERY_COMPLETED,
+              SUBQUERY_COMPLETED_TRANSITION)
+          .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+              QueryEventType.KILL)
+          .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from KILL_WAIT state
+          .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
+              QueryEventType.SUBQUERY_COMPLETED,
+              SUBQUERY_COMPLETED_TRANSITION)
+          .addTransition(QueryState.QUERY_KILL_WAIT,
+              EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
+                  QueryState.QUERY_ERROR),
+              QueryEventType.QUERY_COMPLETED,
+              QUERY_COMPLETED_TRANSITION)
+          .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+          // Ignore-able transitions
+          .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
+              EnumSet.of(QueryEventType.KILL))
+
+          // Transitions from FAILED state
+          .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+          // Ignore-able transitions
+          .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
+              QueryEventType.KILL)
+
+          // Transitions from ERROR state
+          .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+          // Ignore-able transitions
+          .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+              QueryEventType.KILL)
+
+          .installTopology();
+
+  public Query(final QueryMasterTask.QueryMasterTaskContext context, final QueryId id,
+               final long appSubmitTime,
+               final String queryStr,
+               final EventHandler eventHandler,
+               final MasterPlan plan) {
+    this.context = context;
+    this.systemConf = context.getConf();
+    this.id = id;
+    this.clock = context.getClock();
+    this.appSubmitTime = appSubmitTime;
+    this.queryStr = queryStr;
+    subqueries = Maps.newHashMap();
+    this.eventHandler = eventHandler;
+    this.plan = plan;
+    this.sm = context.getStorageManager();
+    cursor = new ExecutionBlockCursor(plan);
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+
+    stateMachine = stateMachineFactory.make(this);
+  }
+
+  public float getProgress() {
+    QueryState state = getStateMachine().getCurrentState();
+    if (state == QueryState.QUERY_SUCCEEDED) {
+      return 1.0f;
+    } else {
+      int idx = 0;
+      List<SubQuery> tempSubQueries = new ArrayList<SubQuery>();
+      synchronized(subqueries) {
+        tempSubQueries.addAll(subqueries.values());
+      }
+      float [] subProgresses = new float[tempSubQueries.size()];
+      boolean finished = true;
+      for (SubQuery subquery: tempSubQueries) {
+        if (subquery.getState() != SubQueryState.NEW) {
+          subProgresses[idx] = subquery.getProgress();
+          if (finished && subquery.getState() != SubQueryState.SUCCEEDED) {
+            finished = false;
+          }
+        } else {
+          subProgresses[idx] = 0.0f;
+        }
+        idx++;
+      }
+
+      if (finished) {
+        return 1.0f;
+      }
+
+      float totalProgress = 0;
+      float proportion = 1.0f / (float)(getExecutionBlockCursor().size() - 1); // minus one is due to
+
+      for (int i = 0; i < subProgresses.length; i++) {
+        totalProgress += subProgresses[i] * proportion;
+      }
+
+      return totalProgress;
+    }
+  }
+
+  public long getAppSubmitTime() {
+    return this.appSubmitTime;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime() {
+    startTime = clock.getTime();
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  public void setFinishTime() {
+    finishTime = clock.getTime();
+  }
+
+  public List<String> getDiagnostics() {
+    readLock.lock();
+    try {
+      return diagnostics;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  protected void addDiagnostic(String diag) {
+    diagnostics.add(diag);
+  }
+
+  public TableDesc getResultDesc() {
+    return resultDesc;
+  }
+
+  public void setResultDesc(TableDesc desc) {
+    resultDesc = desc;
+  }
+
+  public MasterPlan getPlan() {
+    return plan;
+  }
+
+  public StateMachine<QueryState, QueryEventType, QueryEvent> getStateMachine() {
+    return stateMachine;
+  }
+  
+  public void addSubQuery(SubQuery subquery) {
+    subqueries.put(subquery.getId(), subquery);
+  }
+  
+  public QueryId getId() {
+    return this.id;
+  }
+
+  public SubQuery getSubQuery(ExecutionBlockId id) {
+    return this.subqueries.get(id);
+  }
+
+  public Collection<SubQuery> getSubQueries() {
+    return this.subqueries.values();
+  }
+
+  public QueryState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public ExecutionBlockCursor getExecutionBlockCursor() {
+    return cursor;
+  }
+
+  public static class StartTransition
+      implements SingleArcTransition<Query, QueryEvent> {
+
+    @Override
+    public void transition(Query query, QueryEvent queryEvent) {
+      query.setStartTime();
+      SubQuery subQuery = new SubQuery(query.context, query.getPlan(),
+          query.getExecutionBlockCursor().nextBlock(), query.sm);
+      subQuery.setPriority(query.priority--);
+      query.addSubQuery(subQuery);
+
+      subQuery.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_INIT));
+      LOG.debug("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
+    }
+  }
+
+  public static class QueryCompletedTransition implements MultipleArcTransition<Query, QueryEvent, QueryState> {
+
+    @Override
+    public QueryState transition(Query query, QueryEvent queryEvent) {
+      QueryCompletedEvent subQueryEvent = (QueryCompletedEvent) queryEvent;
+      QueryState finalState;
+      if (subQueryEvent.getState() == SubQueryState.SUCCEEDED) {
+        finalizeQuery(query, subQueryEvent);
+        finalState = QueryState.QUERY_SUCCEEDED;
+      } else if (subQueryEvent.getState() == SubQueryState.FAILED) {
+        finalState = QueryState.QUERY_FAILED;
+      } else if (subQueryEvent.getState() == SubQueryState.KILLED) {
+        finalState = QueryState.QUERY_KILLED;
+      } else {
+        finalState = QueryState.QUERY_ERROR;
+      }
+      query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+      query.setFinishTime();
+      return finalState;
+    }
+
+    private void finalizeQuery(Query query, QueryCompletedEvent event) {
+      MasterPlan masterPlan = query.getPlan();
+
+      ExecutionBlock terminal = query.getPlan().getTerminalBlock();
+      DataChannel finalChannel = masterPlan.getChannel(event.getExecutionBlockId(), terminal.getId());
+      Path finalOutputDir = commitOutputData(query);
+
+      QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
+      try {
+        hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(),
+            finalOutputDir);
+      } catch (Exception e) {
+        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
+      }
+    }
+
+    /**
+     * It moves a result data stored in a staging output dir into a final output dir.
+     */
+    public Path commitOutputData(Query query) {
+      QueryContext queryContext = query.context.getQueryContext();
+      Path stagingResultDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);
+      Path finalOutputDir;
+      if (queryContext.hasOutputPath()) {
+        finalOutputDir = queryContext.getOutputPath();
+        try {
+          FileSystem fs = stagingResultDir.getFileSystem(query.systemConf);
+
+          if (queryContext.isOutputOverwrite()) { // INSERT OVERWRITE INTO
+
+            // it moves the original table into the temporary location.
+            // Then it moves the new result table into the original table location.
+            // Upon failed, it recovers the original table if possible.
+            boolean movedToOldTable = false;
+            boolean committed = false;
+            Path oldTableDir = new Path(queryContext.getStagingDir(), TajoConstants.INSERT_OVERWIRTE_OLD_TABLE_NAME);
+            try {
+              if (fs.exists(finalOutputDir)) {
+                fs.rename(finalOutputDir, oldTableDir);
+                movedToOldTable = fs.exists(oldTableDir);
+              } else { // if the parent does not exist, make its parent directory.
+                fs.mkdirs(finalOutputDir.getParent());
+              }
+              fs.rename(stagingResultDir, finalOutputDir);
+              committed = fs.exists(finalOutputDir);
+            } catch (IOException ioe) {
+              // recover the old table
+              if (movedToOldTable && !committed) {
+                fs.rename(oldTableDir, finalOutputDir);
+              }
+            }
+          } else {
+            fs.rename(stagingResultDir, finalOutputDir);
+            LOG.info("Moved from the staging dir to the output directory '" + finalOutputDir);
+          }
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      } else {
+        finalOutputDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);
+      }
+
+      return finalOutputDir;
+    }
+
+    private static interface QueryHook {
+      boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir);
+      void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext, Query query,
+                   ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception;
+    }
+
+    private class QueryHookExecutor {
+      private List<QueryHook> hookList = TUtil.newList();
+      private QueryMaster.QueryMasterContext context;
+
+      public QueryHookExecutor(QueryMaster.QueryMasterContext context) {
+        this.context = context;
+        hookList.add(new MaterializedResultHook());
+        hookList.add(new CreateTableHook());
+        hookList.add(new InsertTableHook());
+      }
+
+      public void execute(QueryContext queryContext, Query query,
+                          ExecutionBlockId finalExecBlockId,
+                          Path finalOutputDir) throws Exception {
+        for (QueryHook hook : hookList) {
+          if (hook.isEligible(queryContext, query, finalExecBlockId, finalOutputDir)) {
+            hook.execute(context, queryContext, query, finalExecBlockId, finalOutputDir);
+          }
+        }
+      }
+    }
+
+    private class MaterializedResultHook implements QueryHook {
+
+      @Override
+      public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+                                Path finalOutputDir) {
+        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        NodeType type = lastStage.getBlock().getPlan().getType();
+        return type != NodeType.CREATE_TABLE && type != NodeType.INSERT;
+      }
+
+      @Override
+      public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+                          Query query, ExecutionBlockId finalExecBlockId,
+                          Path finalOutputDir) throws Exception {
+        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        TableMeta meta = lastStage.getTableMeta();
+        TableStats stats = lastStage.getResultStats();
+
+        TableDesc resultTableDesc =
+            new TableDesc(
+                query.getId().toString(),
+                lastStage.getSchema(),
+                meta,
+                finalOutputDir);
+        resultTableDesc.setExternal(true);
+
+        stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
+        resultTableDesc.setStats(stats);
+        query.setResultDesc(resultTableDesc);
+      }
+    }
+
+    private class CreateTableHook implements QueryHook {
+
+      @Override
+      public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+                                Path finalOutputDir) {
+        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        return lastStage.getBlock().getPlan().getType() == NodeType.CREATE_TABLE;
+      }
+
+      @Override
+      public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+                          Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception {
+        CatalogService catalog = context.getWorkerContext().getCatalog();
+        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        TableStats stats = lastStage.getResultStats();
+
+        CreateTableNode createTableNode = (CreateTableNode) lastStage.getBlock().getPlan();
+        TableMeta meta = new TableMeta(createTableNode.getStorageType(), createTableNode.getOptions());
+
+        TableDesc tableDescTobeCreated =
+            new TableDesc(
+                createTableNode.getTableName(),
+                createTableNode.getTableSchema(),
+                meta,
+                finalOutputDir);
+        tableDescTobeCreated.setExternal(createTableNode.isExternal());
+
+        if (createTableNode.hasPartition()) {
+          tableDescTobeCreated.setPartitionMethod(createTableNode.getPartitionMethod());
+        }
+
+        stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
+        tableDescTobeCreated.setStats(stats);
+        query.setResultDesc(tableDescTobeCreated);
+
+        catalog.createTable(tableDescTobeCreated);
+      }
+    }
+
+    private class InsertTableHook implements QueryHook {
+
+      @Override
+      public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+                                Path finalOutputDir) {
+        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        return lastStage.getBlock().getPlan().getType() == NodeType.INSERT;
+      }
+
+      @Override
+      public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+                          Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir)
+          throws Exception {
+
+        CatalogService catalog = context.getWorkerContext().getCatalog();
+        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        TableMeta meta = lastStage.getTableMeta();
+        TableStats stats = lastStage.getResultStats();
+
+        InsertNode insertNode = (InsertNode) lastStage.getBlock().getPlan();
+
+        TableDesc finalTable;
+        if (insertNode.hasTargetTable()) {
+          String tableName = insertNode.getTableName();
+          finalTable = catalog.getTableDesc(tableName);
+        } else {
+          String tableName = query.getId().toString();
+          finalTable = new TableDesc(tableName, lastStage.getSchema(), meta, finalOutputDir);
+        }
+
+        long volume = getTableVolume(query.systemConf, finalOutputDir);
+        stats.setNumBytes(volume);
+        finalTable.setStats(stats);
+
+        if (insertNode.hasTargetTable()) {
+          catalog.dropTable(insertNode.getTableName());
+          catalog.createTable(finalTable);
+        }
+
+        query.setResultDesc(finalTable);
+      }
+    }
+
+    private long getTableVolume(TajoConf systemConf, Path tablePath) throws IOException {
+      FileSystem fs = tablePath.getFileSystem(systemConf);
+      ContentSummary directorySummary = fs.getContentSummary(tablePath);
+      return directorySummary.getLength();
+    }
+  }
+
+  public static class SubQueryCompletedTransition implements SingleArcTransition<Query, QueryEvent> {
+
+    private boolean hasNext(Query query) {
+      ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
+      ExecutionBlock nextBlock = cursor.peek();
+      return !query.getPlan().isTerminal(nextBlock);
+    }
+
+    private void executeNextBlock(Query query) {
+      ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
+      ExecutionBlock nextBlock = cursor.nextBlock();
+      SubQuery nextSubQuery = new SubQuery(query.context, query.getPlan(), nextBlock, query.sm);
+      nextSubQuery.setPriority(query.priority--);
+      query.addSubQuery(nextSubQuery);
+      nextSubQuery.handle(new SubQueryEvent(nextSubQuery.getId(), SubQueryEventType.SQ_INIT));
+
+      LOG.info("Scheduling SubQuery:" + nextSubQuery.getId());
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Scheduling SubQuery's Priority: " + nextSubQuery.getPriority());
+        LOG.debug("Scheduling SubQuery's Plan: \n" + nextSubQuery.getBlock().getPlan());
+      }
+    }
+
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      try {
+        query.completedSubQueryCount++;
+        SubQueryCompletedEvent castEvent = (SubQueryCompletedEvent) event;
+
+        if (castEvent.getState() == SubQueryState.SUCCEEDED) {
+          query.successedSubQueryCount++;
+        } else if (castEvent.getState() == SubQueryState.KILLED) {
+          query.killedSubQueryCount++;
+        } else if (castEvent.getState() == SubQueryState.FAILED) {
+          query.failedSubQueryCount++;
+        } else if (castEvent.getState() == SubQueryState.ERROR) {
+          query.erroredSubQueryCount++;
+        } else {
+          LOG.error(String.format("Invalid SubQuery (%s) State %s at %s",
+              castEvent.getExecutionBlockId().toString(), castEvent.getState().name(), query.getState().name()));
+          query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
+        }
+
+        // if a subquery is succeeded and a query is running
+        if (castEvent.getState() == SubQueryState.SUCCEEDED &&  // latest subquery succeeded
+            query.getState() == QueryState.QUERY_RUNNING &&     // current state is not in KILL_WAIT, FAILED, or ERROR.
+            hasNext(query)) {                                   // there remains at least one subquery.
+          executeNextBlock(query);
+        } else { // if a query is completed due to finished, kill, failure, or error
+          query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState()));
+        }
+      } catch (Throwable t) {
+        LOG.error(t);
+        query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
+      }
+    }
+  }
+
+  private static class DiagnosticsUpdateTransition implements SingleArcTransition<Query, QueryEvent> {
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      query.addDiagnostic(((QueryDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
+    }
+  }
+
+  private static class KillNewQueryTransition implements SingleArcTransition<Query, QueryEvent> {
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      query.setFinishTime();
+      query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+    }
+  }
+
+  private static class KillSubQueriesTransition implements SingleArcTransition<Query, QueryEvent> {
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      synchronized (query.subqueries) {
+        for (SubQuery subquery : query.subqueries.values()) {
+          query.eventHandler.handle(new SubQueryEvent(subquery.getId(), SubQueryEventType.SQ_KILL));
+        }
+      }
+    }
+  }
+
+  private static class InternalErrorTransition implements SingleArcTransition<Query, QueryEvent> {
+
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      query.setFinishTime();
+      query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+    }
+  }
+
+  @Override
+  public void handle(QueryEvent event) {
+    LOG.info("Processing " + event.getQueryId() + " of type " + event.getType());
+    try {
+      writeLock.lock();
+      QueryState oldState = getState();
+      try {
+        getStateMachine().doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state", e);
+        eventHandler.handle(new QueryEvent(this.id, QueryEventType.INTERNAL_ERROR));
+      }
+
+      //notify the eventhandler of state change
+      if (oldState != getState()) {
+        LOG.info(id + " Query Transitioned from " + oldState + " to " + getState());
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
new file mode 100644
index 0000000..de323cd
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
@@ -0,0 +1,294 @@
+/**
+ * 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.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TajoAsyncDispatcher;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.WorkerResourceManager;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.ipc.TajoMasterProtocol.WorkerAllocatedResource;
+
+public class QueryInProgress extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(QueryInProgress.class.getName());
+
+  private QueryId queryId;
+
+  private Session session;
+
+  private QueryContext queryContext;
+
+  private TajoAsyncDispatcher dispatcher;
+
+  private LogicalRootNode plan;
+
+  private AtomicBoolean querySubmitted = new AtomicBoolean(false);
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private QueryInfo queryInfo;
+
+  private final TajoMaster.MasterContext masterContext;
+
+  private NettyClientBase queryMasterRpc;
+
+  private QueryMasterProtocolService queryMasterRpcClient;
+
+  private YarnProtos.ContainerIdProto qmContainerId;
+
+  public QueryInProgress(
+      TajoMaster.MasterContext masterContext,
+      Session session,
+      QueryContext queryContext,
+      QueryId queryId, String sql, LogicalRootNode plan) {
+    super(QueryInProgress.class.getName());
+    this.masterContext = masterContext;
+    this.session = session;
+    this.queryContext = queryContext;
+    this.queryId = queryId;
+    this.plan = plan;
+
+    queryInfo = new QueryInfo(queryId, sql);
+    queryInfo.setStartTime(System.currentTimeMillis());
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    dispatcher = new TajoAsyncDispatcher("QueryInProgress:" + queryId);
+    this.addService(dispatcher);
+
+    dispatcher.register(QueryJobEvent.Type.class, new QueryInProgressEventHandler());
+    super.init(conf);
+  }
+
+  public void kill() {
+    queryMasterRpcClient.killQuery(null, queryId.getProto(), NullCallback.get());
+  }
+
+  @Override
+  public void stop() {
+    if(stopped.getAndSet(true)) {
+      return;
+    }
+
+    LOG.info("=========================================================");
+    LOG.info("Stop query:" + queryId);
+
+    masterContext.getResourceManager().stopQueryMaster(queryId);
+
+    long startTime = System.currentTimeMillis();
+    while(true) {
+      try {
+        if(masterContext.getResourceManager().isQueryMasterStopped(queryId)) {
+          LOG.info(queryId + " QueryMaster stopped");
+          break;
+        }
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        break;
+      }
+
+      try {
+        synchronized (this){
+          wait(100);
+        }
+      } catch (InterruptedException e) {
+        break;
+      }
+      if(System.currentTimeMillis() - startTime > 60 * 1000) {
+        LOG.warn("Failed to stop QueryMaster:" + queryId);
+        break;
+      }
+    }
+
+    if(queryMasterRpc != null) {
+      RpcConnectionPool.getPool((TajoConf)getConfig()).closeConnection(queryMasterRpc);
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  public EventHandler getEventHandler() {
+    return dispatcher.getEventHandler();
+  }
+
+
+
+  public boolean startQueryMaster() {
+    try {
+      LOG.info("Initializing QueryInProgress for QueryID=" + queryId);
+      WorkerResourceManager resourceManager = masterContext.getResourceManager();
+      WorkerAllocatedResource resource = resourceManager.allocateQueryMaster(this);
+
+      // if no resource to allocate a query master
+      if(resource == null) {
+        LOG.info("No Available Resources for QueryMaster");
+        return false;
+      }
+
+      queryInfo.setQueryMaster(resource.getWorkerHost());
+      queryInfo.setQueryMasterPort(resource.getQueryMasterPort());
+      queryInfo.setQueryMasterclientPort(resource.getClientPort());
+
+      getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, queryInfo));
+
+      return true;
+    } catch (Exception e) {
+      catchException(e);
+      return false;
+    }
+  }
+
+  class QueryInProgressEventHandler implements EventHandler<QueryJobEvent> {
+    @Override
+    public void handle(QueryJobEvent queryJobEvent) {
+      if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_HEARTBEAT) {
+        heartbeat(queryJobEvent.getQueryInfo());
+      } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_MASTER_START) {
+        QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
+        queryInProgress.getEventHandler().handle(
+            new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInProgress.getQueryInfo()));
+      } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_START) {
+        submmitQueryToMaster();
+      } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_FINISH) {
+        stop();
+      } else if (queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_KILL) {
+        kill();
+      }
+    }
+  }
+
+  public QueryMasterProtocolService getQueryMasterRpcClient() {
+    return queryMasterRpcClient;
+  }
+
+  private void connectQueryMaster() throws Exception {
+    InetSocketAddress addr = NetUtils.createSocketAddrForHost(
+        queryInfo.getQueryMasterHost(), queryInfo.getQueryMasterPort());
+    LOG.info("Connect to QueryMaster:" + addr);
+    queryMasterRpc =
+        RpcConnectionPool.getPool((TajoConf) getConfig()).getConnection(addr, QueryMasterProtocol.class, true);
+    queryMasterRpcClient = queryMasterRpc.getStub();
+  }
+
+  private synchronized void submmitQueryToMaster() {
+    if(querySubmitted.get()) {
+      return;
+    }
+
+    try {
+      if(queryMasterRpcClient == null) {
+        connectQueryMaster();
+      }
+      if(queryMasterRpcClient == null) {
+        LOG.info("No QueryMaster conneciton info.");
+        //TODO wait
+        return;
+      }
+      LOG.info("Call executeQuery to :" +
+          queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort() + "," + queryId);
+      queryMasterRpcClient.executeQuery(
+          null,
+          TajoWorkerProtocol.QueryExecutionRequestProto.newBuilder()
+              .setQueryId(queryId.getProto())
+              .setSession(session.getProto())
+              .setQueryContext(queryContext.getProto())
+              .setSql(PrimitiveProtos.StringProto.newBuilder().setValue(queryInfo.getSql()))
+              .setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build())
+              .build(), NullCallback.get());
+      querySubmitted.set(true);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  public void catchException(Exception e) {
+    LOG.error(e.getMessage(), e);
+    queryInfo.setQueryState(TajoProtos.QueryState.QUERY_FAILED);
+    queryInfo.setLastMessage(StringUtils.stringifyException(e));
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public QueryInfo getQueryInfo() {
+    return this.queryInfo;
+  }
+
+  private void heartbeat(QueryInfo queryInfo) {
+    LOG.info("Received QueryMaster heartbeat:" + queryInfo);
+    this.queryInfo.setQueryState(queryInfo.getQueryState());
+    this.queryInfo.setProgress(queryInfo.getProgress());
+    this.queryInfo.setFinishTime(queryInfo.getFinishTime());
+
+    if(queryInfo.getLastMessage() != null && !queryInfo.getLastMessage().isEmpty()) {
+      this.queryInfo.setLastMessage(queryInfo.getLastMessage());
+      LOG.info(queryId + queryInfo.getLastMessage());
+    }
+    if(this.queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_FAILED) {
+      //TODO needed QueryMaster's detail status(failed before or after launching worker)
+      //queryMasterStopped.set(true);
+      LOG.warn(queryId + " failed, " + queryInfo.getLastMessage());
+    }
+
+    if(!querySubmitted.get()) {
+      getEventHandler().handle(
+          new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, this.queryInfo));
+    }
+
+    if(isFinishState(this.queryInfo.getQueryState())) {
+      getEventHandler().handle(
+          new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_FINISH, this.queryInfo));
+    }
+  }
+
+  private boolean isFinishState(TajoProtos.QueryState state) {
+    return state == TajoProtos.QueryState.QUERY_FAILED ||
+        state == TajoProtos.QueryState.QUERY_KILLED ||
+        state == TajoProtos.QueryState.QUERY_SUCCEEDED;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
new file mode 100644
index 0000000..b077b36
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+
+public class QueryInfo {
+  private QueryId queryId;
+  private String sql;
+  private TajoProtos.QueryState queryState;
+  private float progress;
+  private long startTime;
+  private long finishTime;
+  private String lastMessage;
+  private String hostNameOfQM;
+  private int queryMasterPort;
+  private int queryMasterClientPort;
+
+  public QueryInfo(QueryId queryId) {
+    this(queryId, null);
+  }
+
+  public QueryInfo(QueryId queryId, String sql) {
+    this.queryId = queryId;
+    this.sql = sql;
+    this.queryState = TajoProtos.QueryState.QUERY_MASTER_INIT;
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public String getSql() {
+    return sql;
+  }
+
+  public String getQueryMasterHost() {
+    return hostNameOfQM;
+  }
+
+  public void setQueryMaster(String hostName) {
+    this.hostNameOfQM = hostName;
+
+  }
+
+  public void setQueryMasterPort(int port) {
+    this.queryMasterPort = port;
+  }
+
+  public int getQueryMasterPort() {
+    return queryMasterPort;
+  }
+
+  public void setQueryMasterclientPort(int port) {
+    queryMasterClientPort = port;
+  }
+
+  public int getQueryMasterClientPort() {
+    return queryMasterClientPort;
+  }
+
+  public TajoProtos.QueryState getQueryState() {
+    return queryState;
+  }
+
+  public void setQueryState(TajoProtos.QueryState queryState) {
+    this.queryState = queryState;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  public void setFinishTime(long finishTime) {
+    this.finishTime = finishTime;
+  }
+
+  public String getLastMessage() {
+    return lastMessage;
+  }
+
+  public void setLastMessage(String lastMessage) {
+    this.lastMessage = lastMessage;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  public void setProgress(float progress) {
+    this.progress = progress;
+  }
+
+  @Override
+  public String toString() {
+    return queryId.toString() + "state=" + queryState +",progress=" + progress + ", queryMaster="
+        + getQueryMasterHost();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
new file mode 100644
index 0000000..811de1b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
@@ -0,0 +1,44 @@
+/**
+ * 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.master.querymaster;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+public class QueryJobEvent extends AbstractEvent<QueryJobEvent.Type> {
+  private QueryInfo queryInfo;
+
+  public QueryJobEvent(Type type, QueryInfo queryInfo) {
+    super(type);
+
+    this.queryInfo = queryInfo;
+  }
+
+  public QueryInfo getQueryInfo() {
+    return this.queryInfo;
+  }
+
+  public enum Type {
+    QUERY_JOB_START,
+    QUERY_JOB_HEARTBEAT,
+    QUERY_JOB_FINISH,
+    QUERY_MASTER_START,
+    QUERY_MASTER_STOP,
+    QUERY_JOB_KILL
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
new file mode 100644
index 0000000..ca45534
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
@@ -0,0 +1,195 @@
+/**
+ * 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.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.session.Session;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class QueryJobManager extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(QueryJobManager.class.getName());
+
+  // TajoMaster Context
+  private final TajoMaster.MasterContext masterContext;
+
+  private AsyncDispatcher dispatcher;
+
+  private final Map<QueryId, QueryInProgress> runningQueries = new HashMap<QueryId, QueryInProgress>();
+
+  private final Map<QueryId, QueryInProgress> finishedQueries = new HashMap<QueryId, QueryInProgress>();
+
+  public QueryJobManager(final TajoMaster.MasterContext masterContext) {
+    super(QueryJobManager.class.getName());
+    this.masterContext = masterContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    try {
+      this.dispatcher = new AsyncDispatcher();
+      addService(this.dispatcher);
+
+      this.dispatcher.register(QueryJobEvent.Type.class, new QueryJobManagerEventHandler());
+    } catch (Exception e) {
+      catchException(null, e);
+    }
+
+    super.init(conf);
+  }
+
+  @Override
+  public void stop() {
+    synchronized(runningQueries) {
+      for(QueryInProgress eachQueryInProgress: runningQueries.values()) {
+        eachQueryInProgress.stop();
+      }
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  public EventHandler getEventHandler() {
+    return dispatcher.getEventHandler();
+  }
+
+  public Collection<QueryInProgress> getRunningQueries() {
+    return Collections.unmodifiableCollection(runningQueries.values());
+  }
+
+  public Collection<QueryInProgress> getFinishedQueries() {
+    return Collections.unmodifiableCollection(finishedQueries.values());
+  }
+
+  public QueryInfo createNewQueryJob(Session session, QueryContext queryContext, String sql, LogicalRootNode plan)
+      throws Exception {
+    QueryId queryId = QueryIdFactory.newQueryId(masterContext.getResourceManager().getSeedQueryId());
+    QueryInProgress queryInProgress = new QueryInProgress(masterContext, session, queryContext, queryId, sql, plan);
+
+    synchronized(runningQueries) {
+      runningQueries.put(queryId, queryInProgress);
+    }
+
+    addService(queryInProgress);
+    queryInProgress.init(getConfig());
+    queryInProgress.start();
+
+    if(!queryInProgress.startQueryMaster()) {
+      return null;
+    }
+
+    return queryInProgress.getQueryInfo();
+  }
+
+  class QueryJobManagerEventHandler implements EventHandler<QueryJobEvent> {
+    @Override
+    public void handle(QueryJobEvent event) {
+      QueryInProgress queryInProgress = null;
+      synchronized(runningQueries) {
+        queryInProgress = runningQueries.get(event.getQueryInfo().getQueryId());
+        if(queryInProgress == null) {
+          LOG.warn("No query info in running queries.[" + event.getQueryInfo().getQueryId() + "]");
+          return;
+        }
+      }
+      queryInProgress.getEventHandler().handle(event);
+    }
+  }
+
+  public QueryInProgress getQueryInProgress(QueryId queryId) {
+    synchronized(runningQueries) {
+      return runningQueries.get(queryId);
+    }
+  }
+
+  public QueryInProgress getFinishedQuery(QueryId queryId) {
+    synchronized(finishedQueries) {
+      return finishedQueries.get(queryId);
+    }
+  }
+
+  public void stopQuery(QueryId queryId) {
+    LOG.info("Stop QueryInProgress:" + queryId);
+    QueryInProgress queryInProgress = getQueryInProgress(queryId);
+    if(queryInProgress != null) {
+      queryInProgress.stop();
+      synchronized(runningQueries) {
+        runningQueries.remove(queryId);
+        finishedQueries.put(queryId, queryInProgress);
+      }
+    } else {
+      LOG.warn("No QueryInProgress while query stopping: " + queryId);
+    }
+  }
+
+  private void catchException(QueryId queryId, Exception e) {
+    LOG.error(e.getMessage(), e);
+    QueryInProgress queryInProgress = runningQueries.get(queryId);
+    queryInProgress.catchException(e);
+  }
+
+  public synchronized TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand queryHeartbeat(
+      TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+    QueryInProgress queryInProgress = getQueryInProgress(new QueryId(queryHeartbeat.getQueryId()));
+    if(queryInProgress == null) {
+      return null;
+    }
+
+    QueryInfo queryInfo = makeQueryInfoFromHeartbeat(queryHeartbeat);
+    getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_HEARTBEAT, queryInfo));
+
+    return null;
+  }
+
+  private QueryInfo makeQueryInfoFromHeartbeat(TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+    QueryInfo queryInfo = new QueryInfo(new QueryId(queryHeartbeat.getQueryId()));
+    if(queryHeartbeat.getTajoWorkerHost() != null) {
+      queryInfo.setQueryMaster(queryHeartbeat.getTajoWorkerHost());
+      queryInfo.setQueryMasterPort(queryHeartbeat.getTajoQueryMasterPort());
+      queryInfo.setQueryMasterclientPort(queryHeartbeat.getTajoWorkerClientPort());
+    }
+    queryInfo.setLastMessage(queryHeartbeat.getStatusMessage());
+    queryInfo.setQueryState(queryHeartbeat.getState());
+    queryInfo.setProgress(queryHeartbeat.getQueryProgress());
+
+    if (queryHeartbeat.hasQueryFinishTime()) {
+      queryInfo.setFinishTime(queryHeartbeat.getQueryFinishTime());
+    }
+
+    return queryInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
new file mode 100644
index 0000000..523f5ba
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -0,0 +1,504 @@
+/**
+ * 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.master.querymaster;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TajoAsyncDispatcher;
+import org.apache.tajo.master.event.QueryStartEvent;
+import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.StorageManagerFactory;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeat;
+import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeatResponse;
+
+// TODO - when exception, send error status to QueryJobManager
+public class QueryMaster extends CompositeService implements EventHandler {
+  private static final Log LOG = LogFactory.getLog(QueryMaster.class.getName());
+
+  private int querySessionTimeout;
+
+  private Clock clock;
+
+  private TajoAsyncDispatcher dispatcher;
+
+  private GlobalPlanner globalPlanner;
+
+  private AbstractStorageManager storageManager;
+
+  private TajoConf systemConf;
+
+  private Map<QueryId, QueryMasterTask> queryMasterTasks = Maps.newConcurrentMap();
+
+  private Map<QueryId, QueryMasterTask> finishedQueryMasterTasks = Maps.newConcurrentMap();
+
+  private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
+
+  private AtomicBoolean queryMasterStop = new AtomicBoolean(false);
+
+  private QueryMasterContext queryMasterContext;
+
+  private QueryHeartbeatThread queryHeartbeatThread;
+
+  private FinishedQueryMasterTaskCleanThread finishedQueryMasterTaskCleanThread;
+
+  private TajoWorker.WorkerContext workerContext;
+
+  private RpcConnectionPool connPool;
+
+  public QueryMaster(TajoWorker.WorkerContext workerContext) {
+    super(QueryMaster.class.getName());
+    this.workerContext = workerContext;
+  }
+
+  public void init(Configuration conf) {
+    LOG.info("QueryMaster init");
+    try {
+      this.systemConf = (TajoConf)conf;
+      this.connPool = RpcConnectionPool.getPool(systemConf);
+
+      querySessionTimeout = systemConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT);
+      queryMasterContext = new QueryMasterContext(systemConf);
+
+      clock = new SystemClock();
+
+      this.dispatcher = new TajoAsyncDispatcher("querymaster_" + System.currentTimeMillis());
+      addIfService(dispatcher);
+
+      this.storageManager = StorageManagerFactory.getStorageManager(systemConf);
+
+      globalPlanner = new GlobalPlanner(systemConf, workerContext);
+
+      dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler());
+
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+      throw new RuntimeException(t);
+    }
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("QueryMaster start");
+
+    queryHeartbeatThread = new QueryHeartbeatThread();
+    queryHeartbeatThread.start();
+
+    clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
+    clientSessionTimeoutCheckThread.start();
+
+    finishedQueryMasterTaskCleanThread = new FinishedQueryMasterTaskCleanThread();
+    finishedQueryMasterTaskCleanThread.start();
+
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(queryMasterStop.getAndSet(true)){
+      return;
+    }
+
+    if(queryHeartbeatThread != null) {
+      queryHeartbeatThread.interrupt();
+    }
+
+    if(clientSessionTimeoutCheckThread != null) {
+      clientSessionTimeoutCheckThread.interrupt();
+    }
+
+    if(finishedQueryMasterTaskCleanThread != null) {
+      finishedQueryMasterTaskCleanThread.interrupt();
+    }
+    super.stop();
+
+    LOG.info("QueryMaster stop");
+    if(queryMasterContext.getWorkerContext().isYarnContainerMode()) {
+      queryMasterContext.getWorkerContext().stopWorker(true);
+    }
+  }
+
+  private void cleanup(QueryId queryId) {
+    LOG.info("cleanup query resources : " + queryId);
+    NettyClientBase rpc = null;
+    List<TajoMasterProtocol.WorkerResourceProto> workers = getAllWorker();
+
+    for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
+      try {
+        if (worker.getPeerRpcPort() == 0) continue;
+
+        rpc = connPool.getConnection(NetUtils.createSocketAddr(worker.getHost(), worker.getPeerRpcPort()),
+            TajoWorkerProtocol.class, true);
+        TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub();
+
+        tajoWorkerProtocolService.cleanup(null, queryId.getProto(), NullCallback.get());
+      } catch (Exception e) {
+        LOG.error(e.getMessage());
+      } finally {
+        connPool.releaseConnection(rpc);
+      }
+    }
+  }
+
+  public List<TajoMasterProtocol.WorkerResourceProto> getAllWorker() {
+
+    NettyClientBase rpc = null;
+    try {
+      rpc = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+          TajoMasterProtocol.class, true);
+      TajoMasterProtocol.TajoMasterProtocolService masterService = rpc.getStub();
+
+      CallFuture<TajoMasterProtocol.WorkerResourcesRequest> callBack =
+          new CallFuture<TajoMasterProtocol.WorkerResourcesRequest>();
+      masterService.getAllWorkerResource(callBack.getController(),
+          PrimitiveProtos.NullProto.getDefaultInstance(), callBack);
+
+      TajoMasterProtocol.WorkerResourcesRequest workerResourcesRequest = callBack.get(2, TimeUnit.SECONDS);
+      return workerResourcesRequest.getWorkerResourcesList();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    } finally {
+      connPool.releaseConnection(rpc);
+    }
+    return new ArrayList<TajoMasterProtocol.WorkerResourceProto>();
+  }
+
+  public void reportQueryStatusToQueryMaster(QueryId queryId, TajoProtos.QueryState state) {
+    LOG.info("Send QueryMaster Ready to QueryJobManager:" + queryId);
+    NettyClientBase tmClient = null;
+    try {
+      tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+          TajoMasterProtocol.class, true);
+      TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+
+      TajoHeartbeat.Builder queryHeartbeatBuilder = TajoHeartbeat.newBuilder()
+          .setTajoWorkerHost(workerContext.getQueryMasterManagerService().getBindAddr().getHostName())
+          .setTajoQueryMasterPort(workerContext.getQueryMasterManagerService().getBindAddr().getPort())
+          .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+          .setState(state)
+          .setQueryId(queryId.getProto());
+
+      CallFuture<TajoHeartbeatResponse> callBack =
+          new CallFuture<TajoHeartbeatResponse>();
+
+      masterClientService.heartbeat(callBack.getController(), queryHeartbeatBuilder.build(), callBack);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    } finally {
+      connPool.releaseConnection(tmClient);
+    }
+  }
+
+  @Override
+  public void handle(Event event) {
+    dispatcher.getEventHandler().handle(event);
+  }
+
+  public Query getQuery(QueryId queryId) {
+    return queryMasterTasks.get(queryId).getQuery();
+  }
+
+  public QueryMasterTask getQueryMasterTask(QueryId queryId) {
+    return queryMasterTasks.get(queryId);
+  }
+
+  public QueryMasterTask getQueryMasterTask(QueryId queryId, boolean includeFinished) {
+    QueryMasterTask queryMasterTask =  queryMasterTasks.get(queryId);
+    if(queryMasterTask != null) {
+      return queryMasterTask;
+    } else {
+      if(includeFinished) {
+        return finishedQueryMasterTasks.get(queryId);
+      } else {
+        return null;
+      }
+    }
+  }
+
+  public QueryMasterContext getContext() {
+    return this.queryMasterContext;
+  }
+
+  public Collection<QueryMasterTask> getQueryMasterTasks() {
+    return queryMasterTasks.values();
+  }
+
+  public Collection<QueryMasterTask> getFinishedQueryMasterTasks() {
+    return finishedQueryMasterTasks.values();
+  }
+
+  public class QueryMasterContext {
+    private TajoConf conf;
+
+    public QueryMasterContext(TajoConf conf) {
+      this.conf = conf;
+    }
+
+    public TajoConf getConf() {
+      return conf;
+    }
+
+    public TajoAsyncDispatcher getDispatcher() {
+      return dispatcher;
+    }
+
+    public Clock getClock() {
+      return clock;
+    }
+
+    public AbstractStorageManager getStorageManager() {
+      return storageManager;
+    }
+
+    public QueryMaster getQueryMaster() {
+      return QueryMaster.this;
+    }
+
+    public GlobalPlanner getGlobalPlanner() {
+      return globalPlanner;
+    }
+
+    public TajoWorker.WorkerContext getWorkerContext() {
+      return workerContext;
+    }
+
+    public EventHandler getEventHandler() {
+      return dispatcher.getEventHandler();
+    }
+
+    public void stopQuery(QueryId queryId) {
+      QueryMasterTask queryMasterTask;
+      queryMasterTask = queryMasterTasks.remove(queryId);
+      finishedQueryMasterTasks.put(queryId, queryMasterTask);
+
+      if(queryMasterTask != null) {
+        TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(queryMasterTask);
+        CallFuture<TajoHeartbeatResponse> future = new CallFuture<TajoHeartbeatResponse>();
+
+        NettyClientBase tmClient = null;
+        try {
+          tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+              TajoMasterProtocol.class, true);
+          TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+          masterClientService.heartbeat(future.getController(), queryHeartbeat, future);
+        }  catch (Exception e) {
+          //this function will be closed in new thread.
+          //When tajo do stop cluster, tajo master maybe throw closed connection exception
+
+          LOG.error(e.getMessage(), e);
+        } finally {
+          connPool.releaseConnection(tmClient);
+        }
+
+        try {
+          queryMasterTask.stop();
+          if (!systemConf.get(CommonTestingUtil.TAJO_TEST, "FALSE").equalsIgnoreCase("TRUE")
+              && !workerContext.isYarnContainerMode()) {
+            cleanup(queryId);       // TODO We will support yarn mode
+          }
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+        }
+      } else {
+        LOG.warn("No query info:" + queryId);
+      }
+      if(workerContext.isYarnContainerMode()) {
+        stop();
+      }
+    }
+  }
+
+  private TajoHeartbeat buildTajoHeartBeat(QueryMasterTask queryMasterTask) {
+    TajoHeartbeat.Builder builder = TajoHeartbeat.newBuilder();
+
+    builder.setTajoWorkerHost(workerContext.getQueryMasterManagerService().getBindAddr().getHostName());
+    builder.setTajoQueryMasterPort(workerContext.getQueryMasterManagerService().getBindAddr().getPort());
+    builder.setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort());
+    builder.setState(queryMasterTask.getState());
+    builder.setQueryId(queryMasterTask.getQueryId().getProto());
+
+    if (queryMasterTask.getQuery() != null) {
+      builder.setQueryProgress(queryMasterTask.getQuery().getProgress());
+      builder.setQueryFinishTime(queryMasterTask.getQuery().getFinishTime());
+    }
+    return builder.build();
+  }
+
+  private class QueryStartEventHandler implements EventHandler<QueryStartEvent> {
+    @Override
+    public void handle(QueryStartEvent event) {
+      LOG.info("Start QueryStartEventHandler:" + event.getQueryId());
+      QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext,
+          event.getQueryId(), event.getSession(), event.getQueryContext(), event.getSql(), event.getLogicalPlanJson());
+
+      queryMasterTask.init(systemConf);
+      if (!queryMasterTask.isInitError()) {
+        queryMasterTask.start();
+      }
+
+      synchronized(queryMasterTasks) {
+        queryMasterTasks.put(event.getQueryId(), queryMasterTask);
+      }
+
+      if (queryMasterTask.isInitError()) {
+        queryMasterContext.stopQuery(queryMasterTask.getQueryId());
+        return;
+      }
+    }
+  }
+
+  class QueryHeartbeatThread extends Thread {
+    public QueryHeartbeatThread() {
+      super("QueryHeartbeatThread");
+    }
+
+    @Override
+    public void run() {
+      LOG.info("Start QueryMaster heartbeat thread");
+      while(!queryMasterStop.get()) {
+        List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+        synchronized(queryMasterTasks) {
+          tempTasks.addAll(queryMasterTasks.values());
+        }
+        synchronized(queryMasterTasks) {
+          for(QueryMasterTask eachTask: tempTasks) {
+            NettyClientBase tmClient;
+            try {
+              tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+                  TajoMasterProtocol.class, true);
+              TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+
+              CallFuture<TajoHeartbeatResponse> callBack =
+                  new CallFuture<TajoHeartbeatResponse>();
+
+              TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(eachTask);
+              masterClientService.heartbeat(callBack.getController(), queryHeartbeat, callBack);
+            } catch (Throwable t) {
+              t.printStackTrace();
+            }
+          }
+        }
+        synchronized(queryMasterStop) {
+          try {
+            queryMasterStop.wait(2000);
+          } catch (InterruptedException e) {
+            break;
+          }
+        }
+      }
+      LOG.info("QueryMaster heartbeat thread stopped");
+    }
+  }
+
+  class ClientSessionTimeoutCheckThread extends Thread {
+    public void run() {
+      LOG.info("ClientSessionTimeoutCheckThread started");
+      while(!queryMasterStop.get()) {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          break;
+        }
+        List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+        synchronized(queryMasterTasks) {
+          tempTasks.addAll(queryMasterTasks.values());
+        }
+
+        for(QueryMasterTask eachTask: tempTasks) {
+          if(!eachTask.isStopped()) {
+            try {
+              long lastHeartbeat = eachTask.getLastClientHeartbeat();
+              long time = System.currentTimeMillis() - lastHeartbeat;
+              if(lastHeartbeat > 0 && time > querySessionTimeout * 1000) {
+                LOG.warn("Query " + eachTask.getQueryId() + " stopped cause query sesstion timeout: " + time + " ms");
+                eachTask.expiredSessionTimeout();
+              }
+            } catch (Exception e) {
+              LOG.error(eachTask.getQueryId() + ":" + e.getMessage(), e);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  class FinishedQueryMasterTaskCleanThread extends Thread {
+    public void run() {
+      int expireIntervalTime = systemConf.getIntVar(TajoConf.ConfVars.WORKER_HISTORY_EXPIRE_PERIOD);
+      LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime);
+      while(!queryMasterStop.get()) {
+        try {
+          Thread.sleep(60 * 1000 * 60);   // hourly
+        } catch (InterruptedException e) {
+          break;
+        }
+        try {
+          long expireTime = System.currentTimeMillis() - expireIntervalTime * 60 * 1000;
+          cleanExpiredFinishedQueryMasterTask(expireTime);
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+        }
+      }
+    }
+
+    private void cleanExpiredFinishedQueryMasterTask(long expireTime) {
+      synchronized(finishedQueryMasterTasks) {
+        List<QueryId> expiredQueryIds = new ArrayList<QueryId>();
+        for(Map.Entry<QueryId, QueryMasterTask> entry: finishedQueryMasterTasks.entrySet()) {
+          if(entry.getValue().getStartTime() < expireTime) {
+            expiredQueryIds.add(entry.getKey());
+          }
+        }
+
+        for(QueryId eachId: expiredQueryIds) {
+          finishedQueryMasterTasks.remove(eachId);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
new file mode 100644
index 0000000..bf59e9f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
@@ -0,0 +1,239 @@
+/**
+ * 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.master.querymaster;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tajo.*;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.LazyTaskScheduler;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.rpc.AsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.net.InetSocketAddress;
+
+public class QueryMasterManagerService extends CompositeService
+    implements QueryMasterProtocol.QueryMasterProtocolService.Interface {
+  private static final Log LOG = LogFactory.getLog(QueryMasterManagerService.class.getName());
+
+  private AsyncRpcServer rpcServer;
+  private InetSocketAddress bindAddr;
+  private String addr;
+  private int port;
+
+  private QueryMaster queryMaster;
+
+  private TajoWorker.WorkerContext workerContext;
+
+  public QueryMasterManagerService(TajoWorker.WorkerContext workerContext, int port) {
+    super(QueryMasterManagerService.class.getName());
+    this.workerContext = workerContext;
+    this.port = port;
+  }
+
+  public QueryMaster getQueryMaster() {
+    return queryMaster;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    Preconditions.checkArgument(conf instanceof TajoConf);
+    TajoConf tajoConf = (TajoConf) conf;
+    try {
+      // Setup RPC server
+      InetSocketAddress initIsa =
+          new InetSocketAddress("0.0.0.0", port);
+      if (initIsa.getAddress() == null) {
+        throw new IllegalArgumentException("Failed resolve of " + initIsa);
+      }
+
+      int workerNum = tajoConf.getIntVar(TajoConf.ConfVars.QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM);
+      this.rpcServer = new AsyncRpcServer(QueryMasterProtocol.class, this, initIsa, workerNum);
+      this.rpcServer.start();
+
+      this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+      this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+      this.port = bindAddr.getPort();
+
+      queryMaster = new QueryMaster(workerContext);
+      addService(queryMaster);
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    // Get the master address
+    LOG.info("QueryMasterManagerService is bind to " + addr);
+    ((TajoConf)conf).setVar(TajoConf.ConfVars.WORKER_QM_RPC_ADDRESS, addr);
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(rpcServer != null) {
+      rpcServer.shutdown();
+    }
+    LOG.info("QueryMasterManagerService stopped");
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddr() {
+    return bindAddr;
+  }
+
+  public String getHostAndPort() {
+    return bindAddr.getHostName() + ":" + bindAddr.getPort();
+  }
+
+  @Override
+  public void getTask(RpcController controller, TajoWorkerProtocol.GetTaskRequestProto request,
+                      RpcCallback<TajoWorkerProtocol.QueryUnitRequestProto> done) {
+    try {
+      ExecutionBlockId ebId = new ExecutionBlockId(request.getExecutionBlockId());
+      QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(ebId.getQueryId());
+
+      if(queryMasterTask == null || queryMasterTask.isStopped()) {
+        done.run(LazyTaskScheduler.stopTaskRunnerReq);
+      } else {
+        ContainerId cid =
+            queryMasterTask.getQueryTaskContext().getResourceAllocator().makeContainerId(request.getContainerId());
+        LOG.debug("getTask:" + cid + ", ebId:" + ebId);
+        queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(cid, ebId, done));
+      }
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatusProto request,
+                           RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      QueryId queryId = new QueryId(request.getId().getQueryUnitId().getExecutionBlockId().getQueryId());
+      QueryUnitAttemptId attemptId = new QueryUnitAttemptId(request.getId());
+      QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId);
+      if (queryMasterTask == null) {
+        queryMasterTask = queryMaster.getQueryMasterTask(queryId, true);
+      }
+      SubQuery sq = queryMasterTask.getQuery().getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId());
+      QueryUnit task = sq.getQueryUnit(attemptId.getQueryUnitId());
+      QueryUnitAttempt attempt = task.getAttempt(attemptId.getId());
+
+      if(LOG.isDebugEnabled()){
+        LOG.debug(String.format("Task State: %s, Attempt State: %s", task.getState().name(), attempt.getState().name()));
+      }
+
+      if (request.getState() == TajoProtos.TaskAttemptState.TA_KILLED) {
+        LOG.warn(attemptId + " Killed");
+        attempt.handle(
+            new TaskAttemptEvent(new QueryUnitAttemptId(request.getId()), TaskAttemptEventType.TA_LOCAL_KILLED));
+      } else {
+        queryMasterTask.getEventHandler().handle(
+            new TaskAttemptStatusUpdateEvent(new QueryUnitAttemptId(request.getId()), request));
+      }
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void ping(RpcController controller,
+                   TajoIdProtos.QueryUnitAttemptIdProto attemptId,
+                   RpcCallback<PrimitiveProtos.BoolProto> done) {
+    done.run(TajoWorker.TRUE_PROTO);
+  }
+
+  @Override
+  public void fatalError(RpcController controller, TajoWorkerProtocol.TaskFatalErrorReport report,
+                         RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+          new QueryId(report.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+      queryMasterTask.getEventHandler().handle(new TaskFatalErrorEvent(report));
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionReport report,
+                   RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+          new QueryId(report.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+      queryMasterTask.getEventHandler().handle(new TaskCompletionEvent(report));
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void killQuery(RpcController controller, TajoIdProtos.QueryIdProto request,
+                        RpcCallback<PrimitiveProtos.BoolProto> done) {
+    QueryId queryId = new QueryId(request);
+    QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId);
+    queryMasterTask.getQuery().handle(new QueryEvent(queryId, QueryEventType.KILL));
+  }
+
+  @Override
+  public void executeQuery(RpcController controller,
+                           TajoWorkerProtocol.QueryExecutionRequestProto request,
+                           RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      workerContext.getWorkerSystemMetrics().counter("querymaster", "numQuery").inc();
+
+      QueryId queryId = new QueryId(request.getQueryId());
+      LOG.info("Receive executeQuery request:" + queryId);
+      queryMaster.handle(new QueryStartEvent(queryId,
+          new Session(request.getSession()),
+          new QueryContext(request.getQueryContext()), request.getSql().getValue(),
+          request.getLogicalPlanJson().getValue()));
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      workerContext.getWorkerSystemMetrics().counter("querymaster", "errorQuery").inc();
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
new file mode 100644
index 0000000..56dd789
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
@@ -0,0 +1,149 @@
+/**
+ * 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.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.io.PrintWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+
+@Deprecated
+public class QueryMasterRunner extends AbstractService {
+  private static final Log LOG = LogFactory.getLog(QueryMasterRunner.class);
+  private TajoConf systemConf;
+  private QueryMaster queryMaster;
+  private QueryId queryId;
+  private String queryMasterManagerAddress;
+
+  public QueryMasterRunner(QueryId queryId, String queryMasterManagerAddress) {
+    super(QueryMasterRunner.class.getName());
+    this.queryId = queryId;
+    this.queryMasterManagerAddress = queryMasterManagerAddress;
+  }
+
+  private class ShutdownHook implements Runnable {
+    @Override
+    public void run() {
+      LOG.info("============================================");
+      LOG.info("QueryMaster received SIGINT Signal");
+      LOG.info("============================================");
+      stop();
+    }
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.systemConf = (TajoConf)conf;
+    RackResolver.init(systemConf);
+    Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    //create QueryMaster
+    QueryMaster query = new QueryMaster(null);
+
+    query.init(systemConf);
+    query.start();
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  public static void main(String[] args) throws Exception {
+    LOG.info("QueryMasterRunner started");
+
+    final TajoConf conf = new TajoConf();
+    conf.addResource(new Path(TajoConstants.SYSTEM_CONF_FILENAME));
+
+    UserGroupInformation.setConfiguration(conf);
+
+    final QueryId queryId = TajoIdUtils.parseQueryId(args[0]);
+    final String queryMasterManagerAddr = args[1];
+
+    LOG.info("Received QueryId:" + queryId);
+
+    QueryMasterRunner queryMasterRunner = new QueryMasterRunner(queryId, queryMasterManagerAddr);
+    queryMasterRunner.init(conf);
+    queryMasterRunner.start();
+
+    synchronized(queryId) {
+      queryId.wait();
+    }
+
+    System.exit(0);
+  }
+
+  public static void printThreadInfo(PrintWriter stream, String title) {
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    final int STACK_DEPTH = 60;
+    boolean contention = threadBean.isThreadContentionMonitoringEnabled();
+    long[] threadIds = threadBean.getAllThreadIds();
+    stream.println("Process Thread Dump: " + title);
+    stream.println(threadIds.length + " active threads");
+    for (long tid : threadIds) {
+      ThreadInfo info = threadBean.getThreadInfo(tid, STACK_DEPTH);
+      if (info == null) {
+        stream.println("  Inactive");
+        continue;
+      }
+      stream.println("Thread " + getTaskName(info.getThreadId(), info.getThreadName()) + ":");
+      Thread.State state = info.getThreadState();
+      stream.println("  State: " + state);
+      stream.println("  Blocked count: " + info.getBlockedCount());
+      stream.println("  Waited count: " + info.getWaitedCount());
+      if (contention) {
+        stream.println("  Blocked time: " + info.getBlockedTime());
+        stream.println("  Waited time: " + info.getWaitedTime());
+      }
+      if (state == Thread.State.WAITING) {
+        stream.println("  Waiting on " + info.getLockName());
+      } else if (state == Thread.State.BLOCKED) {
+        stream.println("  Blocked on " + info.getLockName());
+        stream.println("  Blocked by " + getTaskName(info.getLockOwnerId(), info.getLockOwnerName()));
+      }
+      stream.println("  Stack:");
+      for (StackTraceElement frame : info.getStackTrace()) {
+        stream.println("    " + frame.toString());
+      }
+    }
+    stream.flush();
+  }
+
+  private static String getTaskName(long id, String name) {
+    if (name == null) {
+      return Long.toString(id);
+    }
+    return id + " (" + name + ")";
+  }
+}


[45/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Sleep.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Sleep.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Sleep.java
new file mode 100644
index 0000000..0ae8386
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Sleep.java
@@ -0,0 +1,52 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "sleep",
+  description = "sleep for seconds",
+  example = "> SELECT sleep(1) from table1;",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})}
+)
+public class Sleep extends GeneralFunction {
+
+  public Sleep() {
+    super(NoArgs);
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    try {
+      Thread.sleep(params.getInt4(0) * 1000);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    return DatumFactory.createInt4(params.getInt4(0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
new file mode 100644
index 0000000..ece7487
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
@@ -0,0 +1,82 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 sum(value FLOAT8)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a set of numbers",
+  example = "> SELECT sum(expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+)
+public class SumDouble extends AggFunction<Datum> {
+
+  public SumDouble() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new SumContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    ((SumContext)ctx).sum += params.get(0).asFloat8();
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+  }
+
+  @Override
+  public Float8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  private class SumContext implements FunctionContext {
+    double sum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDoubleDistinct.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDoubleDistinct.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDoubleDistinct.java
new file mode 100644
index 0000000..9b5b190
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDoubleDistinct.java
@@ -0,0 +1,98 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 sum(distinct value FLOAT8)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a distinct and non-null values",
+  example = "> SELECT sum(distinct expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+)
+public class SumDoubleDistinct extends AggFunction<Datum> {
+
+  public SumDoubleDistinct() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new SumContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+  }
+
+  @Override
+  public void merge(FunctionContext context, Tuple params) {
+    SumContext distinctContext = (SumContext) context;
+    Datum value = params.get(0);
+    if ((distinctContext.latest == null || (!distinctContext.latest.equals(value)) && !(value instanceof NullDatum))) {
+      distinctContext.latest = value;
+      distinctContext.sum += value.asFloat8();
+    }
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+  }
+
+  @Override
+  public Float8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  private class SumContext implements FunctionContext {
+    double sum;
+    Datum latest;
+  }
+
+  public CatalogProtos.FunctionType getFunctionType() {
+    return CatalogProtos.FunctionType.DISTINCT_AGGREGATION;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
new file mode 100644
index 0000000..72c4a6b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 sum(value FLOAT4)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a set of numbers",
+  example = "> SELECT sum(expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+)
+public class SumFloat extends AggFunction<Datum> {
+  public SumFloat() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT4)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new SumContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    ((SumContext)ctx).sum += params.get(0).asFloat4();
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+  }
+
+  @Override
+  public Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  private class SumContext implements FunctionContext {
+    private double sum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloatDistinct.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloatDistinct.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloatDistinct.java
new file mode 100644
index 0000000..f230622
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloatDistinct.java
@@ -0,0 +1,98 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 sum(distinct value FLOAT4)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a distinct and non-null values",
+  example = "> SELECT sum(distinct expr);",
+  returnType = Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+)
+public class SumFloatDistinct extends AggFunction<Datum> {
+
+  public SumFloatDistinct() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT4)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new SumContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+  }
+
+  @Override
+  public void merge(FunctionContext context, Tuple params) {
+    SumContext distinctContext = (SumContext) context;
+    Datum value = params.get(0);
+    if ((distinctContext.latest == null || (!distinctContext.latest.equals(value)) && !(value instanceof NullDatum))) {
+      distinctContext.latest = value;
+      distinctContext.sum += value.asFloat4();
+    }
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.FLOAT8);
+  }
+
+  @Override
+  public Float8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
+  }
+
+  private class SumContext implements FunctionContext {
+    double sum;
+    Datum latest;
+  }
+
+  public CatalogProtos.FunctionType getFunctionType() {
+    return CatalogProtos.FunctionType.DISTINCT_AGGREGATION;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
new file mode 100644
index 0000000..fff3a23
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
@@ -0,0 +1,82 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT4 sum(value INT4)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a set of numbers",
+  example = "> SELECT sum(expr);",
+  returnType = Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+)
+public class SumInt extends AggFunction<Datum> {
+
+  public SumInt() {
+    super(new Column[] {
+        new Column("expr", Type.INT4)
+    });
+  }
+
+  @Override
+  public SumIntContext newContext() {
+    return new SumIntContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    SumIntContext sumCtx = (SumIntContext) ctx;
+    sumCtx.sum += params.get(0).asInt8();
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createInt4(((SumIntContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.INT4);
+  }
+
+  @Override
+  public Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt4(((SumIntContext) ctx).sum);
+  }
+
+  private class SumIntContext implements FunctionContext {
+    int sum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumIntDistinct.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumIntDistinct.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumIntDistinct.java
new file mode 100644
index 0000000..e1f2176
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumIntDistinct.java
@@ -0,0 +1,98 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 sum(distinct value INT4)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a distinct and non-null values",
+  example = "> SELECT sum(distinct expr);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+)
+public class SumIntDistinct extends AggFunction<Datum> {
+
+  public SumIntDistinct() {
+    super(new Column[] {
+        new Column("expr", Type.INT4)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new SumContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+  }
+
+  @Override
+  public void merge(FunctionContext context, Tuple params) {
+    SumContext distinctContext = (SumContext) context;
+    Datum value = params.get(0);
+    if ((distinctContext.latest == null || (!distinctContext.latest.equals(value)) && !(value instanceof NullDatum))) {
+      distinctContext.latest = value;
+      distinctContext.sum += value.asInt4();
+    }
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.INT8);
+  }
+
+  @Override
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
+  }
+
+  private class SumContext implements FunctionContext {
+    long sum;
+    Datum latest;
+  }
+
+  public CatalogProtos.FunctionType getFunctionType() {
+    return CatalogProtos.FunctionType.DISTINCT_AGGREGATION;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java
new file mode 100644
index 0000000..894348d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java
@@ -0,0 +1,82 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 sum(value INT8)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a set of numbers",
+  example = "> SELECT sum(expr);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
+)
+public class SumLong extends AggFunction<Datum> {
+
+  public SumLong() {
+    super(new Column[] {
+        new Column("expr", Type.INT8)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new SumContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    ((SumContext)ctx).sum += params.get(0).asInt8();
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.INT8);
+  }
+
+  @Override
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
+  }
+
+  private class SumContext implements FunctionContext {
+    long sum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLongDistinct.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLongDistinct.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLongDistinct.java
new file mode 100644
index 0000000..d899c37
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLongDistinct.java
@@ -0,0 +1,98 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 sum(distinct value INT8)
+ */
+@Description(
+  functionName = "sum",
+  description = "the sum of a distinct and non-null values",
+  example = "> SELECT sum(distinct expr);",
+  returnType = Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
+)
+public class SumLongDistinct extends AggFunction<Datum> {
+
+  public SumLongDistinct() {
+    super(new Column[] {
+        new Column("expr", Type.INT8)
+    });
+  }
+
+  @Override
+  public FunctionContext newContext() {
+    return new SumContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+  }
+
+  @Override
+  public void merge(FunctionContext context, Tuple params) {
+    SumContext distinctContext = (SumContext) context;
+    Datum value = params.get(0);
+    if ((distinctContext.latest == null || (!distinctContext.latest.equals(value)) && !(value instanceof NullDatum))) {
+      distinctContext.latest = value;
+      distinctContext.sum += value.asInt8();
+    }
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newSimpleDataType(Type.INT8);
+  }
+
+  @Override
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
+  }
+
+  private class SumContext implements FunctionContext {
+    long sum;
+    Datum latest;
+  }
+
+  public CatalogProtos.FunctionType getFunctionType() {
+    return CatalogProtos.FunctionType.DISTINCT_AGGREGATION;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java
new file mode 100644
index 0000000..157e545
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java
@@ -0,0 +1,46 @@
+/**
+ * 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.function.builtin;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+  functionName = "today",
+  description = "get current time millis",
+  example = "> SELECT today();",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {})}
+)
+public class Today extends GeneralFunction {
+
+  public Today() {
+    super(NoArgs);
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    return DatumFactory.createInt8(System.currentTimeMillis());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromDate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromDate.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromDate.java
new file mode 100644
index 0000000..a010a7d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromDate.java
@@ -0,0 +1,194 @@
+/**
+ * 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.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+
+@Description(
+    functionName = "date_part",
+    description = "Extract field from date",
+    example = "> SELECT date_part('month', date '2014-01-17');\n"
+        + "1.0",
+    returnType = TajoDataTypes.Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.DATE})}
+)
+public class DatePartFromDate extends GeneralFunction {
+  public DatePartFromDate() {
+    super(new Column[] {
+        new Column("target", FLOAT8),
+        new Column("source", TEXT)
+    });
+  }
+
+  private DatePartExtractorFromDate extractor = null;
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum target = params.get(0);
+    DateDatum date = null;
+
+    if(target instanceof NullDatum || params.get(1) instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    if(params.get(1) instanceof DateDatum) {
+      date = (DateDatum)(params.get(1));
+    } else {
+      return NullDatum.get();
+    }
+
+    if (extractor == null) {
+      String extractType = target.asChars().toLowerCase();
+
+      if (extractType.equals("century")) {
+        extractor = new CenturyExtractorFromDate();
+      } else if (extractType.equals("day")) {
+        extractor = new DayExtractorFromDate();
+      } else if (extractType.equals("decade")) {
+        extractor = new DecadeExtractorFromDate();
+      } else if (extractType.equals("dow")) {
+        extractor = new DowExtractorFromDate();
+      } else if (extractType.equals("doy")) {
+        extractor = new DoyExtractorFromDate();
+      } else if (extractType.equals("isodow")) {
+        extractor = new ISODowExtractorFromDate();
+      } else if (extractType.equals("isoyear")) {
+        extractor = new ISOYearExtractorFromDate();
+      } else if (extractType.equals("millennium")) {
+        extractor = new MillenniumExtractorFromDate();
+      } else if (extractType.equals("month")) {
+        extractor = new MonthExtractorFromDate();
+      } else if (extractType.equals("quarter")) {
+        extractor = new QuarterExtractorFromDate();
+      } else if (extractType.equals("week")) {
+        extractor = new WeekExtractorFromDate();
+      } else if (extractType.equals("year")) {
+        extractor = new YearExtractorFromDate();
+      } else {
+        extractor = new NullExtractorFromDate();
+      }
+    }
+
+    return extractor.extract(date);
+  }
+
+  private interface DatePartExtractorFromDate {
+    public Datum extract(DateDatum date);
+  }
+
+  private class CenturyExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getCenturyOfEra());
+    }
+  }
+
+  private class DayExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getDayOfMonth());
+    }
+  }
+
+  private class DecadeExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) (date.getYear() / 10));
+    }
+  }
+
+  private class DowExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      Integer tdow = date.getDayOfWeek();
+      return DatumFactory.createFloat8((double) ((tdow == 7) ? 0 : tdow));
+    }
+  }
+
+  private class DoyExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getDayOfYear());
+    }
+  }
+
+  private class ISODowExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getDayOfWeek());
+    }
+  }
+
+  private class ISOYearExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getWeekyear());
+    }
+  }
+
+  private class MillenniumExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) (((date.getYear() - 1) / 1000) + 1));
+    }
+  }
+
+  private class MonthExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getMonthOfYear());
+    }
+  }
+
+  private class QuarterExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) (((date.getMonthOfYear() - 1) / 3) + 1));
+    }
+  }
+
+  private class WeekExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getWeekOfWeekyear());
+    }
+  }
+
+  private class YearExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return DatumFactory.createFloat8((double) date.getYear());
+    }
+  }
+
+  private class NullExtractorFromDate implements DatePartExtractorFromDate {
+    @Override
+    public Datum extract(DateDatum date) {
+      return NullDatum.get();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
new file mode 100644
index 0000000..28e14fb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
@@ -0,0 +1,140 @@
+/**
+ * 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.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+
+@Description(
+    functionName = "date_part",
+    description = "Extract field from time",
+    example = "> SELECT date_part('second', time '10:09:37.5');\n"
+        + "37.5",
+    returnType = TajoDataTypes.Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TIME})}
+)
+public class DatePartFromTime extends GeneralFunction {
+  public DatePartFromTime() {
+    super(new Column[] {
+        new Column("target", FLOAT8),
+        new Column("source", TEXT)
+    });
+  }
+
+  private DatePartExtractorFromTime extractor = null;
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum target = params.get(0);
+    TimeDatum time = null;
+
+    if(target instanceof NullDatum || params.get(1) instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    if(params.get(1) instanceof TimeDatum) {
+      time = (TimeDatum)(params.get(1));
+    } else {
+      return NullDatum.get();
+    }
+
+    if (extractor == null) {
+      String extractType = target.asChars().toLowerCase();
+
+      if (extractType.equals("hour")) {
+        extractor = new HourExtractorFromTime();
+      } else if (extractType.equals("microseconds")) {
+        extractor = new MicrosecondsExtractorFromTime();
+      } else if (extractType.equals("milliseconds")) {
+        extractor = new MillisecondsExtractorFromTime();
+      } else if (extractType.equals("minute")) {
+        extractor = new MinuteExtractorFromTime();
+      } else if (extractType.equals("second")) {
+        extractor = new SecondExtractorFromTime();
+      } else if (extractType.equals("timezone")) {
+        extractor = new NullExtractorFromTime();
+      } else if (extractType.equals("timezone_hour")) {
+        extractor = new NullExtractorFromTime();
+      } else if (extractType.equals("timezone_minute")) {
+        extractor = new NullExtractorFromTime();
+      } else {
+        extractor = new NullExtractorFromTime();
+      }
+    }
+
+    return extractor.extract(time);
+  }
+
+  private interface DatePartExtractorFromTime {
+    public Datum extract(TimeDatum time);
+  }
+
+  private class HourExtractorFromTime implements DatePartExtractorFromTime {
+    @Override
+    public Datum extract(TimeDatum time) {
+      return DatumFactory.createFloat8((double) time.getHourOfDay());
+    }
+  }
+
+  private class MicrosecondsExtractorFromTime implements DatePartExtractorFromTime {
+    @Override
+    public Datum extract(TimeDatum time) {
+      return DatumFactory.createFloat8((double) (time.getSecondOfMinute() * 1000000 + time.getMillisOfSecond() * 1000));
+    }
+  }
+
+  private class MillisecondsExtractorFromTime implements DatePartExtractorFromTime {
+    @Override
+    public Datum extract(TimeDatum time) {
+      return DatumFactory.createFloat8((double) (time.getSecondOfMinute() * 1000 + time.getMillisOfSecond()));
+    }
+  }
+
+  private class MinuteExtractorFromTime implements DatePartExtractorFromTime {
+    @Override
+    public Datum extract(TimeDatum time) {
+      return DatumFactory.createFloat8((double) time.getMinuteOfHour());
+    }
+  }
+
+  private class SecondExtractorFromTime implements DatePartExtractorFromTime {
+    @Override
+    public Datum extract(TimeDatum time) {
+      if (time.getMillisOfSecond() != 0) {
+        return DatumFactory.createFloat8(time.getSecondOfMinute() + (((double) time.getMillisOfSecond()) / 1000));
+      } else {
+        return DatumFactory.createFloat8((double) time.getSecondOfMinute());
+      }
+    }
+  }
+
+  private class NullExtractorFromTime implements DatePartExtractorFromTime {
+    @Override
+    public Datum extract(TimeDatum time) {
+      return NullDatum.get();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
new file mode 100644
index 0000000..3b46929
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
@@ -0,0 +1,259 @@
+/**
+ * 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.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+
+@Description(
+    functionName = "date_part",
+    description = "Extract field from timestamp",
+    example = "> SELECT date_part('year', timestamp '2014-01-17 10:09:37.5');\n"
+        + "2014.0",
+    returnType = TajoDataTypes.Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TIMESTAMP})}
+)
+public class DatePartFromTimestamp extends GeneralFunction {
+  public DatePartFromTimestamp() {
+    super(new Column[] {
+        new Column("target", FLOAT8),
+        new Column("source", TEXT)
+    });
+  }
+
+  private DatePartExtractorFromTimestamp extractor = null;
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum target = params.get(0);
+    TimestampDatum timestamp = null;
+
+    if(target instanceof NullDatum || params.get(1) instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    if(params.get(1) instanceof TimestampDatum) {
+      timestamp = (TimestampDatum)(params.get(1));
+    } else {
+      return NullDatum.get();
+    }
+
+    if (extractor == null) {
+      String extractType = target.asChars().toLowerCase();
+
+      if (extractType.equals("century")) {
+        extractor = new CenturyExtractorFromTimestamp();
+      } else if (extractType.equals("day")) {
+        extractor = new DayExtractorFromTimestamp();
+      } else if (extractType.equals("decade")) {
+        extractor = new DecadeExtractorFromTimestamp();
+      } else if (extractType.equals("dow")) {
+        extractor = new DowExtractorFromTimestamp();
+      } else if (extractType.equals("doy")) {
+        extractor = new DoyExtractorFromTimestamp();
+      } else if (extractType.equals("epoch")) {
+        extractor = new EpochExtractorFromTimestamp();
+      } else if (extractType.equals("hour")) {
+        extractor = new HourExtractorFromTimestamp();
+      } else if (extractType.equals("isodow")) {
+        extractor = new ISODowExtractorFromTimestamp();
+      } else if (extractType.equals("isoyear")) {
+        extractor = new ISOYearExtractorFromTimestamp();
+      } else if (extractType.equals("microseconds")) {
+        extractor = new MicrosecondsExtractorFromTimestamp();
+      } else if (extractType.equals("millennium")) {
+        extractor = new MillenniumExtractorFromTimestamp();
+      } else if (extractType.equals("milliseconds")) {
+        extractor = new MillisecondsExtractorFromTimestamp();
+      } else if (extractType.equals("minute")) {
+        extractor = new MinuteExtractorFromTimestamp();
+      } else if (extractType.equals("month")) {
+        extractor = new MonthExtractorFromTimestamp();
+      } else if (extractType.equals("quarter")) {
+        extractor = new QuarterExtractorFromTimestamp();
+      } else if (extractType.equals("second")) {
+        extractor = new SecondExtractorFromTimestamp();
+      } else if (extractType.equals("timezone")) {
+        extractor = new NullExtractorFromTimestamp();
+      } else if (extractType.equals("timezone_hour")) {
+        extractor = new NullExtractorFromTimestamp();
+      } else if (extractType.equals("timezone_minute")) {
+        extractor = new NullExtractorFromTimestamp();
+      } else if (extractType.equals("week")) {
+        extractor = new WeekExtractorFromTimestamp();
+      } else if (extractType.equals("year")) {
+        extractor = new YearExtractorFromTimestamp();
+      } else {
+        extractor = new NullExtractorFromTimestamp();
+      }
+    }
+
+    return extractor.extract(timestamp);
+  }
+
+  private interface DatePartExtractorFromTimestamp {
+    public Datum extract(TimestampDatum timestamp);
+  }
+
+  private class CenturyExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getCenturyOfEra());
+    }
+  } 
+
+  private class DayExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getDayOfMonth());
+    }
+  }
+
+  private class DecadeExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) (timestamp.getYear() / 10));
+    }
+  }
+
+  private class DowExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      Integer tdow = timestamp.getDayOfWeek();
+      return DatumFactory.createFloat8((double) ((tdow == 7) ? 0 : tdow));
+    }
+  }
+
+  private class DoyExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getDayOfYear());
+    }
+  }
+
+  private class EpochExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getUnixTime());
+    }
+  }
+
+  private class HourExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getHourOfDay());
+    }
+  }
+
+  private class ISODowExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getDayOfWeek());
+    }
+  }
+
+  private class ISOYearExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getWeekyear());
+    }
+  }
+
+  private class MicrosecondsExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) (timestamp.getSecondOfMinute() * 1000000 + timestamp.getMillisOfSecond() * 1000));
+    }
+  }
+
+  private class MillenniumExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) (((timestamp.getYear() - 1) / 1000) + 1));
+    }
+  }
+
+  private class MillisecondsExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) (timestamp.getSecondOfMinute() * 1000 + timestamp.getMillisOfSecond()));
+    }
+  }
+
+  private class MinuteExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getMinuteOfHour());
+    }
+  }
+
+  private class MonthExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getMonthOfYear());
+    }
+  }
+
+  private class QuarterExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) (((timestamp.getMonthOfYear() - 1) / 3) + 1));
+    }
+  }
+
+  private class SecondExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      if (timestamp.getMillisOfSecond() != 0) {
+        return DatumFactory.createFloat8(timestamp.getSecondOfMinute() + (((double) timestamp.getMillisOfSecond()) / 1000));
+      } else {
+        return DatumFactory.createFloat8((double) timestamp.getSecondOfMinute());
+      }
+    }
+  }
+
+  private class WeekExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getWeekOfWeekyear());
+    }
+  }
+
+  private class YearExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return DatumFactory.createFloat8((double) timestamp.getYear());
+    }
+  }
+
+  private class NullExtractorFromTimestamp implements DatePartExtractorFromTimestamp {
+    @Override
+    public Datum extract(TimestampDatum timestamp) {
+      return NullDatum.get();
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DateTimePartFromUnixTimeStamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DateTimePartFromUnixTimeStamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DateTimePartFromUnixTimeStamp.java
new file mode 100644
index 0000000..6aaded0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DateTimePartFromUnixTimeStamp.java
@@ -0,0 +1,141 @@
+/**
+ * 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.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.TimeStampUtil;
+import org.joda.time.DateTime;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+
+
+@Description(
+        functionName = "utc_usec_to",
+        description = "Extract field from time",
+        example = "> SELECT utc_usec_to('day', 1274259481071200);\n"
+                + "1274227200000000",
+        returnType = TajoDataTypes.Type.INT8,
+        paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.INT8}),
+                @ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.INT8, TajoDataTypes.Type.INT4})}
+)
+public class DateTimePartFromUnixTimeStamp extends GeneralFunction {
+
+    private DateTimePartExtractorFromUnixTime extractor = null;
+    private WeekPartExtractorFromUnixTime weekExtractor = null;
+
+    public DateTimePartFromUnixTimeStamp() {
+        super(new Column[]{
+                new Column("target", TEXT),
+                new Column("source", INT8),
+                new Column("dayOfWeek", INT4),
+
+        });
+    }
+
+    @Override
+    public Datum eval(Tuple params) {
+
+        Datum target = params.get(0);
+        DateTime dateTime;
+        Int4Datum dayOfWeek = null;
+
+        if (target instanceof NullDatum || params.get(1) instanceof NullDatum) {
+            return NullDatum.get();
+        }
+
+        if (params.get(1) instanceof Int8Datum) {
+            dateTime = TimeStampUtil.getUTCDateTime((Int8Datum) (params.get(1)));
+        } else {
+            return NullDatum.get();
+        }
+
+
+        if ( null == extractor || null == weekExtractor) {
+
+            String extractType = target.asChars().toLowerCase();
+
+            if (extractType.equals("day")) {
+                extractor = new DayExtractorFromTime();
+            } else if (extractType.equals("hour")) {
+                extractor = new HourExtractorFromTime();
+            } else if (extractType.equals("month")) {
+                extractor = new MonthExtractorFromTime();
+            } else if (extractType.equals("year")) {
+                extractor = new YearExtractorFromTime();
+            } else if (extractType.equals("week")) {
+                if (params.get(2) instanceof NullDatum) {
+                    return NullDatum.get();
+                }
+                dayOfWeek = (Int4Datum) params.get(2);
+                weekExtractor = new WeekExtractorFromTime();
+            }
+        }
+
+        return null != weekExtractor ? weekExtractor.extract(dateTime, dayOfWeek.asInt4()) : extractor.extract(dateTime);
+    }
+
+    private interface DateTimePartExtractorFromUnixTime {
+        public Datum extract(DateTime dateTime);
+    }
+
+    private interface WeekPartExtractorFromUnixTime {
+        public Datum extract(DateTime dateTime, int week);
+    }
+
+    private class DayExtractorFromTime implements DateTimePartExtractorFromUnixTime {
+        @Override
+        public Datum extract(DateTime dateTime) {
+            return DatumFactory.createInt8(TimeStampUtil.getDay(dateTime));
+        }
+    }
+
+    private class HourExtractorFromTime implements DateTimePartExtractorFromUnixTime {
+        @Override
+        public Datum extract(DateTime dateTime) {
+            return DatumFactory.createInt8(TimeStampUtil.getHour(dateTime));
+        }
+    }
+
+    private class MonthExtractorFromTime implements DateTimePartExtractorFromUnixTime {
+        @Override
+        public Datum extract(DateTime dateTime) {
+            return DatumFactory.createInt8(TimeStampUtil.getMonth(dateTime));
+        }
+    }
+
+    private class YearExtractorFromTime implements DateTimePartExtractorFromUnixTime {
+        @Override
+        public Datum extract(DateTime dateTime) {
+            return DatumFactory.createInt8(TimeStampUtil.getYear(dateTime));
+        }
+    }
+
+    private class WeekExtractorFromTime implements WeekPartExtractorFromUnixTime {
+        @Override
+        public Datum extract(DateTime dateTime , int week) {
+            return DatumFactory.createInt8(TimeStampUtil.getDayOfWeek(dateTime,week));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
new file mode 100644
index 0000000..2a74ff5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
@@ -0,0 +1,79 @@
+/**
+ * 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.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+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.datum.TimestampDatum;
+import org.apache.tajo.engine.eval.FunctionEval;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.INT8;
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+@Description(
+  functionName = "to_char",
+  description = "Convert time stamp to string",
+  example = "> SELECT to_char(1389071652, 'yyyy-MM');\n"
+          + "2014-01",
+  returnType = TajoDataTypes.Type.TEXT,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TIMESTAMP, TajoDataTypes.Type.TEXT})}
+)
+public class ToCharTimestamp extends GeneralFunction {
+  private boolean constantFormat;
+  private DateTimeFormatter formatter;
+
+  public ToCharTimestamp() {
+    super(new Column[] {
+        new Column("timestamp", INT8),
+        new Column("format", TEXT)
+    });
+  }
+
+  @Override
+  public void init(FunctionEval.ParamType[] paramTypes) {
+    if (paramTypes[1] == FunctionEval.ParamType.CONSTANT) {
+      constantFormat = true;
+    }
+  }
+
+
+  @Override
+  public Datum eval(Tuple params) {
+    if(params.isNull(0) || params.isNull(1)) {
+      return NullDatum.get();
+    }
+
+    TimestampDatum valueDatum = (TimestampDatum) params.get(0);
+    Datum pattern = params.get(1);
+
+    if (formatter == null || !constantFormat) {
+      formatter = DateTimeFormat.forPattern(pattern.asChars());
+    }
+    return DatumFactory.createText(valueDatum.toChars(formatter));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java
new file mode 100644
index 0000000..1cf6870
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java
@@ -0,0 +1,55 @@
+/**
+ * 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.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.INT4;
+
+@Description(
+  functionName = "to_timestamp",
+  description = "Convert UNIX epoch to time stamp",
+  example = "> SELECT to_timestamp(1389071574);\n"
+        + "2014-01-07 14:12:54",
+  returnType = TajoDataTypes.Type.TIMESTAMP,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4}),
+      @ParamTypes(paramTypes = {TajoDataTypes.Type.INT8})}
+)
+public class ToTimestamp extends GeneralFunction {
+  public ToTimestamp() {
+    super(new Column[] {new Column("timestamp", INT4)});
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum value = params.get(0);
+    if (value instanceof NullDatum) {
+      return NullDatum.get();
+    }
+    return DatumFactory.createTimeStamp(value.asInt4());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryInet4.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryInet4.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryInet4.java
new file mode 100644
index 0000000..d922e76
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryInet4.java
@@ -0,0 +1,55 @@
+/*
+ * 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.function.geoip;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.GeoIPUtil;
+
+@Description(
+    functionName = "geoip_country_code",
+    description = "Convert an ipv4 address to a geoip country code.",
+    example = "> SELECT geoip_country_code(8.8.8.8);\n"
+        + "US",
+    returnType = TajoDataTypes.Type.TEXT,
+    paramTypes = {@ParamTypes(paramTypes = {Type.INET4})}
+)
+public class GeoIPCountryInet4 extends GeneralFunction {
+
+  public GeoIPCountryInet4() {
+    super(new Column[] {new Column("ipv4_address", TajoDataTypes.Type.INET4)});
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if (valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+    return new TextDatum(GeoIPUtil.getCountryCode(params.get(0).asChars()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryText.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryText.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryText.java
new file mode 100644
index 0000000..17a43be
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPCountryText.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.geoip;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.GeoIPUtil;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+@Description(
+    functionName = "geoip_country_code",
+    description = "Convert an ipv4 address string to a geoip country code.",
+    example = "> SELECT geoip_country_code('8.8.8.8');\n"
+        + "US",
+    returnType = TajoDataTypes.Type.TEXT,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+)
+public class GeoIPCountryText extends GeneralFunction {
+
+  public GeoIPCountryText() {
+    super(new Column[] {new Column("ipv4_address_string", TEXT)});
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if (valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+    return new TextDatum(GeoIPUtil.getCountryCode(params.get(0).asChars()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryInet4.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryInet4.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryInet4.java
new file mode 100644
index 0000000..19ce8ba
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryInet4.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.geoip;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.GeoIPUtil;
+
+@Description(
+    functionName = "geoip_in_country",
+    description = "If the given country code is same with the country code of the given address, it returns true. "
+        + "Otherwise, returns false",
+    example = "geoip_in_country(8.8.8.8, 'US')"
+        + "true",
+    returnType = TajoDataTypes.Type.BOOLEAN,
+    paramTypes = {@ParamTypes(paramTypes = {Type.INET4, Type.TEXT})}
+)
+public class GeoIPInCountryInet4 extends GeneralFunction {
+
+  public GeoIPInCountryInet4() {
+    super(new Column[] {new Column("ipv4_address", TajoDataTypes.Type.INET4),
+        new Column("country_code", TajoDataTypes.Type.TEXT)});
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    if (params.get(0) instanceof NullDatum || params.get(1) instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String addr = params.get(0).asChars();
+    String otherCode = params.get(1).asChars();
+    String thisCode = GeoIPUtil.getCountryCode(addr);
+
+    return DatumFactory.createBool(thisCode.equals(otherCode));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryText.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryText.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryText.java
new file mode 100644
index 0000000..168f86c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/geoip/GeoIPInCountryText.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.geoip;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.GeoIPUtil;
+
+@Description(
+    functionName = "geoip_in_country",
+    description = "If the given country code is same with the country code of the given address, it returns true. "
+        + "Otherwise, returns false",
+    example = "geoip_in_country('8.8.8.8', 'US')"
+        + "true",
+    returnType = TajoDataTypes.Type.BOOLEAN,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
+)
+public class GeoIPInCountryText extends GeneralFunction {
+
+  public GeoIPInCountryText() {
+    super(new Column[] {new Column("ipv4_address_string", TajoDataTypes.Type.TEXT),
+        new Column("country_code", TajoDataTypes.Type.TEXT)});
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    if (params.get(0) instanceof NullDatum || params.get(1) instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String addr = params.get(0).asChars();
+    String otherCode = params.get(1).asChars();
+    String thisCode = GeoIPUtil.getCountryCode(addr);
+
+    return DatumFactory.createBool(thisCode.equals(otherCode));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsDouble.java
new file mode 100644
index 0000000..1ff2f29
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsDouble.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT8 abs(value FLOAT8)
+ */
+@Description(
+  functionName = "abs",
+  description = "Absolute value",
+  detail = "",
+  example = "> SELECT abs(-10);\n"
+          + "10",
+  returnType = TajoDataTypes.Type.FLOAT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT8})}
+)
+public class AbsDouble extends GeneralFunction {
+  public AbsDouble() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.abs(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsFloat.java
new file mode 100644
index 0000000..4e3f54c
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsFloat.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * FLOAT4 abs(value FLOAT4)
+ */
+@Description(
+  functionName = "abs",
+  description = "Absolute value",
+  detail = "",
+  example = "> SELECT abs(-10);\n"
+      + "10",
+  returnType = TajoDataTypes.Type.FLOAT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.FLOAT4})}
+)
+public class AbsFloat extends GeneralFunction {
+  public AbsFloat() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.FLOAT4)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat4(Math.abs(valueDatum.asFloat4()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsInt.java
new file mode 100644
index 0000000..b1fb79e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsInt.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT4 abs(value INT4)
+ */
+@Description(
+  functionName = "abs",
+  description = "Absolute value",
+  detail = "",
+  example = "> SELECT abs(-10);\n"
+      + "10",
+  returnType = TajoDataTypes.Type.INT4,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT4})}
+)
+public class AbsInt extends GeneralFunction {
+  public AbsInt() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt4(Math.abs(valueDatum.asInt4()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsLong.java
new file mode 100644
index 0000000..cdb5a29
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/math/AbsLong.java
@@ -0,0 +1,60 @@
+/**
+ * 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.function.math;
+
+import org.apache.tajo.catalog.Column;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 abs(value INT8)
+ */
+@Description(
+  functionName = "abs",
+  description = "Absolute value",
+  example = "> SELECT abs(-10);\n"
+      + "10",
+  returnType = TajoDataTypes.Type.INT8,
+  paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT8})}
+)
+public class AbsLong extends GeneralFunction {
+  public AbsLong() {
+    super(new Column[] {
+      new Column("x", TajoDataTypes.Type.INT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8(Math.abs(valueDatum.asInt8()));
+  }
+}


[10/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java b/tajo-core/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
new file mode 100644
index 0000000..88d913d
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
@@ -0,0 +1,175 @@
+/**
+ * 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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.LocalContainerLauncher;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.util.JarFinder;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.pullserver.PullServerAuxService;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+/**
+ * Configures and starts the Tajo-specific components in the YARN cluster.
+ *
+ */
+public class MiniTajoYarnCluster extends MiniYARNCluster {
+
+  public static final String APPJAR = JarFinder
+      .getJar(LocalContainerLauncher.class);
+
+  private static final Log LOG = LogFactory.getLog(MiniTajoYarnCluster.class);
+
+  public MiniTajoYarnCluster(String testName) {
+    this(testName, 1);
+  }
+
+  public MiniTajoYarnCluster(String testName, int noOfNMs) {
+    super(testName, noOfNMs, 1, 1);
+  }
+
+  @Override
+  public void init(Configuration conf) {
+
+    conf.setSocketAddr(YarnConfiguration.RM_ADDRESS, new InetSocketAddress("127.0.0.1", 0));
+    conf.setSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS, new InetSocketAddress("127.0.0.1", 0));
+
+    conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
+    if (conf.get(MRJobConfig.MR_AM_STAGING_DIR) == null) {
+      conf.set(MRJobConfig.MR_AM_STAGING_DIR, new File(getTestWorkDir(),
+          "apps_staging_dir/").getAbsolutePath());
+    }
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY,  "000");
+
+    try {
+      Path stagingPath = FileContext.getFileContext(conf).makeQualified(
+          new Path(conf.get(MRJobConfig.MR_AM_STAGING_DIR)));
+      FileContext fc=FileContext.getFileContext(stagingPath.toUri(), conf);
+      if (fc.util().exists(stagingPath)) {
+        LOG.info(stagingPath + " exists! deleting...");
+        fc.delete(stagingPath, true);
+      }
+      LOG.info("mkdir: " + stagingPath);
+      //mkdir the staging directory so that right permissions are set while running as proxy user
+      fc.mkdir(stagingPath, null, true);
+      //mkdir done directory as well
+      String doneDir = JobHistoryUtils
+          .getConfiguredHistoryServerDoneDirPrefix(conf);
+      Path doneDirPath = fc.makeQualified(new Path(doneDir));
+      fc.mkdir(doneDirPath, null, true);
+    } catch (IOException e) {
+      throw new YarnRuntimeException("Could not create staging directory. ", e);
+    }
+    conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of
+    // which shuffle doesn't happen
+    //configure the shuffle service in NM
+    conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, PullServerAuxService.PULLSERVER_SERVICEID);
+    conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT,
+        PullServerAuxService.PULLSERVER_SERVICEID), PullServerAuxService.class,
+        Service.class);
+
+    // Non-standard shuffle port
+    conf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.name(), 0);
+
+    // local directory
+    conf.set(TajoConf.ConfVars.WORKER_TEMPORAL_DIR.name(), "/tmp/tajo-localdir");
+
+    conf.setClass(YarnConfiguration.NM_CONTAINER_EXECUTOR,
+        DefaultContainerExecutor.class, ContainerExecutor.class);
+
+    // TestMRJobs is for testing non-uberized operation only; see TestUberAM
+    // for corresponding uberized tests.
+    conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
+
+    conf.setInt("yarn.nodemanager.delete.debug-delay-sec", 600);
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+
+    LOG.info("MiniTajoYarn NM Local Dir: " + getConfig().get(YarnConfiguration.NM_LOCAL_DIRS));
+  }
+
+  private class JobHistoryServerWrapper extends AbstractService {
+    public JobHistoryServerWrapper() {
+      super(JobHistoryServerWrapper.class.getName());
+    }
+
+    @Override
+    public synchronized void start() {
+      try {
+        if (!getConfig().getBoolean(
+            JHAdminConfig.MR_HISTORY_MINICLUSTER_FIXED_PORTS,
+            JHAdminConfig.DEFAULT_MR_HISTORY_MINICLUSTER_FIXED_PORTS)) {
+          // pick free random ports.
+          getConfig().set(JHAdminConfig.MR_HISTORY_ADDRESS,
+              MiniYARNCluster.getHostname() + ":0");
+          getConfig().set(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS,
+              MiniYARNCluster.getHostname() + ":0");
+        }
+        super.start();
+      } catch (Throwable t) {
+        throw new YarnRuntimeException(t);
+      }
+
+      LOG.info("MiniMRYARN ResourceManager address: " +
+          getConfig().get(YarnConfiguration.RM_ADDRESS));
+      LOG.info("MiniMRYARN ResourceManager web address: " +
+          getConfig().get(YarnConfiguration.RM_WEBAPP_ADDRESS));
+      LOG.info("MiniMRYARN HistoryServer address: " +
+          getConfig().get(JHAdminConfig.MR_HISTORY_ADDRESS));
+      LOG.info("MiniMRYARN HistoryServer web address: " +
+          getConfig().get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS));
+    }
+
+    @Override
+    public synchronized void stop() {
+      super.stop();
+    }
+  }
+
+  public static void main(String [] args) {
+    MiniTajoYarnCluster cluster = new MiniTajoYarnCluster(MiniTajoYarnCluster.class.getName());
+    cluster.init(new TajoConf());
+    cluster.start();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
new file mode 100644
index 0000000..961184c
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -0,0 +1,539 @@
+/**
+ * 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;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.cli.ParsedResult;
+import org.apache.tajo.cli.SimpleParser;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.util.FileUtil;
+import org.junit.*;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.*;
+
+import static org.junit.Assert.*;
+
+/**
+ * (Note that this class is not thread safe. Do not execute maven test in any parallel mode.)
+ * <br />
+ * <code>QueryTestCaseBase</code> provides useful methods to easily execute queries and verify their results.
+ *
+ * This class basically uses four resource directories:
+ * <ul>
+ *   <li>src/test/resources/dataset - contains a set of data files. It contains sub directories, each of which
+ *   corresponds each test class. All data files in each sub directory can be used in the corresponding test class.</li>
+ *
+ *   <li>src/test/resources/queries - This is the query directory. It contains sub directories, each of which
+ *   corresponds each test class. All query files in each sub directory can be used in the corresponding test
+ *   class.</li>
+ *
+ *   <li>src/test/resources/results - This is the result directory. It contains sub directories, each of which
+ *   corresponds each test class. All result files in each sub directory can be used in the corresponding test class.
+ *   </li>
+ * </ul>
+ *
+ * For example, if you create a test class named <code>TestJoinQuery</code>, you should create a pair of query and
+ * result set directories as follows:
+ *
+ * <pre>
+ *   src-|
+ *       |- resources
+ *             |- dataset
+ *             |     |- TestJoinQuery
+ *             |              |- table1.tbl
+ *             |              |- table2.tbl
+ *             |
+ *             |- queries
+ *             |     |- TestJoinQuery
+ *             |              |- TestInnerJoin.sql
+ *             |              |- table1_ddl.sql
+ *             |              |- table2_ddl.sql
+ *             |
+ *             |- results
+ *                   |- TestJoinQuery
+ *                            |- TestInnerJoin.result
+ * </pre>
+ *
+ * <code>QueryTestCaseBase</code> basically provides the following methods:
+ * <ul>
+ *  <li><code>{@link #executeQuery()}</code> - executes a corresponding query and returns an ResultSet instance</li>
+ *  <li><code>{@link #executeFile(String)}</code> - executes a given query file included in the corresponding query
+ *  file in the current class's query directory</li>
+ *  <li><code>assertResultSet()</code> - check if the query result is equivalent to the expected result included
+ *  in the corresponding result file in the current class's result directory.</li>
+ *  <li><code>cleanQuery()</code> - clean up all resources</li>
+ *  <li><code>executeDDL()</code> - execute a DDL query like create or drop table.</li>
+ * </ul>
+ *
+ * In order to make use of the above methods, query files and results file must be as follows:
+ * <ul>
+ *  <li>Each query file must be located on the subdirectory whose structure must be src/resources/queries/${ClassName},
+ *  where ${ClassName} indicates an actual test class's simple name.</li>
+ *  <li>Each result file must be located on the subdirectory whose structure must be src/resources/results/${ClassName},
+ *  where ${ClassName} indicates an actual test class's simple name.</li>
+ * </ul>
+ *
+ * Especially, {@link #executeQuery() and {@link #assertResultSet(java.sql.ResultSet)} methods automatically finds
+ * a query file to be executed and a result to be compared, which are corresponding to the running class and method.
+ * For them, query and result files additionally must be follows as:
+ * <ul>
+ *  <li>Each result file must have the file extension '.result'</li>
+ *  <li>Each query file must have the file extension '.sql'.</li>
+ * </ul>
+ */
+public class QueryTestCaseBase {
+  private static final Log LOG = LogFactory.getLog(QueryTestCaseBase.class);
+  protected static final TpchTestBase testBase;
+  protected static final TajoTestingCluster testingCluster;
+  protected static TajoConf conf;
+  protected static TajoClient client;
+  protected static final CatalogService catalog;
+  protected static final SQLAnalyzer sqlParser = new SQLAnalyzer();
+
+  /** the base path of dataset directories */
+  protected static final Path datasetBasePath;
+  /** the base path of query directories */
+  protected static final Path queryBasePath;
+  /** the base path of result directories */
+  protected static final Path resultBasePath;
+
+  static {
+    testBase = TpchTestBase.getInstance();
+    testingCluster = testBase.getTestingCluster();
+    conf = testBase.getTestingCluster().getConfiguration();
+    catalog = testBase.getTestingCluster().getMaster().getCatalog();
+    URL datasetBaseURL = ClassLoader.getSystemResource("dataset");
+    datasetBasePath = new Path(datasetBaseURL.toString());
+    URL queryBaseURL = ClassLoader.getSystemResource("queries");
+    queryBasePath = new Path(queryBaseURL.toString());
+    URL resultBaseURL = ClassLoader.getSystemResource("results");
+    resultBasePath = new Path(resultBaseURL.toString());
+  }
+
+  /** It transiently contains created tables for the running test class. */
+  private static String currentDatabase;
+  private static Set<String> createdTableGlobalSet = new HashSet<String>();
+  // queries and results directory corresponding to subclass class.
+  private Path currentQueryPath;
+  private Path currentResultPath;
+  private Path currentDatasetPath;
+
+  // for getting a method name
+  @Rule public TestName name = new TestName();
+
+  @BeforeClass
+  public static void setUpClass() throws IOException {
+    conf = testBase.getTestingCluster().getConfiguration();
+    client = new TajoClient(conf);
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws ServiceException {
+    for (String tableName : createdTableGlobalSet) {
+      client.updateQuery("DROP TABLE IF EXISTS " + CatalogUtil.denormalizeIdentifier(tableName));
+    }
+    createdTableGlobalSet.clear();
+
+    // if the current database is "default", shouldn't drop it.
+    if (!currentDatabase.equals(TajoConstants.DEFAULT_DATABASE_NAME)) {
+      for (String tableName : catalog.getAllTableNames(currentDatabase)) {
+        client.updateQuery("DROP TABLE IF EXISTS " + tableName);
+      }
+
+      client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME);
+      client.dropDatabase(currentDatabase);
+    }
+    client.close();
+  }
+
+  public QueryTestCaseBase() {
+    // hive 0.12 does not support quoted identifier.
+    // So, we use lower case database names when Tajo uses HCatalogStore.
+    if (testingCluster.isHCatalogStoreRunning()) {
+      this.currentDatabase = getClass().getSimpleName().toLowerCase();
+    } else {
+      this.currentDatabase = getClass().getSimpleName();
+    }
+    init();
+  }
+
+  public QueryTestCaseBase(String currentDatabase) {
+    this.currentDatabase = currentDatabase;
+    init();
+  }
+
+  private void init() {
+    String className = getClass().getSimpleName();
+    currentQueryPath = new Path(queryBasePath, className);
+    currentResultPath = new Path(resultBasePath, className);
+    currentDatasetPath = new Path(datasetBasePath, className);
+
+    try {
+      // if the current database is "default", we don't need create it because it is already prepated at startup time.
+      if (!currentDatabase.equals(TajoConstants.DEFAULT_DATABASE_NAME)) {
+        client.updateQuery("CREATE DATABASE IF NOT EXISTS " + CatalogUtil.denormalizeIdentifier(currentDatabase));
+      }
+      client.selectDatabase(currentDatabase);
+    } catch (ServiceException e) {
+      e.printStackTrace();
+    }
+    testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname, "false");
+  }
+
+  protected TajoClient getClient() {
+    return client;
+  }
+
+  public String getCurrentDatabase() {
+    return currentDatabase;
+  }
+
+  protected ResultSet executeString(String sql) throws Exception {
+    return testBase.execute(sql);
+  }
+
+  /**
+   * Execute a query contained in the file located in src/test/resources/results/<i>ClassName</i>/<i>MethodName</i>.
+   * <i>ClassName</i> and <i>MethodName</i> will be replaced by actual executed class and methods.
+   *
+   * @return ResultSet of query execution.
+   */
+  public ResultSet executeQuery() throws Exception {
+    return executeFile(name.getMethodName() + ".sql");
+  }
+
+  /**
+   * Execute a query contained in the given named file. This methods tries to find the given file within the directory
+   * src/test/resources/results/<i>ClassName</i>.
+   *
+   * @param queryFileName The file name to be used to execute a query.
+   * @return ResultSet of query execution.
+   */
+  public ResultSet executeFile(String queryFileName) throws Exception {
+    Path queryFilePath = getQueryFilePath(queryFileName);
+    FileSystem fs = currentQueryPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
+    assertTrue(queryFilePath.toString() + " existence check", fs.exists(queryFilePath));
+
+    List<ParsedResult> parsedResults = SimpleParser.parseScript(FileUtil.readTextFile(new File(queryFilePath.toUri())));
+    if (parsedResults.size() > 1) {
+      assertNotNull("This script \"" + queryFileName + "\" includes two or more queries");
+    }
+    ResultSet result = client.executeQueryAndGetResult(parsedResults.get(0).getStatement());
+    assertNotNull("Query succeeded test", result);
+    return result;
+  }
+
+  /**
+   * Assert the equivalence between the expected result and an actual query result.
+   * If it isn't it throws an AssertionError.
+   *
+   * @param result Query result to be compared.
+   */
+  public final void assertResultSet(ResultSet result) throws IOException {
+    assertResultSet("Result Verification", result, name.getMethodName() + ".result");
+  }
+
+  /**
+   * Assert the equivalence between the expected result and an actual query result.
+   * If it isn't it throws an AssertionError.
+   *
+   * @param result Query result to be compared.
+   * @param resultFileName The file name containing the result to be compared
+   */
+  public final void assertResultSet(ResultSet result, String resultFileName) throws IOException {
+    assertResultSet("Result Verification", result, resultFileName);
+  }
+
+  /**
+   * Assert the equivalence between the expected result and an actual query result.
+   * If it isn't it throws an AssertionError with the given message.
+   *
+   * @param message message The message to printed if the assertion is failed.
+   * @param result Query result to be compared.
+   */
+  public final void assertResultSet(String message, ResultSet result, String resultFileName) throws IOException {
+    FileSystem fs = currentQueryPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
+    Path resultFile = getResultFile(resultFileName);
+    assertTrue(resultFile.toString() + " existence check", fs.exists(resultFile));
+    try {
+      verifyResultText(message, result, resultFile);
+    } catch (SQLException e) {
+      throw new IOException(e);
+    }
+  }
+
+  public final void assertStrings(String actual) throws IOException {
+    assertStrings(actual, name.getMethodName() + ".result");
+  }
+
+  public final void assertStrings(String actual, String resultFileName) throws IOException {
+    assertStrings("Result Verification", actual, resultFileName);
+  }
+
+  public final void assertStrings(String message, String actual, String resultFileName) throws IOException {
+    FileSystem fs = currentQueryPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
+    Path resultFile = getResultFile(resultFileName);
+    assertTrue(resultFile.toString() + " existence check", fs.exists(resultFile));
+
+    String expectedResult = FileUtil.readTextFile(new File(resultFile.toUri()));
+    assertEquals(message, expectedResult, actual);
+  }
+
+  /**
+   * Release all resources
+   *
+   * @param resultSet ResultSet
+   */
+  public final void cleanupQuery(ResultSet resultSet) throws IOException {
+    if (resultSet == null) {
+      return;
+    }
+    try {
+      resultSet.close();
+    } catch (SQLException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Assert that the database exists.
+   * @param databaseName The database name to be checked. This name is case sensitive.
+   */
+  public void assertDatabaseExists(String databaseName) throws ServiceException {
+    assertTrue(client.existDatabase(databaseName));
+  }
+
+  /**
+   * Assert that the database does not exists.
+   * @param databaseName The database name to be checked. This name is case sensitive.
+   */
+  public void assertDatabaseNotExists(String databaseName) throws ServiceException {
+    assertTrue(!client.existDatabase(databaseName));
+  }
+
+  /**
+   * Assert that the table exists.
+   *
+   * @param tableName The table name to be checked. This name is case sensitive.
+   * @throws ServiceException
+   */
+  public void assertTableExists(String tableName) throws ServiceException {
+    assertTrue(client.existTable(tableName));
+  }
+
+  /**
+   * Assert that the table does not exist.
+   *
+   * @param tableName The table name to be checked. This name is case sensitive.
+   */
+  public void assertTableNotExists(String tableName) throws ServiceException {
+    assertTrue(!client.existTable(tableName));
+  }
+
+  public void assertColumnExists(String tableName,String columnName) throws ServiceException {
+    TableDesc tableDesc = fetchTableMetaData(tableName);
+    assertTrue(tableDesc.getSchema().containsByName(columnName));
+  }
+
+  private TableDesc fetchTableMetaData(String tableName) throws ServiceException {
+    return client.getTableDesc(tableName);
+  }
+
+  /**
+   * It transforms a ResultSet instance to rows represented as strings.
+   *
+   * @param resultSet ResultSet that contains a query result
+   * @return String
+   * @throws SQLException
+   */
+  public String resultSetToString(ResultSet resultSet) throws SQLException {
+    StringBuilder sb = new StringBuilder();
+    ResultSetMetaData rsmd = resultSet.getMetaData();
+    int numOfColumns = rsmd.getColumnCount();
+
+    for (int i = 1; i <= numOfColumns; i++) {
+      if (i > 1) sb.append(",");
+      String columnName = rsmd.getColumnName(i);
+      sb.append(columnName);
+    }
+    sb.append("\n-------------------------------\n");
+
+    while (resultSet.next()) {
+      for (int i = 1; i <= numOfColumns; i++) {
+        if (i > 1) sb.append(",");
+        String columnValue = resultSet.getObject(i).toString();
+        sb.append(columnValue);
+      }
+      sb.append("\n");
+    }
+    return sb.toString();
+  }
+
+  private void verifyResultText(String message, ResultSet res, Path resultFile) throws SQLException, IOException {
+    String actualResult = resultSetToString(res);
+    String expectedResult = FileUtil.readTextFile(new File(resultFile.toUri()));
+    assertEquals(message, expectedResult.trim(), actualResult.trim());
+  }
+
+  private Path getQueryFilePath(String fileName) {
+    return StorageUtil.concatPath(currentQueryPath, fileName);
+  }
+
+  private Path getResultFile(String fileName) {
+    return StorageUtil.concatPath(currentResultPath, fileName);
+  }
+
+  private Path getDataSetFile(String fileName) {
+    return StorageUtil.concatPath(currentDatasetPath, fileName);
+  }
+
+  public List<String> executeDDL(String ddlFileName, @Nullable String [] args) throws Exception {
+    return executeDDL(ddlFileName, null, true, args);
+  }
+
+  /**
+   *
+   * Execute a data definition language (DDL) template. A general SQL DDL statement can be included in this file. But,
+   * for user-specified table name or exact external table path, you must use some format string to indicate them.
+   * The format string will be replaced by the corresponding arguments.
+   *
+   * The below is predefined format strings:
+   * <ul>
+   *   <li>${table.path} - It is replaced by the absolute file path that <code>dataFileName</code> points. </li>
+   *   <li>${i} - It is replaced by the corresponding element of <code>args</code>. For example, ${0} and ${1} are
+   *   replaced by the first and second elements of <code>args</code> respectively</li>. It uses zero-based index.
+   * </ul>
+   *
+   * @param ddlFileName A file name, containing a data definition statement.
+   * @param dataFileName A file name, containing data rows, which columns have to be separated by vertical bar '|'.
+   *                     This file name is used for replacing some format string indicating an external table location.
+   * @param args A list of arguments, each of which is used to replace corresponding variable which has a form of ${i}.
+   * @return The table names created
+   */
+  public List<String> executeDDL(String ddlFileName, @Nullable String dataFileName, @Nullable String ... args)
+      throws Exception {
+
+    return executeDDL(ddlFileName, dataFileName, true, args);
+  }
+
+  private List<String> executeDDL(String ddlFileName, @Nullable String dataFileName, boolean isLocalTable,
+                                  @Nullable String[] args) throws Exception {
+
+    Path ddlFilePath = new Path(currentQueryPath, ddlFileName);
+    FileSystem fs = ddlFilePath.getFileSystem(conf);
+    assertTrue(ddlFilePath + " existence check", fs.exists(ddlFilePath));
+
+    String template = FileUtil.readTextFile(new File(ddlFilePath.toUri()));
+    String dataFilePath = null;
+    if (dataFileName != null) {
+      dataFilePath = getDataSetFile(dataFileName).toString();
+    }
+    String compiled = compileTemplate(template, dataFilePath, args);
+
+    List<ParsedResult> parsedResults = SimpleParser.parseScript(compiled);
+    List<String> createdTableNames = new ArrayList<String>();
+
+    for (ParsedResult parsedResult : parsedResults) {
+      // parse a statement
+      Expr expr = sqlParser.parse(parsedResult.getStatement());
+      assertNotNull(ddlFilePath + " cannot be parsed", expr);
+
+      if (expr.getType() == OpType.CreateTable) {
+        CreateTable createTable = (CreateTable) expr;
+        String tableName = createTable.getTableName();
+        assertTrue("Table [" + tableName + "] creation is failed.", client.updateQuery(parsedResult.getStatement()));
+
+        TableDesc createdTable = client.getTableDesc(tableName);
+        String createdTableName = createdTable.getName();
+
+        assertTrue("table '" + createdTableName + "' creation check", client.existTable(createdTableName));
+        if (isLocalTable) {
+          createdTableGlobalSet.add(createdTableName);
+          createdTableNames.add(tableName);
+        }
+      } else if (expr.getType() == OpType.DropTable) {
+        DropTable dropTable = (DropTable) expr;
+        String tableName = dropTable.getTableName();
+        assertTrue("table '" + tableName + "' existence check",
+            client.existTable(CatalogUtil.buildFQName(currentDatabase, tableName)));
+        assertTrue("table drop is failed.", client.updateQuery(parsedResult.getStatement()));
+        assertFalse("table '" + tableName + "' dropped check",
+            client.existTable(CatalogUtil.buildFQName(currentDatabase, tableName)));
+        if (isLocalTable) {
+          createdTableGlobalSet.remove(tableName);
+        }
+      } else if (expr.getType() == OpType.AlterTable) {
+        AlterTable alterTable = (AlterTable) expr;
+        String tableName = alterTable.getTableName();
+        assertTrue("table '" + tableName + "' existence check", client.existTable(tableName));
+        client.updateQuery(compiled);
+        if (isLocalTable) {
+          createdTableGlobalSet.remove(tableName);
+        }
+      } else {
+        assertTrue(ddlFilePath + " is not a Create or Drop Table statement", false);
+      }
+    }
+
+    return createdTableNames;
+  }
+
+  /**
+   * Replace format strings by a given parameters.
+   *
+   * @param template
+   * @param dataFileName The data file name to replace <code>${table.path}</code>
+   * @param args The list argument to replace each corresponding format string ${i}. ${i} uses zero-based index.
+   * @return A string compiled
+   */
+  private String compileTemplate(String template, @Nullable String dataFileName, @Nullable String ... args) {
+    String result;
+    if (dataFileName != null) {
+      result = template.replace("${table.path}", "\'" + dataFileName + "'");
+    } else {
+      result = template;
+    }
+
+    if (args != null) {
+      for (int i = 0; i < args.length; i++) {
+        result = result.replace("${" + i + "}", args[i]);
+      }
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
new file mode 100644
index 0000000..ed5e4bc
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -0,0 +1,630 @@
+/**
+ * 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;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.io.Closeables;
+import com.google.common.io.Files;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.master.rm.YarnTajoResourceManager;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.*;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.sql.ResultSet;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+public class TajoTestingCluster {
+	private static Log LOG = LogFactory.getLog(TajoTestingCluster.class);
+	private TajoConf conf;
+
+  protected MiniTajoYarnCluster yarnCluster;
+  private FileSystem defaultFS;
+  private MiniDFSCluster dfsCluster;
+	private MiniCatalogServer catalogServer;
+
+  private TajoMaster tajoMaster;
+  private List<TajoWorker> tajoWorkers = new ArrayList<TajoWorker>();
+  private boolean standbyWorkerMode = false;
+
+	// If non-null, then already a cluster running.
+	private File clusterTestBuildDir = null;
+
+	/**
+	 * System property key to get test directory value.
+	 * Name is as it is because mini dfs has hard-codings to put test data here.
+	 */
+	public static final String TEST_DIRECTORY_KEY = MiniDFSCluster.PROP_TEST_BUILD_DATA;
+
+	/**
+	 * Default parent directory for test output.
+	 */
+	public static final String DEFAULT_TEST_DIRECTORY = "target/test-data";
+
+  /**
+   * True If HCatalogStore is used. Otherwise, it is FALSE.
+   */
+  public Boolean isHCatalogStoreUse = false;
+
+  public TajoTestingCluster() {
+    this.conf = new TajoConf();
+    initPropertiesAndConfigs();
+	}
+
+  void initPropertiesAndConfigs() {
+    if (System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname) != null) {
+      String testResourceManager = System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname);
+      Preconditions.checkState(
+          testResourceManager.equals(TajoWorkerResourceManager.class.getCanonicalName()) ||
+              testResourceManager.equals(YarnTajoResourceManager.class.getCanonicalName()),
+          ConfVars.RESOURCE_MANAGER_CLASS.varname + " must be either " + TajoWorkerResourceManager.class.getCanonicalName() + " or " +
+              YarnTajoResourceManager.class.getCanonicalName() +"."
+      );
+      conf.set(ConfVars.RESOURCE_MANAGER_CLASS.varname, System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname));
+    }
+    conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 1024);
+    conf.setFloat(ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.varname, 2.0f);
+
+    this.standbyWorkerMode = conf.getVar(ConfVars.RESOURCE_MANAGER_CLASS)
+        .indexOf(TajoWorkerResourceManager.class.getName()) >= 0;
+    conf.set(CommonTestingUtil.TAJO_TEST, "TRUE");
+  }
+
+	public TajoConf getConfiguration() {
+		return this.conf;
+	}
+
+	public void initTestDir() {
+		if (System.getProperty(TEST_DIRECTORY_KEY) == null) {
+			clusterTestBuildDir = setupClusterTestBuildDir();
+			System.setProperty(TEST_DIRECTORY_KEY,
+          clusterTestBuildDir.getAbsolutePath());
+		}
+	}
+
+	/**
+	 * @return Where to write test data on local filesystem; usually
+	 * {@link #DEFAULT_TEST_DIRECTORY}
+	 * @see #setupClusterTestBuildDir()
+	 */
+	public static File getTestDir() {
+		return new File(System.getProperty(TEST_DIRECTORY_KEY,
+			DEFAULT_TEST_DIRECTORY));
+	}
+
+	/**
+	 * @param subdirName
+	 * @return Path to a subdirectory named <code>subdirName</code> under
+	 * {@link #getTestDir()}.
+	 * @see #setupClusterTestBuildDir()
+	 */
+	public static File getTestDir(final String subdirName) {
+		return new File(getTestDir(), subdirName);
+  }
+
+	public File setupClusterTestBuildDir() {
+		String randomStr = UUID.randomUUID().toString();
+		String dirStr = getTestDir(randomStr).toString();
+		File dir = new File(dirStr).getAbsoluteFile();
+		// Have it cleaned up on exit
+		dir.deleteOnExit();
+		return dir;
+	}
+
+  ////////////////////////////////////////////////////////
+  // HDFS Section
+  ////////////////////////////////////////////////////////
+  /**
+   * Start a minidfscluster.
+   * @param servers How many DNs to start.
+   * @throws Exception
+   * @see {@link #shutdownMiniDFSCluster()}
+   * @return The mini dfs cluster created.
+   */
+  public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception {
+    return startMiniDFSCluster(servers, null, null);
+  }
+
+  /**
+   * Start a minidfscluster.
+   * Can only create one.
+   * @param servers How many DNs to start.
+   * @param dir Where to home your dfs cluster.
+   * @param hosts hostnames DNs to run on.
+   * @throws Exception
+   * @see {@link #shutdownMiniDFSCluster()}
+   * @return The mini dfs cluster created.
+   * @throws java.io.IOException
+   */
+  public MiniDFSCluster startMiniDFSCluster(int servers,
+                                            final File dir,
+                                            final String hosts[])
+      throws IOException {
+    if (dir == null) {
+      this.clusterTestBuildDir = setupClusterTestBuildDir();
+    } else {
+      this.clusterTestBuildDir = dir;
+    }
+
+    System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
+        this.clusterTestBuildDir.toString());
+
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(new HdfsConfiguration(conf));
+    builder.hosts(hosts);
+    builder.numDataNodes(servers);
+    builder.format(true);
+    builder.manageNameDfsDirs(true);
+    builder.manageDataDfsDirs(true);
+    builder.waitSafeMode(true);
+    this.dfsCluster = builder.build();
+
+    // Set this just-started cluser as our filesystem.
+    this.defaultFS = this.dfsCluster.getFileSystem();
+    this.conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultFS.getUri().toString());
+    this.conf.setVar(TajoConf.ConfVars.ROOT_DIR, defaultFS.getUri() + "/tajo");
+
+    return this.dfsCluster;
+  }
+
+  public void shutdownMiniDFSCluster() throws Exception {
+    if (this.dfsCluster != null) {
+      try {
+        FileSystem fs = this.dfsCluster.getFileSystem();
+        if (fs != null) fs.close();
+      } catch (IOException e) {
+        System.err.println("error closing file system: " + e);
+      }
+      // The below throws an exception per dn, AsynchronousCloseException.
+      this.dfsCluster.shutdown();
+    }
+  }
+
+  public boolean isRunningDFSCluster() {
+    return this.defaultFS != null;
+  }
+
+  public MiniDFSCluster getMiniDFSCluster() {
+    return this.dfsCluster;
+  }
+
+  public FileSystem getDefaultFileSystem() {
+    return this.defaultFS;
+  }
+
+  ////////////////////////////////////////////////////////
+  // Catalog Section
+  ////////////////////////////////////////////////////////
+  public MiniCatalogServer startCatalogCluster() throws Exception {
+    TajoConf c = getConfiguration();
+
+    if(clusterTestBuildDir == null) {
+      clusterTestBuildDir = setupClusterTestBuildDir();
+    }
+
+    conf.set(CatalogConstants.STORE_CLASS, "org.apache.tajo.catalog.store.MemStore");
+    conf.set(CatalogConstants.CATALOG_URI, "jdbc:derby:" + clusterTestBuildDir.getAbsolutePath() + "/db");
+    LOG.info("Apache Derby repository is set to "+conf.get(CatalogConstants.CATALOG_URI));
+    conf.setVar(ConfVars.CATALOG_ADDRESS, "localhost:0");
+
+    catalogServer = new MiniCatalogServer(conf);
+    CatalogServer catServer = catalogServer.getCatalogServer();
+    InetSocketAddress sockAddr = catServer.getBindAddress();
+    c.setVar(ConfVars.CATALOG_ADDRESS, NetUtils.normalizeInetSocketAddress(sockAddr));
+
+    return this.catalogServer;
+  }
+
+  public void shutdownCatalogCluster() {
+    if (catalogServer != null) {
+      this.catalogServer.shutdown();
+    }
+  }
+
+  public MiniCatalogServer getMiniCatalogCluster() {
+    return this.catalogServer;
+  }
+
+  public boolean isHCatalogStoreRunning() {
+    return isHCatalogStoreUse;
+  }
+
+  ////////////////////////////////////////////////////////
+  // Tajo Cluster Section
+  ////////////////////////////////////////////////////////
+  private void startMiniTajoCluster(File testBuildDir,
+                                               final int numSlaves,
+                                               boolean local) throws Exception {
+    TajoConf c = getConfiguration();
+    c.setVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, "localhost:0");
+    c.setVar(ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS, "localhost:0");
+    c.setVar(ConfVars.RESOURCE_TRACKER_RPC_ADDRESS, "localhost:0");
+    c.setVar(ConfVars.WORKER_PEER_RPC_ADDRESS, "localhost:0");
+    c.setVar(ConfVars.WORKER_TEMPORAL_DIR, "file://" + testBuildDir.getAbsolutePath() + "/tajo-localdir");
+
+    LOG.info("derby repository is set to "+conf.get(CatalogConstants.CATALOG_URI));
+
+    if (!local) {
+      c.setVar(ConfVars.ROOT_DIR,
+          getMiniDFSCluster().getFileSystem().getUri() + "/tajo");
+    } else {
+      c.setVar(ConfVars.ROOT_DIR, clusterTestBuildDir.getAbsolutePath() + "/tajo");
+    }
+
+    setupCatalogForTesting(c, clusterTestBuildDir);
+
+    tajoMaster = new TajoMaster();
+    tajoMaster.init(c);
+    tajoMaster.start();
+
+    this.conf.setVar(ConfVars.WORKER_PEER_RPC_ADDRESS, c.getVar(ConfVars.WORKER_PEER_RPC_ADDRESS));
+    this.conf.setVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, c.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS));
+
+    InetSocketAddress tajoMasterAddress = tajoMaster.getContext().getTajoMasterService().getBindAddress();
+
+    this.conf.setVar(ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS,
+        tajoMasterAddress.getHostName() + ":" + tajoMasterAddress.getPort());
+    this.conf.setVar(ConfVars.RESOURCE_TRACKER_RPC_ADDRESS, c.getVar(ConfVars.RESOURCE_TRACKER_RPC_ADDRESS));
+    this.conf.setVar(ConfVars.CATALOG_ADDRESS, c.getVar(ConfVars.CATALOG_ADDRESS));
+
+    if(standbyWorkerMode) {
+      startTajoWorkers(numSlaves);
+    }
+    LOG.info("Mini Tajo cluster is up");
+    LOG.info("====================================================================================");
+    LOG.info("=                           MiniTajoCluster starts up                              =");
+    LOG.info("====================================================================================");
+    LOG.info("= * Master Address: " + tajoMaster.getMasterName());
+    LOG.info("= * CatalogStore: " + tajoMaster.getCatalogServer().getStoreClassName());
+    LOG.info("------------------------------------------------------------------------------------");
+    LOG.info("= * Warehouse Dir: " + TajoConf.getWarehouseDir(c));
+    LOG.info("= * Worker Tmp Dir: " + c.getVar(ConfVars.WORKER_TEMPORAL_DIR));
+    LOG.info("====================================================================================");
+  }
+
+  private void setupCatalogForTesting(TajoConf c, File testBuildDir) throws IOException {
+    final String HCATALOG_CLASS_NAME = "org.apache.tajo.catalog.store.HCatalogStore";
+    boolean hcatalogClassExists = false;
+    try {
+      getClass().getClassLoader().loadClass(HCATALOG_CLASS_NAME);
+      hcatalogClassExists = true;
+    } catch (ClassNotFoundException e) {
+      LOG.info("HCatalogStore is not available.");
+    }
+    String driverClass = System.getProperty(CatalogConstants.STORE_CLASS);
+
+    if (hcatalogClassExists &&
+        driverClass != null && driverClass.equals(HCATALOG_CLASS_NAME)) {
+      try {
+        getClass().getClassLoader().loadClass(HCATALOG_CLASS_NAME);
+        String jdbcUri = "jdbc:derby:;databaseName="+ testBuildDir.toURI().getPath()  + "/metastore_db;create=true";
+        c.set("hive.metastore.warehouse.dir", TajoConf.getWarehouseDir(c).toString() + "/default");
+        c.set("javax.jdo.option.ConnectionURL", jdbcUri);
+        c.set(TajoConf.ConfVars.WAREHOUSE_DIR.varname, conf.getVar(ConfVars.WAREHOUSE_DIR));
+        c.set(CatalogConstants.STORE_CLASS, HCATALOG_CLASS_NAME);
+        Path defaultDatabasePath = new Path(TajoConf.getWarehouseDir(c).toString() + "/default");
+        FileSystem fs = defaultDatabasePath.getFileSystem(c);
+        if (!fs.exists(defaultDatabasePath)) {
+          fs.mkdirs(defaultDatabasePath);
+        }
+        isHCatalogStoreUse = true;
+      } catch (ClassNotFoundException cnfe) {
+        throw new IOException(cnfe);
+      }
+    } else { // for derby
+      c.set(CatalogConstants.STORE_CLASS, "org.apache.tajo.catalog.store.MemStore");
+      c.set(CatalogConstants.CATALOG_URI, "jdbc:derby:" + testBuildDir.getAbsolutePath() + "/db");
+    }
+    c.setVar(ConfVars.CATALOG_ADDRESS, "localhost:0");
+  }
+
+  private void startTajoWorkers(int numSlaves) throws Exception {
+    for(int i = 0; i < 1; i++) {
+      TajoWorker tajoWorker = new TajoWorker();
+
+      TajoConf workerConf  = new TajoConf(this.conf);
+
+      workerConf.setVar(ConfVars.WORKER_INFO_ADDRESS, "localhost:0");
+      workerConf.setVar(ConfVars.WORKER_CLIENT_RPC_ADDRESS, "localhost:0");
+      workerConf.setVar(ConfVars.WORKER_PEER_RPC_ADDRESS, "localhost:0");
+
+      workerConf.setVar(ConfVars.WORKER_QM_RPC_ADDRESS, "localhost:0");
+      
+      tajoWorker.startWorker(workerConf, new String[]{"standby"});
+
+      LOG.info("MiniTajoCluster Worker #" + (i + 1) + " started.");
+      tajoWorkers.add(tajoWorker);
+    }
+  }
+
+  public void restartTajoCluster(int numSlaves) throws Exception {
+    tajoMaster.stop();
+    tajoMaster.start();
+
+    LOG.info("Minicluster has been restarted");
+  }
+
+  public TajoMaster getMaster() {
+    return this.tajoMaster;
+  }
+
+  public List<TajoWorker> getTajoWorkers() {
+    return this.tajoWorkers;
+  }
+
+  public void shutdownMiniTajoCluster() {
+    if(this.tajoMaster != null) {
+      this.tajoMaster.stop();
+    }
+    for(TajoWorker eachWorker: tajoWorkers) {
+      eachWorker.stopWorkerForce();
+    }
+    tajoWorkers.clear();
+    this.tajoMaster= null;
+  }
+
+  ////////////////////////////////////////////////////////
+  // Meta Cluster Section
+  ////////////////////////////////////////////////////////
+  /**
+   * @throws java.io.IOException If a cluster -- dfs or engine -- already running.
+   */
+  void isRunningCluster(String passedBuildPath) throws IOException {
+    if (this.clusterTestBuildDir == null || passedBuildPath != null) return;
+    throw new IOException("Cluster already running at " +
+        this.clusterTestBuildDir);
+  }
+
+  /**
+   * This method starts up a tajo cluster with a given number of clusters in
+   * distributed mode.
+   *
+   * @param numSlaves the number of tajo cluster to start up
+   * @throws Exception
+   */
+  public void startMiniCluster(final int numSlaves)
+      throws Exception {
+    startMiniCluster(numSlaves, null);
+  }
+
+  public void startMiniCluster(final int numSlaves, final String [] dataNodeHosts) throws Exception {
+
+    int numDataNodes = numSlaves;
+    if(dataNodeHosts != null && dataNodeHosts.length != 0) {
+      numDataNodes = dataNodeHosts.length;
+    }
+
+    LOG.info("Starting up minicluster with 1 master(s) and " +
+        numSlaves + " worker(s) and " + numDataNodes + " datanode(s)");
+
+    // If we already put up a cluster, fail.
+    String testBuildPath = conf.get(TEST_DIRECTORY_KEY, null);
+    isRunningCluster(testBuildPath);
+    if (testBuildPath != null) {
+      LOG.info("Using passed path: " + testBuildPath);
+    }
+
+    // Make a new random dir to home everything in.  Set it as system property.
+    // minidfs reads home from system property.
+    this.clusterTestBuildDir = testBuildPath == null?
+        setupClusterTestBuildDir() : new File(testBuildPath);
+
+    System.setProperty(TEST_DIRECTORY_KEY,
+        this.clusterTestBuildDir.getAbsolutePath());
+
+    startMiniDFSCluster(numDataNodes, this.clusterTestBuildDir, dataNodeHosts);
+    this.dfsCluster.waitClusterUp();
+
+    if(!standbyWorkerMode) {
+      startMiniYarnCluster();
+    }
+
+    startMiniTajoCluster(this.clusterTestBuildDir, numSlaves, false);
+  }
+
+  private void startMiniYarnCluster() throws Exception {
+    LOG.info("Starting up YARN cluster");
+    // Scheduler properties required for YARN to work
+    conf.set("yarn.scheduler.capacity.root.queues", "default");
+    conf.set("yarn.scheduler.capacity.root.default.capacity", "100");
+
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 384);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 3000);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, 1);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 2);
+
+    if (yarnCluster == null) {
+      yarnCluster = new MiniTajoYarnCluster(TajoTestingCluster.class.getName(), 3);
+      yarnCluster.init(conf);
+      yarnCluster.start();
+
+      ResourceManager resourceManager = yarnCluster.getResourceManager();
+      InetSocketAddress rmAddr = resourceManager.getClientRMService().getBindAddress();
+      InetSocketAddress rmSchedulerAddr = resourceManager.getApplicationMasterService().getBindAddress();
+      conf.set(YarnConfiguration.RM_ADDRESS, NetUtils.normalizeInetSocketAddress(rmAddr));
+      conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, NetUtils.normalizeInetSocketAddress(rmSchedulerAddr));
+
+      URL url = Thread.currentThread().getContextClassLoader().getResource("yarn-site.xml");
+      if (url == null) {
+        throw new RuntimeException("Could not find 'yarn-site.xml' dummy file in classpath");
+      }
+      yarnCluster.getConfig().set("yarn.application.classpath", new File(url.getPath()).getParent());
+      OutputStream os = new FileOutputStream(new File(url.getPath()));
+      yarnCluster.getConfig().writeXml(os);
+      os.close();
+    }
+  }
+
+  public void startMiniClusterInLocal(final int numSlaves) throws Exception {
+    // If we already put up a cluster, fail.
+    String testBuildPath = conf.get(TEST_DIRECTORY_KEY, null);
+    isRunningCluster(testBuildPath);
+    if (testBuildPath != null) {
+      LOG.info("Using passed path: " + testBuildPath);
+    }
+
+    // Make a new random dir to home everything in.  Set it as system property.
+    // minidfs reads home from system property.
+    this.clusterTestBuildDir = testBuildPath == null?
+        setupClusterTestBuildDir() : new File(testBuildPath);
+
+    System.setProperty(TEST_DIRECTORY_KEY,
+        this.clusterTestBuildDir.getAbsolutePath());
+
+    startMiniTajoCluster(this.clusterTestBuildDir, numSlaves, true);
+  }
+
+  public void shutdownMiniCluster() throws IOException {
+    LOG.info("========================================");
+    LOG.info("Minicluster is stopping");
+    LOG.info("========================================");
+
+    try {
+      Thread.sleep(3000);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    shutdownMiniTajoCluster();
+
+    if(this.catalogServer != null) {
+      shutdownCatalogCluster();
+    }
+
+    if(this.yarnCluster != null) {
+      this.yarnCluster.stop();
+    }
+
+    try {
+      Thread.sleep(3000);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    if(this.dfsCluster != null) {
+
+      try {
+        FileSystem fs = this.dfsCluster.getFileSystem();
+        if (fs != null) fs.close();
+        this.dfsCluster.shutdown();
+      } catch (IOException e) {
+        System.err.println("error closing file system: " + e);
+      }
+    }
+
+    if(this.clusterTestBuildDir != null && this.clusterTestBuildDir.exists()) {
+      if(!ShutdownHookManager.get().isShutdownInProgress()) {
+        //TODO clean test dir when ShutdownInProgress
+        LocalFileSystem localFS = LocalFileSystem.getLocal(conf);
+        localFS.delete(new Path(clusterTestBuildDir.toString()), true);
+        localFS.close();
+      }
+      this.clusterTestBuildDir = null;
+    }
+    LOG.info("Minicluster is down");
+  }
+
+  public static ResultSet run(String[] names,
+                              Schema[] schemas,
+                              Options option,
+                              String[][] tables,
+                              String query) throws Exception {
+    TpchTestBase instance = TpchTestBase.getInstance();
+    TajoTestingCluster util = instance.getTestingCluster();
+    while(true) {
+      if(util.getMaster().isMasterRunning()) {
+        break;
+      }
+      Thread.sleep(1000);
+    }
+    TajoConf conf = util.getConfiguration();
+    TajoClient client = new TajoClient(conf);
+
+    FileSystem fs = util.getDefaultFileSystem();
+    Path rootDir = util.getMaster().
+        getStorageManager().getWarehouseDir();
+    fs.mkdirs(rootDir);
+    for (int i = 0; i < names.length; i++) {
+      Path tablePath = new Path(rootDir, names[i]);
+      fs.mkdirs(tablePath);
+      Path dfsPath = new Path(tablePath, names[i] + ".tbl");
+      FSDataOutputStream out = fs.create(dfsPath);
+      for (int j = 0; j < tables[i].length; j++) {
+        out.write((tables[i][j]+"\n").getBytes());
+      }
+      out.close();
+      TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV, option);
+      client.createExternalTable(names[i], schemas[i], tablePath, meta);
+    }
+    Thread.sleep(1000);
+    ResultSet res = client.executeQueryAndGetResult(query);
+    return res;
+  }
+
+    /**
+    * Write lines to a file.
+    *
+    * @param file File to write lines to
+    * @param lines Strings written to the file
+    * @throws java.io.IOException
+    */
+  private static void writeLines(File file, String... lines)
+      throws IOException {
+    Writer writer = Files.newWriter(file, Charsets.UTF_8);
+    try {
+      for (String line : lines) {
+        writer.write(line);
+        writer.write('\n');
+      }
+    } finally {
+      Closeables.closeQuietly(writer);
+    }
+  }
+
+  public void setAllTajoDaemonConfValue(String key, String value) {
+    tajoMaster.getContext().getConf().set(key, value);
+    for (TajoWorker eachWorker: tajoWorkers) {
+      eachWorker.getConfig().set(key, value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/TestQueryIdFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/TestQueryIdFactory.java b/tajo-core/src/test/java/org/apache/tajo/TestQueryIdFactory.java
new file mode 100644
index 0000000..912400b
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/TestQueryIdFactory.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestQueryIdFactory {
+  
+  @Before
+  public void setup() {
+  }
+
+  @Test
+  public void testNewQueryId() {
+    QueryId qid1 = LocalTajoTestingUtility.newQueryId();
+    QueryId qid2 = LocalTajoTestingUtility.newQueryId();
+    assertTrue(qid1.compareTo(qid2) < 0);
+  }
+  
+  @Test
+  public void testNewSubQueryId() {
+    QueryId qid = LocalTajoTestingUtility.newQueryId();
+    MasterPlan plan = new MasterPlan(qid, null, null);
+    ExecutionBlockId subqid1 = plan.newExecutionBlockId();
+    ExecutionBlockId subqid2 = plan.newExecutionBlockId();
+    assertTrue(subqid1.compareTo(subqid2) < 0);
+  }
+  
+  @Test
+  public void testNewQueryUnitId() {
+    QueryId qid = LocalTajoTestingUtility.newQueryId();
+    MasterPlan plan = new MasterPlan(qid, null, null);
+    ExecutionBlockId subid = plan.newExecutionBlockId();
+    QueryUnitId quid1 = QueryIdFactory.newQueryUnitId(subid);
+    QueryUnitId quid2 = QueryIdFactory.newQueryUnitId(subid);
+    assertTrue(quid1.compareTo(quid2) < 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/TestTajoIds.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/TestTajoIds.java b/tajo-core/src/test/java/org/apache/tajo/TestTajoIds.java
new file mode 100644
index 0000000..60b6c22
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/TestTajoIds.java
@@ -0,0 +1,168 @@
+/**
+ * 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;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.util.TajoIdUtils;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestTajoIds {
+  @Test
+  public void testQueryId() {
+    long ts1 = 1315890136000l;
+    long ts2 = 1315890136001l;
+
+    QueryId j1 = createQueryId(ts1, 2);
+    QueryId j2 = createQueryId(ts1, 1);
+    QueryId j3 = createQueryId(ts2, 1);
+    QueryId j4 = createQueryId(ts1, 2);
+
+    assertTrue(j1.equals(j4));
+    assertFalse(j1.equals(j2));
+    assertFalse(j1.equals(j3));
+
+    assertTrue(j1.compareTo(j4) == 0);
+    assertTrue(j1.compareTo(j2) > 0);
+    assertTrue(j1.compareTo(j3) < 0);
+
+    assertTrue(j1.hashCode() == j4.hashCode());
+    assertFalse(j1.hashCode() == j2.hashCode());
+    assertFalse(j1.hashCode() == j3.hashCode());
+
+    QueryId j5 = createQueryId(ts1, 231415);
+    assertEquals("q_" + ts1 + "_0002", j1.toString());
+    assertEquals("q_" + ts1 + "_231415", j5.toString());
+  }
+
+  @Test
+  public void testQueryIds() {
+    long timeId = 1315890136000l;
+    
+    QueryId queryId = createQueryId(timeId, 1);
+    assertEquals("q_" + timeId + "_0001", queryId.toString());
+    
+    ExecutionBlockId subId = QueryIdFactory.newExecutionBlockId(queryId, 2);
+    assertEquals("eb_" + timeId +"_0001_000002", subId.toString());
+    
+    QueryUnitId qId = new QueryUnitId(subId, 5);
+    assertEquals("t_" + timeId + "_0001_000002_000005", qId.toString());
+
+    QueryUnitAttemptId attemptId = new QueryUnitAttemptId(qId, 4);
+    assertEquals("ta_" + timeId + "_0001_000002_000005_04", attemptId.toString());
+  }
+
+  @Test
+  public void testEqualsObject() {
+    long timeId = System.currentTimeMillis();
+    
+    QueryId queryId1 = createQueryId(timeId, 1);
+    QueryId queryId2 = createQueryId(timeId, 2);
+    assertNotSame(queryId1, queryId2);    
+    QueryId queryId3 = createQueryId(timeId, 1);
+    assertEquals(queryId1, queryId3);
+    
+    ExecutionBlockId sid1 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+    ExecutionBlockId sid2 = QueryIdFactory.newExecutionBlockId(queryId1, 2);
+    assertNotSame(sid1, sid2);
+    ExecutionBlockId sid3 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+    assertEquals(sid1, sid3);
+    
+    QueryUnitId qid1 = new QueryUnitId(sid1, 9);
+    QueryUnitId qid2 = new QueryUnitId(sid1, 10);
+    assertNotSame(qid1, qid2);
+    QueryUnitId qid3 = new QueryUnitId(sid1, 9);
+    assertEquals(qid1, qid3);
+  }
+
+  @Test
+  public void testCompareTo() {
+    long time = System.currentTimeMillis();
+    
+    QueryId queryId1 = createQueryId(time, 1);
+    QueryId queryId2 = createQueryId(time, 2);
+    QueryId queryId3 = createQueryId(time, 1);
+    assertEquals(-1, queryId1.compareTo(queryId2));
+    assertEquals(1, queryId2.compareTo(queryId1));
+    assertEquals(0, queryId3.compareTo(queryId1));
+
+    ExecutionBlockId sid1 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+    ExecutionBlockId sid2 = QueryIdFactory.newExecutionBlockId(queryId1, 2);
+    ExecutionBlockId sid3 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+    assertEquals(-1, sid1.compareTo(sid2));
+    assertEquals(1, sid2.compareTo(sid1));
+    assertEquals(0, sid3.compareTo(sid1));
+    
+    QueryUnitId qid1 = new QueryUnitId(sid1, 9);
+    QueryUnitId qid2 = new QueryUnitId(sid1, 10);
+    QueryUnitId qid3 = new QueryUnitId(sid1, 9);
+    assertEquals(-1, qid1.compareTo(qid2));
+    assertEquals(1, qid2.compareTo(qid1));
+    assertEquals(0, qid3.compareTo(qid1));
+  }
+  
+  @Test
+  public void testConstructFromString() {
+    QueryId qid1 = LocalTajoTestingUtility.newQueryId();
+    QueryId qid2 = TajoIdUtils.parseQueryId(qid1.toString());
+    assertEquals(qid1, qid2);
+
+    MasterPlan plan1 = new MasterPlan(qid1, null, null);
+    ExecutionBlockId sub1 = plan1.newExecutionBlockId();
+    ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
+    assertEquals(sub1, sub2);
+    
+    QueryUnitId u1 = QueryIdFactory.newQueryUnitId(sub1);
+    QueryUnitId u2 = new QueryUnitId(u1.getProto());
+    assertEquals(u1, u2);
+
+    QueryUnitAttemptId attempt1 = new QueryUnitAttemptId(u1, 1);
+    QueryUnitAttemptId attempt2 = new QueryUnitAttemptId(attempt1.getProto());
+    assertEquals(attempt1, attempt2);
+  }
+
+  @Test
+  public void testConstructFromPB() {
+    QueryId qid1 = LocalTajoTestingUtility.newQueryId();
+    QueryId qid2 = new QueryId(qid1.getProto());
+    assertEquals(qid1, qid2);
+
+    MasterPlan plan = new MasterPlan(qid1, null, null);
+    ExecutionBlockId sub1 = plan.newExecutionBlockId();
+    ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
+    assertEquals(sub1, sub2);
+
+    QueryUnitId u1 = QueryIdFactory.newQueryUnitId(sub1);
+    QueryUnitId u2 = new QueryUnitId(u1.getProto());
+    assertEquals(u1, u2);
+
+    QueryUnitAttemptId attempt1 = new QueryUnitAttemptId(u1, 1);
+    QueryUnitAttemptId attempt2 = new QueryUnitAttemptId(attempt1.getProto());
+    assertEquals(attempt1, attempt2);
+  }
+
+  public static QueryId createQueryId(long timestamp, int id) {
+    ApplicationId appId = BuilderUtils.newApplicationId(timestamp, id);
+
+    return QueryIdFactory.newQueryId(appId.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java b/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java
new file mode 100644
index 0000000..8995d81
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java
@@ -0,0 +1,117 @@
+/**
+ * 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;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.FileUtil;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.util.Map;
+
+public class TpchTestBase {
+  private static final Log LOG = LogFactory.getLog(TpchTestBase.class);
+
+  String [] names;
+  String [] paths;
+  String [][] tables;
+  Schema[] schemas;
+  Map<String, Integer> nameMap = Maps.newHashMap();
+  protected TPCH tpch;
+  protected LocalTajoTestingUtility util;
+
+  private static TpchTestBase testBase;
+
+  static {
+    try {
+      testBase = new TpchTestBase();
+      testBase.setUp();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  private TpchTestBase() throws IOException {
+    names = new String[] {"customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier", "empty_orders"};
+    paths = new String[names.length];
+    for (int i = 0; i < names.length; i++) {
+      nameMap.put(names[i], i);
+    }
+
+    tpch = new TPCH();
+    tpch.loadSchemas();
+    tpch.loadQueries();
+
+    schemas = new Schema[names.length];
+    for (int i = 0; i < names.length; i++) {
+      schemas[i] = tpch.getSchema(names[i]);
+    }
+
+    tables = new String[names.length][];
+    File file;
+    for (int i = 0; i < names.length; i++) {
+      file = new File("src/test/tpch/" + names[i] + ".tbl");
+      if(!file.exists()) {
+        file = new File(System.getProperty("user.dir") + "/tajo-core/src/test/tpch/" + names[i]
+            + ".tbl");
+      }
+      tables[i] = FileUtil.readTextFile(file).split("\n");
+      paths[i] = file.getAbsolutePath();
+    }
+    try {
+      Thread.sleep(1000);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private void setUp() throws Exception {
+    util = new LocalTajoTestingUtility();
+    Options opt = new Options();
+    opt.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    util.setup(names, paths, schemas, opt);
+  }
+
+  public static TpchTestBase getInstance() {
+    return testBase;
+  }
+
+  public ResultSet execute(String query) throws Exception {
+    return util.execute(query);
+  }
+
+  public TajoTestingCluster getTestingCluster() {
+    return util.getTestingCluster();
+  }
+
+  public void tearDown() throws IOException {
+    try {
+      Thread.sleep(2000);
+    } catch (InterruptedException e) {
+    }
+    util.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java
new file mode 100644
index 0000000..4ca9b3e
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.benchmark;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+
+@Category(IntegrationTest.class)
+public class TestTPCH extends QueryTestCaseBase {
+
+  public TestTPCH() {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test
+  public void testQ1OrderBy() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testQ2FourJoins() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testTPCH14Expr() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java b/tajo-core/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java
new file mode 100644
index 0000000..9c6e760
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java
@@ -0,0 +1,46 @@
+/**
+ * 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.cli;
+
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.conf.TajoConf;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestExecExternalShellCommand {
+  @Test
+  public void testCommand() throws Exception {
+    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    TajoCli cli = new TajoCli(tajoConf, new String[]{}, null, out);
+
+    cli.executeMetaCommand("\\! echo \"this is test\"");
+    String consoleResult = new String(out.toByteArray());
+    assertEquals("this is test\n", consoleResult);
+
+    cli.executeMetaCommand("\\! error_command");
+    consoleResult = new String(out.toByteArray());
+    assertEquals("this is test\nERROR: /bin/bash: error_command: command not found\n", consoleResult);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java b/tajo-core/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java
new file mode 100644
index 0000000..b51835f
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java
@@ -0,0 +1,46 @@
+/**
+ * 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.cli;
+
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.conf.TajoConf;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHdfsCommand {
+  @Test
+  public void testHdfCommand() throws Exception {
+    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    System.setOut(new PrintStream(out));
+    System.setErr(new PrintStream(out));
+    TajoCli cli = new TajoCli(tajoConf, new String[]{}, null, out);
+
+    cli.executeMetaCommand("\\dfs -test");
+    String consoleResult = new String(out.toByteArray());
+    assertEquals("-test: Not enough arguments: expected 1 but got 0\n" +
+        "Usage: hadoop fs [generic options] -test -[defsz] <path>\n", consoleResult);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/cli/TestSimpleParser.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/TestSimpleParser.java b/tajo-core/src/test/java/org/apache/tajo/cli/TestSimpleParser.java
new file mode 100644
index 0000000..9c02b65
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/TestSimpleParser.java
@@ -0,0 +1,179 @@
+/**
+ * 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.cli;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestSimpleParser {
+
+  @Test
+  public final void testSpecialCases() throws InvalidStatementException {
+    List<ParsedResult> res1 = SimpleParser.parseScript("");
+    assertEquals(0, res1.size());
+
+    List<ParsedResult> res2 = SimpleParser.parseScript("a");
+    assertEquals(1, res2.size());
+
+    List<ParsedResult> res3 = SimpleParser.parseScript("?");
+    assertEquals(0, res3.size());
+
+    List<ParsedResult> res4 = SimpleParser.parseScript("\\");
+    assertEquals(1, res4.size());
+  }
+
+  @Test
+  public final void testMetaCommands() throws InvalidStatementException {
+    List<ParsedResult> res1 = SimpleParser.parseScript("\\d");
+    assertEquals(1, res1.size());
+    assertEquals(ParsedResult.StatementType.META, res1.get(0).getType());
+    assertEquals("\\d", res1.get(0).getStatement());
+
+    List<ParsedResult> res2 = SimpleParser.parseScript("\\d;\\c;\\f;");
+    assertEquals(3, res2.size());
+    assertEquals(ParsedResult.StatementType.META, res2.get(0).getType());
+    assertEquals("\\d", res2.get(0).getStatement());
+    assertEquals(ParsedResult.StatementType.META, res2.get(1).getType());
+    assertEquals("\\c", res2.get(1).getStatement());
+    assertEquals(ParsedResult.StatementType.META, res2.get(2).getType());
+    assertEquals("\\f", res2.get(2).getStatement());
+
+    List<ParsedResult> res3 = SimpleParser.parseScript("\n\t\t  \\d;\n\\c;\t\t\\f  ;");
+    assertEquals(3, res3.size());
+    assertEquals(ParsedResult.StatementType.META, res3.get(0).getType());
+    assertEquals("\\d", res3.get(0).getStatement());
+    assertEquals(ParsedResult.StatementType.META, res3.get(1).getType());
+    assertEquals("\\c", res3.get(1).getStatement());
+    assertEquals(ParsedResult.StatementType.META, res3.get(2).getType());
+    assertEquals("\\f", res3.get(2).getStatement());
+
+    List<ParsedResult> res4 = SimpleParser.parseScript("\\\td;");
+    assertEquals(1, res4.size());
+    assertEquals("\\\td", res4.get(0).getStatement());
+  }
+
+  @Test
+  public final void testStatements() throws InvalidStatementException {
+    List<ParsedResult> res1 = SimpleParser.parseScript("select * from test;");
+    assertEquals(1, res1.size());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res1.get(0).getType());
+    assertEquals("select * from test", res1.get(0).getStatement());
+
+    List<ParsedResult> res2 = SimpleParser.parseScript("select * from test;");
+    assertEquals(1, res2.size());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res2.get(0).getType());
+    assertEquals("select * from test", res2.get(0).getStatement());
+
+    List<ParsedResult> res3 = SimpleParser.parseScript("select * from test1;select * from test2;");
+    assertEquals(2, res3.size());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res3.get(0).getType());
+    assertEquals("select * from test1", res3.get(0).getStatement());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res3.get(1).getType());
+    assertEquals("select * from test2", res3.get(1).getStatement());
+
+    List<ParsedResult> res4 = SimpleParser.parseScript("\t\t\n\rselect * from \ntest1;select * from test2\n;");
+    assertEquals(2, res4.size());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res4.get(0).getType());
+    assertEquals("select * from \ntest1", res4.get(0).getStatement());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res4.get(1).getType());
+    assertEquals("select * from test2", res4.get(1).getStatement());
+
+    List<ParsedResult> res5 =
+        SimpleParser.parseScript("\t\t\n\rselect * from \ntest1;\\d test;select * from test2;\n\nselect 1;");
+    assertEquals(4, res5.size());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res5.get(0).getType());
+    assertEquals("select * from \ntest1", res5.get(0).getStatement());
+    assertEquals(ParsedResult.StatementType.META, res5.get(1).getType());
+    assertEquals("\\d test", res5.get(1).getStatement());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res5.get(2).getType());
+    assertEquals("select * from test2", res5.get(2).getStatement());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res5.get(3).getType());
+    assertEquals("select 1", res5.get(3).getStatement());
+  }
+
+  @Test
+  public final void testQuoted() throws InvalidStatementException {
+    List<ParsedResult> res1 = SimpleParser.parseScript("select '\n;' from test;");
+    assertEquals(1, res1.size());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res1.get(0).getType());
+    assertEquals("select '\n;' from test", res1.get(0).getStatement());
+
+    List<ParsedResult> res2 = SimpleParser.parseScript("select 'abc\nbbc\nddf' from test;");
+    assertEquals(1, res2.size());
+    assertEquals(ParsedResult.StatementType.STATEMENT, res2.get(0).getType());
+    assertEquals("select 'abc\nbbc\nddf' from test", res2.get(0).getStatement());
+
+    try {
+      SimpleParser.parseScript("select 'abc");
+      assertTrue(false);
+    } catch (InvalidStatementException is) {
+      assertTrue(true);
+    }
+  }
+
+  @Test
+  public final void testParseLines1() throws InvalidStatementException {
+    String [] lines = {
+      "select abc, ",
+      "bbc from test"
+    };
+    SimpleParser parser = new SimpleParser();
+    List<ParsedResult> result1 = parser.parseLines(lines[0]);
+    assertEquals(0, result1.size());
+    List<ParsedResult> result2 = parser.parseLines(lines[1]);
+    assertEquals(0, result2.size());
+    List<ParsedResult> result3 = parser.EOF();
+    assertEquals(1, result3.size());
+    assertEquals(lines[0] + lines[1], result3.get(0).getStatement());
+  }
+
+  @Test
+  public final void testParseLines2() throws InvalidStatementException {
+    String [] lines = {
+        "select abc, '",
+        "bbc' from test; select * from test3;"
+    };
+    SimpleParser parser = new SimpleParser();
+    List<ParsedResult> result1 = parser.parseLines(lines[0]);
+    assertEquals(0, result1.size());
+    List<ParsedResult> result2 = parser.parseLines(lines[1]);
+    assertEquals(2, result2.size());
+    assertEquals("select abc, 'bbc' from test", result2.get(0).getStatement());
+    assertEquals("select * from test3", result2.get(1).getStatement());
+  }
+
+  @Test
+  public final void testParseLines3() throws InvalidStatementException {
+    String [] lines = {
+        "select abc, 'bbc",
+        "' from test; select * from test3;"
+    };
+    SimpleParser parser = new SimpleParser();
+    List<ParsedResult> result1 = parser.parseLines(lines[0]);
+    assertEquals(0, result1.size());
+    List<ParsedResult> result2 = parser.parseLines(lines[1]);
+    assertEquals(2, result2.size());
+    assertEquals("select abc, 'bbc' from test", result2.get(0).getStatement());
+    assertEquals("select * from test3", result2.get(1).getStatement());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/client/TestDDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestDDLBuilder.java b/tajo-core/src/test/java/org/apache/tajo/client/TestDDLBuilder.java
new file mode 100644
index 0000000..1855217
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestDDLBuilder.java
@@ -0,0 +1,133 @@
+/**
+ * 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.client;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.FileUtil;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+
+public class TestDDLBuilder {
+  private static final Schema schema1;
+  private static final TableMeta meta1;
+  private static final PartitionMethodDesc partitionMethod1;
+
+  static {
+    schema1 = new Schema();
+    schema1.addColumn("name", TajoDataTypes.Type.BLOB);
+    schema1.addColumn("addr", TajoDataTypes.Type.TEXT);
+
+    meta1 = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV);
+    meta1.putOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    meta1.putOption(StorageConstants.COMPRESSION_CODEC, GzipCodec.class.getName());
+
+    Schema expressionSchema = new Schema();
+    expressionSchema.addColumn("key", TajoDataTypes.Type.INT4);
+    expressionSchema.addColumn("key2", TajoDataTypes.Type.TEXT);
+    partitionMethod1 = new PartitionMethodDesc(
+        "db1",
+        "table1",
+        CatalogProtos.PartitionType.COLUMN,
+        "key,key2",
+        expressionSchema);
+  }
+
+  @Test
+  public void testBuildDDLForExternalTable() throws Exception {
+    TableDesc desc = new TableDesc("db1.table1", schema1, meta1, new Path("/table1"));
+    desc.setPartitionMethod(partitionMethod1);
+    desc.setExternal(true);
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLForExternalTable.result"),
+        DDLBuilder.buildDDLForExternalTable(desc));
+  }
+
+  @Test
+  public void testBuildDDLQuotedTableName() throws Exception {
+    Schema schema2 = new Schema();
+    schema2.addColumn("name", TajoDataTypes.Type.BLOB);
+    schema2.addColumn("addr", TajoDataTypes.Type.TEXT);
+    schema2.addColumn("FirstName", TajoDataTypes.Type.TEXT);
+    schema2.addColumn("LastName", TajoDataTypes.Type.TEXT);
+    schema2.addColumn("with", TajoDataTypes.Type.TEXT);
+
+    Schema expressionSchema2 = new Schema();
+    expressionSchema2.addColumn("BirthYear", TajoDataTypes.Type.INT4);
+
+    PartitionMethodDesc partitionMethod2 = new PartitionMethodDesc(
+        "db1",
+        "table1",
+        CatalogProtos.PartitionType.COLUMN,
+        "key,key2",
+        expressionSchema2);
+
+    TableDesc desc = new TableDesc("db1.TABLE2", schema2, meta1, new Path("/table1"));
+    desc.setPartitionMethod(partitionMethod2);
+    desc.setExternal(true);
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName1.result"),
+        DDLBuilder.buildDDLForExternalTable(desc));
+
+    desc = new TableDesc("db1.TABLE1", schema2, meta1, new Path("/table1"));
+    desc.setPartitionMethod(partitionMethod2);
+    desc.setExternal(false);
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName2.result"),
+        DDLBuilder.buildDDLForBaseTable(desc));
+  }
+
+  @Test
+  public void testBuildDDLForBaseTable() throws Exception {
+    TableDesc desc = new TableDesc("db1.table2", schema1, meta1, new Path("/table1"));
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLForBaseTable.result"),
+        DDLBuilder.buildDDLForBaseTable(desc));
+  }
+
+  @Test
+  public void testBuildColumn() throws Exception {
+    String [] tobeUnquoted = {
+        "column_name",
+        "columnname",
+        "column_1",
+    };
+
+    for (String columnName : tobeUnquoted) {
+      assertFalse(CatalogUtil.isShouldBeQuoted(columnName));
+    }
+
+    String [] quoted = {
+        "Column_Name",
+        "COLUMN_NAME",
+        "컬럼",
+        "$column_name",
+        "Column_Name1",
+        "with",
+        "when"
+    };
+
+    for (String columnName : quoted) {
+      assertTrue(CatalogUtil.isShouldBeQuoted(columnName));
+    }
+  }
+}


[09/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
new file mode 100644
index 0000000..49e6874
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -0,0 +1,666 @@
+/**
+ * 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.client;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.protobuf.ServiceException;
+import com.sun.org.apache.commons.logging.Log;
+import com.sun.org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.FunctionDesc;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.jdbc.TajoResultSet;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.*;
+
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
+@Category(IntegrationTest.class)
+public class TestTajoClient {
+  private static TajoTestingCluster cluster;
+  private static TajoConf conf;
+  private static TajoClient client;
+  private static Path testDir;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = TpchTestBase.getInstance().getTestingCluster();
+    conf = cluster.getConfiguration();
+    client = new TajoClient(conf);
+    testDir = CommonTestingUtil.getTestDir();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    client.close();
+  }
+
+  private static Path writeTmpTable(String tableName) throws IOException {
+    Path tablePath = StorageUtil.concatPath(testDir, tableName);
+    BackendTestingUtil.writeTmpTable(conf, tablePath);
+    return tablePath;
+  }
+
+  @Test
+  public final void testCreateAndDropDatabases() throws ServiceException {
+    int currentNum = client.getAllDatabaseNames().size();
+
+    String prefix = CatalogUtil.normalizeIdentifier("testCreateDatabase_");
+    for (int i = 0; i < 10; i++) {
+      // test allDatabaseNames
+      assertEquals(currentNum + i, client.getAllDatabaseNames().size());
+
+      // test existence
+      assertFalse(client.existDatabase(prefix + i));
+      assertTrue(client.createDatabase(prefix + i));
+      assertTrue(client.existDatabase(prefix + i));
+
+      // test allDatabaseNames
+      assertEquals(currentNum + i + 1, client.getAllDatabaseNames().size());
+      assertTrue(client.getAllDatabaseNames().contains(prefix + i));
+    }
+
+    // test dropDatabase, existDatabase and getAllDatabaseNames()
+    for (int i = 0; i < 10; i++) {
+      assertTrue(client.existDatabase(prefix + i));
+      assertTrue(client.getAllDatabaseNames().contains(prefix + i));
+      assertTrue(client.dropDatabase(prefix + i));
+      assertFalse(client.existDatabase(prefix + i));
+      assertFalse(client.getAllDatabaseNames().contains(prefix + i));
+    }
+
+    assertEquals(currentNum, client.getAllDatabaseNames().size());
+  }
+
+  @Test
+  public final void testCurrentDatabase() throws IOException, ServiceException, InterruptedException {
+    int currentNum = client.getAllDatabaseNames().size();
+    assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase());
+
+    String databaseName = CatalogUtil.normalizeIdentifier("testcurrentdatabase");
+    assertTrue(client.createDatabase(databaseName));
+    assertEquals(currentNum + 1, client.getAllDatabaseNames().size());
+    assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase());
+    assertTrue(client.selectDatabase(databaseName));
+    assertEquals(databaseName, client.getCurrentDatabase());
+    assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME));
+    assertTrue(client.dropDatabase(databaseName));
+
+    assertEquals(currentNum, client.getAllDatabaseNames().size());
+  }
+
+  @Test
+  public final void testSelectDatabaseToInvalidOne() throws IOException, ServiceException, InterruptedException {
+    int currentNum = client.getAllDatabaseNames().size();
+    assertFalse(client.existDatabase("invaliddatabase"));
+
+    try {
+      assertTrue(client.selectDatabase("invaliddatabase"));
+      assertFalse(true);
+    } catch (Throwable t) {
+      assertFalse(false);
+    }
+
+    assertEquals(currentNum, client.getAllDatabaseNames().size());
+  }
+
+  @Test
+  public final void testDropCurrentDatabase() throws IOException, ServiceException, InterruptedException {
+    int currentNum = client.getAllDatabaseNames().size();
+    String databaseName = CatalogUtil.normalizeIdentifier("testdropcurrentdatabase");
+    assertTrue(client.createDatabase(databaseName));
+    assertTrue(client.selectDatabase(databaseName));
+    assertEquals(databaseName, client.getCurrentDatabase());
+
+    try {
+      client.dropDatabase(databaseName);
+      assertFalse(true);
+    } catch (Throwable t) {
+      assertFalse(false);
+    }
+
+    assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME));
+    assertTrue(client.dropDatabase(databaseName));
+    assertEquals(currentNum, client.getAllDatabaseNames().size());
+  }
+
+  @Test
+  public final void testSessionVariables() throws IOException, ServiceException, InterruptedException {
+    String prefixName = "key_";
+    String prefixValue = "val_";
+    for (int i = 0; i < 10; i++) {
+      String key = prefixName + i;
+      String val = prefixValue + i;
+
+      assertEquals(i, client.getAllSessionVariables().size());
+      assertFalse(client.getAllSessionVariables().containsKey(key));
+      assertFalse(client.existSessionVariable(key));
+
+      Map<String, String> map = Maps.newHashMap();
+      map.put(key, val);
+      client.updateSessionVariables(map);
+
+      assertEquals(i + 1, client.getAllSessionVariables().size());
+      assertTrue(client.getAllSessionVariables().containsKey(key));
+      assertTrue(client.existSessionVariable(key));
+    }
+
+    int totalSessionVarNum = client.getAllSessionVariables().size();
+
+    for (int i = 0; i < 10; i++) {
+      String key = prefixName + i;
+
+      assertTrue(client.getAllSessionVariables().containsKey(key));
+      assertTrue(client.existSessionVariable(key));
+
+      client.unsetSessionVariables(Lists.newArrayList(key));
+
+      assertFalse(client.getAllSessionVariables().containsKey(key));
+      assertFalse(client.existSessionVariable(key));
+    }
+
+    assertEquals(totalSessionVarNum - 10, client.getAllSessionVariables().size());
+  }
+
+  @Test
+  public final void testKillQuery() throws IOException, ServiceException, InterruptedException {
+    ClientProtos.SubmitQueryResponse res = client.executeQuery("select sleep(1) from lineitem");
+    Thread.sleep(1000);
+    QueryId queryId = new QueryId(res.getQueryId());
+    client.killQuery(queryId);
+    assertEquals(TajoProtos.QueryState.QUERY_KILLED, client.getQueryStatus(queryId).getState());
+  }
+
+  @Test
+  public final void testUpdateQuery() throws IOException, ServiceException {
+    final String tableName = CatalogUtil.normalizeIdentifier("testUpdateQuery");
+    Path tablePath = writeTmpTable(tableName);
+
+    assertFalse(client.existTable(tableName));
+    String sql =
+        "create external table " + tableName + " (deptname text, score integer) "
+            + "using csv location '" + tablePath + "'";
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+    client.dropTable(tableName);
+    assertFalse(client.existTable(tableName));
+  }
+
+  @Test
+  public final void testCreateAndDropExternalTable()
+      throws IOException, ServiceException, SQLException {
+    final String tableName = "testCreateAndDropExternalTable";
+    Path tablePath = writeTmpTable(tableName);
+    LOG.error("Full path:" + tablePath.toUri().getRawPath());
+    FileSystem fs = tablePath.getFileSystem(conf);
+    assertTrue(fs.exists(tablePath));
+
+    assertFalse(client.existTable(tableName));
+
+    client.createExternalTable(tableName, BackendTestingUtil.mockupSchema, tablePath, BackendTestingUtil.mockupMeta);
+    assertTrue(client.existTable(tableName));
+    client.dropTable(tableName);
+    assertFalse(client.existTable(tableName));
+    fs = tablePath.getFileSystem(conf);
+    assertTrue(fs.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndPurgeExternalTable() throws IOException, ServiceException, SQLException {
+    final String tableName = "testCreateAndPurgeExternalTable";
+    Path tablePath = writeTmpTable(tableName);
+    LOG.error("Full path:" + tablePath.toUri().getRawPath());
+    FileSystem fs = tablePath.getFileSystem(conf);
+    assertTrue(fs.exists(tablePath));
+
+    assertFalse(client.existTable(tableName));
+
+    client.createExternalTable(tableName, BackendTestingUtil.mockupSchema, tablePath, BackendTestingUtil.mockupMeta);
+    assertTrue(client.existTable(tableName));
+    client.dropTable(tableName, true);
+    assertFalse(client.existTable(tableName));
+    fs = tablePath.getFileSystem(conf);
+    assertFalse("Checking if table data exists", fs.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndDropExternalTableByExecuteQuery() throws IOException, ServiceException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropExternalTableByExecuteQuery");
+
+    Path tablePath = writeTmpTable(tableName);
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create external table " + tableName + " (deptname text, score int4) " + "using csv location '"
+        + tablePath + "'";
+
+    client.executeQueryAndGetResult(sql);
+    assertTrue(client.existTable(tableName));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable("tableName"));
+    FileSystem localFS = FileSystem.getLocal(conf);
+    assertTrue(localFS.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndPurgeExternalTableByExecuteQuery() throws IOException, ServiceException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndPurgeExternalTableByExecuteQuery");
+
+    Path tablePath = writeTmpTable(tableName);
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create external table " + tableName + " (deptname text, score int4) " + "using csv location '"
+        + tablePath + "'";
+
+    client.executeQueryAndGetResult(sql);
+    assertTrue(client.existTable(tableName));
+
+    client.updateQuery("drop table " + tableName + " purge");
+    assertFalse(client.existTable(tableName));
+    FileSystem localFS = FileSystem.getLocal(conf);
+    assertFalse(localFS.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndDropTableByExecuteQuery() throws IOException, ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropTableByExecuteQuery");
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable(tableName));
+    assertTrue(hdfs.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndPurgeTableByExecuteQuery() throws IOException, ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndPurgeTableByExecuteQuery");
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName + " purge");
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  @Test
+  public final void testDDLByExecuteQuery() throws IOException, ServiceException {
+    final String tableName = CatalogUtil.normalizeIdentifier("testDDLByExecuteQuery");
+    Path tablePath = writeTmpTable(tableName);
+
+    assertFalse(client.existTable(tableName));
+    String sql =
+        "create external table " + tableName + " (deptname text, score int4) "
+            + "using csv location '" + tablePath + "'";
+    client.executeQueryAndGetResult(sql);
+    assertTrue(client.existTable(tableName));
+  }
+
+  @Test
+  public final void testGetTableList() throws IOException, ServiceException {
+    String tableName1 = "GetTableList1".toLowerCase();
+    String tableName2 = "GetTableList2".toLowerCase();
+
+    assertFalse(client.existTable(tableName1));
+    assertFalse(client.existTable(tableName2));
+    client.updateQuery("create table GetTableList1 (age int, name text);");
+    client.updateQuery("create table GetTableList2 (age int, name text);");
+
+    assertTrue(client.existTable(tableName1));
+    assertTrue(client.existTable(tableName2));
+
+    Set<String> tables = Sets.newHashSet(client.getTableList(null));
+    assertTrue(tables.contains(tableName1));
+    assertTrue(tables.contains(tableName2));
+  }
+
+  Log LOG = LogFactory.getLog(TestTajoClient.class);
+
+  @Test
+  public final void testGetTableDesc() throws IOException, ServiceException, SQLException {
+    final String tableName1 = CatalogUtil.normalizeIdentifier("table3");
+    Path tablePath = writeTmpTable(tableName1);
+    LOG.error("Full path:" + tablePath.toUri().getRawPath());
+    FileSystem fs = tablePath.getFileSystem(conf);
+    assertTrue(fs.exists(tablePath));
+
+    assertNotNull(tablePath);
+    assertFalse(client.existTable(tableName1));
+
+    client.createExternalTable("table3", BackendTestingUtil.mockupSchema, tablePath, BackendTestingUtil.mockupMeta);
+    assertTrue(client.existTable(tableName1));
+
+    TableDesc desc = client.getTableDesc(tableName1);
+    assertNotNull(desc);
+    assertEquals(CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, tableName1), desc.getName());
+    assertTrue(desc.getStats().getNumBytes() > 0);
+  }
+
+  //@Test
+  public final void testCreateAndDropTablePartitionedHash1ByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedHash1ByExecuteQuery";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += " PARTITION BY HASH (deptname)";
+    sql += " (PARTITION sub_part1, PARTITION sub_part2, PARTITION sub_part3)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable(tableName));
+    assertTrue(hdfs.exists(tablePath));
+  }
+
+  //@Test
+  public final void testCreateAndPurgeTablePartitionedHash1ByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndPurgeTablePartitionedHash1ByExecuteQuery";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += " PARTITION BY HASH (deptname)";
+    sql += " (PARTITION sub_part1, PARTITION sub_part2, PARTITION sub_part3)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName + " purge");
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  //@Test
+  public final void testCreateAndDropTablePartitionedHash2ByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedHash2ByExecuteQuery";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY HASH (deptname)";
+    sql += "PARTITIONS 2";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName + " purge");
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  //@Test
+  public final void testCreateAndDropTablePartitionedListByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedListByExecuteQuery";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY LIST (deptname)";
+    sql += "( PARTITION sub_part1 VALUES('r&d', 'design'),";
+    sql += "PARTITION sub_part2 VALUES('sales', 'hr') )";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName + " purge");
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  //@Test
+  public final void testCreateAndDropTablePartitionedRangeByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedRangeByExecuteQuery";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY RANGE (score)";
+    sql += "( PARTITION sub_part1 VALUES LESS THAN (2),";
+    sql += "PARTITION sub_part2 VALUES LESS THAN (5),";
+    sql += "PARTITION sub_part2 VALUES LESS THAN (MAXVALUE) )";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName +" purge");
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  @Test
+  public final void testFailCreateTablePartitionedOtherExceptColumn() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testFailCreateTablePartitionedOtherExceptColumn";
+
+    assertFalse(client.existTable(tableName));
+
+    String rangeSql = "create table " + tableName + " (deptname text, score int4)";
+    rangeSql += "PARTITION BY RANGE (score)";
+    rangeSql += "( PARTITION sub_part1 VALUES LESS THAN (2),";
+    rangeSql += "PARTITION sub_part2 VALUES LESS THAN (5),";
+    rangeSql += "PARTITION sub_part2 VALUES LESS THAN (MAXVALUE) )";
+
+    assertFalse(client.updateQuery(rangeSql));
+ 
+    String listSql = "create table " + tableName + " (deptname text, score int4)";
+    listSql += "PARTITION BY LIST (deptname)";
+    listSql += "( PARTITION sub_part1 VALUES('r&d', 'design'),";
+    listSql += "PARTITION sub_part2 VALUES('sales', 'hr') )";
+
+    assertFalse(client.updateQuery(listSql));
+
+    String hashSql = "create table " + tableName + " (deptname text, score int4)";
+    hashSql += "PARTITION BY HASH (deptname)";
+    hashSql += "PARTITIONS 2";
+
+    assertFalse(client.updateQuery(hashSql));
+  }
+
+  @Test
+  public final void testCreateAndDropTablePartitionedColumnByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropTablePartitionedColumnByExecuteQuery");
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY COLUMN (key1 text)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName + " purge");
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  @Test
+  public final void testGetFunctions() throws IOException,
+      ServiceException, SQLException {
+    Collection<FunctionDesc> catalogFunctions = cluster.getMaster().getCatalog().getFunctions();
+    String functionName = "sum";
+    int numFunctions = 0;
+    for(FunctionDesc eachFunction: catalogFunctions) {
+      if(functionName.equals(eachFunction.getSignature())) {
+        numFunctions++;
+      }
+    }
+
+    List<CatalogProtos.FunctionDescProto> functions = client.getFunctions(functionName);
+    assertEquals(numFunctions, functions.size());
+
+    functions = client.getFunctions("notmatched");
+    assertEquals(0, functions.size());
+
+    functions = client.getFunctions(null);
+    assertEquals(catalogFunctions.size(), functions.size());
+  }
+
+  @Test
+  public final void testGetFinishedQueryList() throws IOException,
+      ServiceException, SQLException {
+    final String tableName = CatalogUtil.normalizeIdentifier("testGetFinishedQueryList");
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    int numFinishedQueries = client.getFinishedQueryList().size();
+    ResultSet resultSet = client.executeQueryAndGetResult("select * from " + tableName + " order by deptname");
+    assertNotNull(resultSet);
+
+    resultSet = client.executeQueryAndGetResult("select * from " + tableName + " order by deptname");
+    assertNotNull(resultSet);
+    assertEquals(numFinishedQueries + 2, client.getFinishedQueryList().size());
+
+    resultSet.close();
+  }
+
+  /**
+   * The main objective of this test is to get the status of a query which is actually finished.
+   * Statuses of queries regardless of its status should be available for a specified time duration.
+   */
+  @Test(timeout = 20 * 1000)
+  public final void testGetQueryStatusAndResultAfterFinish() throws Exception {
+    String sql = "select * from lineitem order by l_orderkey";
+    ClientProtos.SubmitQueryResponse response = client.executeQuery(sql);
+
+    assertNotNull(response);
+    QueryId queryId = new QueryId(response.getQueryId());
+
+    try {
+      long startTime = System.currentTimeMillis();
+      while (true) {
+        Thread.sleep(5 * 1000);
+
+        List<ClientProtos.BriefQueryInfo> finishedQueries = client.getFinishedQueryList();
+        boolean finished = false;
+        if (finishedQueries != null) {
+          for (ClientProtos.BriefQueryInfo eachQuery: finishedQueries) {
+            if (eachQuery.getQueryId().equals(queryId.getProto())) {
+              finished = true;
+              break;
+            }
+          }
+        }
+
+        if (finished) {
+          break;
+        }
+        if(System.currentTimeMillis() - startTime > 20 * 1000) {
+          fail("Too long time execution query");
+        }
+      }
+
+      QueryStatus queryStatus = client.getQueryStatus(queryId);
+      assertNotNull(queryStatus);
+      assertTrue(!TajoClient.isQueryRunnning(queryStatus.getState()));
+
+      TajoResultSet resultSet = (TajoResultSet) client.getQueryResult(queryId);
+      assertNotNull(resultSet);
+
+      int count = 0;
+      while(resultSet.next()) {
+        count++;
+      }
+
+      assertEquals(5, count);
+    } finally {
+      client.closeQuery(queryId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java b/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java
new file mode 100644
index 0000000..513187d
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java
@@ -0,0 +1,102 @@
+/**
+ * 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.cluster;
+
+import org.junit.Test;
+import org.apache.tajo.master.cluster.ServerName;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestServerName {
+
+	@Test
+	public void testServerNameStringInt() {
+		ServerName server = new ServerName("ex1.com",50030);
+		assertEquals("ex1.com", server.getHostname());
+		assertEquals(50030, server.getPort());
+	}
+
+	@Test
+	public void testServerNameString() {
+		ServerName server = new ServerName("ex1.com:50030");
+		assertEquals("ex1.com", server.getHostname());
+		assertEquals(50030, server.getPort());
+	}
+
+	@Test
+	public void testParseHostname() {
+		assertEquals("ex1.com",ServerName.parseHostname("ex1.com:50030"));
+	}
+
+	@Test
+	public void testParsePort() {
+		assertEquals(50030,ServerName.parsePort("ex1.com:50030"));
+	}
+
+	@Test
+	public void testToString() {
+		ServerName server = new ServerName("ex1.com",50030);
+		assertEquals("ex1.com:50030", server.toString());
+	}
+
+	@Test
+	public void testGetServerName() {
+		ServerName server = new ServerName("ex1.com",50030);
+		assertEquals("ex1.com:50030", server.getServerName());
+	}
+
+	@Test
+	public void testGetHostname() {
+		ServerName server = new ServerName("ex1.com",50030);
+		assertEquals("ex1.com", server.getHostname());
+	}
+
+	@Test
+	public void testGetPort() {
+		ServerName server = new ServerName("ex1.com",50030);
+		assertEquals(50030, server.getPort());
+	}
+
+	@Test
+	public void testGetServerNameStringInt() {
+		assertEquals("ex2.com:50030",ServerName.getServerName("ex2.com", 50030));
+	}
+
+	@Test
+	public void testCompareTo() {
+		ServerName s1 = new ServerName("ex1.com:50030");
+		ServerName s2 = new ServerName("ex1.com:60030");
+		
+		assertTrue(s1.compareTo(s2) < 0);
+		assertTrue(s2.compareTo(s1) > 0);
+		
+		ServerName s3 = new ServerName("ex1.com:50030");
+		assertTrue(s1.compareTo(s3) == 0);
+		
+		ServerName s4 = new ServerName("ex2.com:50030");
+		assertTrue(s1.compareTo(s4) < 0);
+		assertTrue(s4.compareTo(s1) > 0);
+	}
+
+  @Test (expected = IllegalArgumentException.class)
+  public void testException() {
+    new ServerName("ex1.com");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
new file mode 100644
index 0000000..d19b4d6
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -0,0 +1,210 @@
+/**
+ * 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.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.cli.InvalidStatementException;
+import org.apache.tajo.cli.ParsedResult;
+import org.apache.tajo.cli.SimpleParser;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.LazyTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.util.Bytes;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class ExprTestBase {
+  private static TajoTestingCluster util;
+  private static CatalogService cat;
+  private static SQLAnalyzer analyzer;
+  private static PreLogicalPlanVerifier preLogicalPlanVerifier;
+  private static LogicalPlanner planner;
+  private static LogicalOptimizer optimizer;
+  private static LogicalPlanVerifier annotatedPlanVerifier;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.startCatalogCluster();
+    cat = util.getMiniCatalogCluster().getCatalog();
+    cat.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse");
+    cat.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+    for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+      cat.createFunction(funcDesc);
+    }
+
+    analyzer = new SQLAnalyzer();
+    preLogicalPlanVerifier = new PreLogicalPlanVerifier(cat);
+    planner = new LogicalPlanner(cat);
+    optimizer = new LogicalOptimizer(util.getConfiguration());
+    annotatedPlanVerifier = new LogicalPlanVerifier(util.getConfiguration(), cat);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  private static void assertJsonSerDer(EvalNode expr) {
+    String json = CoreGsonHelper.toJson(expr, EvalNode.class);
+    EvalNode fromJson = CoreGsonHelper.fromJson(json, EvalNode.class);
+    assertEquals(expr, fromJson);
+  }
+
+  /**
+   * verify query syntax and get raw targets.
+   *
+   * @param query a query for execution
+   * @param condition this parameter means whether it is for success case or is not for failure case.
+   * @return
+   * @throws PlanningException
+   */
+  private static Target[] getRawTargets(String query, boolean condition) throws PlanningException,
+      InvalidStatementException {
+
+    Session session = LocalTajoTestingUtility.createDummySession();
+    List<ParsedResult> parsedResults = SimpleParser.parseScript(query);
+    if (parsedResults.size() > 1) {
+      throw new RuntimeException("this query includes two or more statements.");
+    }
+    Expr expr = analyzer.parse(parsedResults.get(0).getStatement());
+    VerificationState state = new VerificationState();
+    preLogicalPlanVerifier.verify(session, state, expr);
+    if (state.getErrorMessages().size() > 0) {
+      if (!condition && state.getErrorMessages().size() > 0) {
+        throw new PlanningException(state.getErrorMessages().get(0));
+      }
+      assertFalse(state.getErrorMessages().get(0), true);
+    }
+    LogicalPlan plan = planner.createPlan(session, expr, true);
+    optimizer.optimize(plan);
+    annotatedPlanVerifier.verify(session, state, plan);
+
+    if (state.getErrorMessages().size() > 0) {
+      assertFalse(state.getErrorMessages().get(0), true);
+    }
+
+    Target [] targets = plan.getRootBlock().getRawTargets();
+    if (targets == null) {
+      throw new PlanningException("Wrong query statement or query plan: " + parsedResults.get(0).getStatement());
+    }
+    for (Target t : targets) {
+      assertJsonSerDer(t.getEvalTree());
+    }
+    return targets;
+  }
+
+  public void testSimpleEval(String query, String [] expected) throws IOException {
+    testEval(null, null, null, query, expected);
+  }
+
+  public void testSimpleEval(String query, String [] expected, boolean condition) throws IOException {
+    testEval(null, null, null, query, expected, ',', condition);
+  }
+
+  public void testEval(Schema schema, String tableName, String csvTuple, String query, String [] expected)
+      throws IOException {
+    testEval(schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple, query,
+        expected, ',', true);
+  }
+
+  public void testEval(Schema schema, String tableName, String csvTuple, String query, String [] expected,
+                       char delimiter, boolean condition) throws IOException {
+    LazyTuple lazyTuple;
+    VTuple vtuple  = null;
+    String qualifiedTableName =
+        CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME,
+            tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null);
+    Schema inputSchema = null;
+    if (schema != null) {
+      inputSchema = SchemaUtil.clone(schema);
+      inputSchema.setQualifier(qualifiedTableName);
+
+      int targetIdx [] = new int[inputSchema.size()];
+      for (int i = 0; i < targetIdx.length; i++) {
+        targetIdx[i] = i;
+      }
+
+      lazyTuple =
+          new LazyTuple(inputSchema, Bytes.splitPreserveAllTokens(csvTuple.getBytes(), delimiter, targetIdx),0);
+      vtuple = new VTuple(inputSchema.size());
+      for (int i = 0; i < inputSchema.size(); i++) {
+        // If null value occurs, null datum is manually inserted to an input tuple.
+        if (lazyTuple.get(i) instanceof TextDatum && lazyTuple.get(i).asChars().equals("")) {
+          vtuple.put(i, NullDatum.get());
+        } else {
+          vtuple.put(i, lazyTuple.get(i));
+        }
+      }
+      cat.createTable(new TableDesc(qualifiedTableName, inputSchema,
+          CatalogProtos.StoreType.CSV, new Options(), CommonTestingUtil.getTestDir()));
+    }
+
+    Target [] targets;
+
+    try {
+      targets = getRawTargets(query, condition);
+
+      Tuple outTuple = new VTuple(targets.length);
+      for (int i = 0; i < targets.length; i++) {
+        EvalNode eval = targets[i].getEvalTree();
+        outTuple.put(i, eval.eval(inputSchema, vtuple));
+      }
+
+      for (int i = 0; i < expected.length; i++) {
+        assertEquals(query, expected[i], outTuple.get(i).asChars());
+      }
+    } catch (InvalidStatementException e) {
+      assertFalse(e.getMessage(), true);
+    } catch (PlanningException e) {
+      // In failure test case, an exception must occur while executing query.
+      // So, we should check an error message, and return it.
+      if (!condition) {
+        assertEquals(expected[0], e.getMessage());
+      } else {
+        assertFalse(e.getMessage(), true);
+      }
+    } finally {
+      if (schema != null) {
+        cat.dropTable(qualifiedTableName);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java
new file mode 100644
index 0000000..9d7e438
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java
@@ -0,0 +1,323 @@
+/**
+ * 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.DataType;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.junit.Test;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.*;
+import static org.junit.Assert.*;
+
+public class TestEvalTree extends ExprTestBase{
+  @Test
+  public void testTupleEval() throws CloneNotSupportedException {
+    ConstEval e1 = new ConstEval(DatumFactory.createInt4(1));
+    assertCloneEqual(e1);
+    FieldEval e2 = new FieldEval("table1.score", CatalogUtil.newSimpleDataType(INT4)); // it indicates
+    assertCloneEqual(e2);
+
+    Schema schema1 = new Schema();
+    schema1.addColumn("table1.id", INT4);
+    schema1.addColumn("table1.score", INT4);
+    
+    BinaryEval expr = new BinaryEval(EvalType.PLUS, e1, e2);
+    assertCloneEqual(expr);
+    VTuple tuple = new VTuple(2);
+    tuple.put(0, DatumFactory.createInt4(1)); // put 0th field
+    tuple.put(1, DatumFactory.createInt4(99)); // put 0th field
+
+    // the result of evaluation must be 100.
+    assertEquals(expr.eval(schema1, tuple).asInt4(), 100);
+  }
+
+  public static class MockTrueEval extends EvalNode {
+
+    public MockTrueEval() {
+      super(EvalType.CONST);
+    }
+
+    @Override
+    public String getName() {
+      return this.getClass().getName();
+    }
+
+    @Override
+    public Datum eval(Schema schema, Tuple tuple) {
+      return DatumFactory.createBool(true);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return true;
+    }
+
+    @Override
+    public DataType getValueType() {
+      return CatalogUtil.newSimpleDataType(BOOLEAN);
+    }
+
+  }
+
+  public static class MockFalseExpr extends EvalNode {
+
+    public MockFalseExpr() {
+      super(EvalType.CONST);
+    }
+
+    @Override
+    public Datum eval(Schema schema, Tuple tuple) {
+      return DatumFactory.createBool(false);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return true;
+    }
+
+    @Override
+    public String getName() {
+      return this.getClass().getName();
+    }
+
+    @Override
+    public DataType getValueType() {
+      return CatalogUtil.newSimpleDataType(BOOLEAN);
+    }
+  }
+
+  @Test
+  public void testAndTest() {
+    MockTrueEval trueExpr = new MockTrueEval();
+    MockFalseExpr falseExpr = new MockFalseExpr();
+
+    BinaryEval andExpr = new BinaryEval(EvalType.AND, trueExpr, trueExpr);
+    assertTrue(andExpr.eval(null, null).asBool());
+
+    andExpr = new BinaryEval(EvalType.AND, falseExpr, trueExpr);
+    assertFalse(andExpr.eval(null, null).asBool());
+
+    andExpr = new BinaryEval(EvalType.AND, trueExpr, falseExpr);
+    assertFalse(andExpr.eval(null, null).asBool());
+
+    andExpr = new BinaryEval(EvalType.AND, falseExpr, falseExpr);
+    assertFalse(andExpr.eval(null, null).asBool());
+  }
+
+  @Test
+  public void testOrTest() {
+    MockTrueEval trueExpr = new MockTrueEval();
+    MockFalseExpr falseExpr = new MockFalseExpr();
+
+    BinaryEval orExpr = new BinaryEval(EvalType.OR, trueExpr, trueExpr);
+    assertTrue(orExpr.eval(null, null).asBool());
+
+    orExpr = new BinaryEval(EvalType.OR, falseExpr, trueExpr);
+    assertTrue(orExpr.eval(null, null).asBool());
+
+    orExpr = new BinaryEval(EvalType.OR, trueExpr, falseExpr);
+    assertTrue(orExpr.eval(null, null).asBool());
+
+    orExpr = new BinaryEval(EvalType.OR, falseExpr, falseExpr);
+    assertFalse(orExpr.eval(null, null).asBool());
+  }
+
+  @Test
+  public final void testCompOperator() {
+    ConstEval e1;
+    ConstEval e2;
+    BinaryEval expr;
+
+    // Constant
+    e1 = new ConstEval(DatumFactory.createInt4(9));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
+    expr = new BinaryEval(EvalType.LTH, e1, e2);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.LEQ, e1, e2);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.LTH, e2, e1);
+    assertFalse(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.LEQ, e2, e1);
+    assertFalse(expr.eval(null, null).asBool());
+
+    expr = new BinaryEval(EvalType.GTH, e2, e1);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.GEQ, e2, e1);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.GTH, e1, e2);
+    assertFalse(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.GEQ, e1, e2);
+    assertFalse(expr.eval(null, null).asBool());
+
+    BinaryEval plus = new BinaryEval(EvalType.PLUS, e1, e2);
+    expr = new BinaryEval(EvalType.LTH, e1, plus);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.LEQ, e1, plus);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.LTH, plus, e1);
+    assertFalse(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.LEQ, plus, e1);
+    assertFalse(expr.eval(null, null).asBool());
+
+    expr = new BinaryEval(EvalType.GTH, plus, e1);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.GEQ, plus, e1);
+    assertTrue(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.GTH, e1, plus);
+    assertFalse(expr.eval(null, null).asBool());
+    expr = new BinaryEval(EvalType.GEQ, e1, plus);
+    assertFalse(expr.eval(null, null).asBool());
+  }
+
+  @Test
+  public final void testArithmaticsOperator() 
+      throws CloneNotSupportedException {
+    ConstEval e1;
+    ConstEval e2;
+
+    // PLUS
+    e1 = new ConstEval(DatumFactory.createInt4(9));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
+    BinaryEval expr = new BinaryEval(EvalType.PLUS, e1, e2);
+    assertEquals(expr.eval(null, null).asInt4(), 43);
+    assertCloneEqual(expr);
+    
+    // MINUS
+    e1 = new ConstEval(DatumFactory.createInt4(5));
+    e2 = new ConstEval(DatumFactory.createInt4(2));
+    expr = new BinaryEval(EvalType.MINUS, e1, e2);
+    assertEquals(expr.eval(null, null).asInt4(), 3);
+    assertCloneEqual(expr);
+    
+    // MULTIPLY
+    e1 = new ConstEval(DatumFactory.createInt4(5));
+    e2 = new ConstEval(DatumFactory.createInt4(2));
+    expr = new BinaryEval(EvalType.MULTIPLY, e1, e2);
+    assertEquals(expr.eval(null, null).asInt4(), 10);
+    assertCloneEqual(expr);
+    
+    // DIVIDE
+    e1 = new ConstEval(DatumFactory.createInt4(10));
+    e2 = new ConstEval(DatumFactory.createInt4(5));
+    expr = new BinaryEval(EvalType.DIVIDE, e1, e2);
+    assertEquals(expr.eval(null, null).asInt4(), 2);
+    assertCloneEqual(expr);
+  }
+
+  @Test
+  public final void testGetReturnType() {
+    ConstEval e1;
+    ConstEval e2;
+
+    // PLUS
+    e1 = new ConstEval(DatumFactory.createInt4(9));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
+    BinaryEval expr = new BinaryEval(EvalType.PLUS, e1, e2);
+    assertEquals(CatalogUtil.newSimpleDataType(INT4), expr.getValueType());
+
+    expr = new BinaryEval(EvalType.LTH, e1, e2);
+    assertTrue(expr.eval(null, null).asBool());
+    assertEquals(CatalogUtil.newSimpleDataType(BOOLEAN), expr.getValueType());
+
+    e1 = new ConstEval(DatumFactory.createFloat8(9.3));
+    e2 = new ConstEval(DatumFactory.createFloat8(34.2));
+    expr = new BinaryEval(EvalType.PLUS, e1, e2);
+    assertEquals(CatalogUtil.newSimpleDataType(FLOAT8), expr.getValueType());
+  }
+  
+  @Test
+  public final void testEquals() throws CloneNotSupportedException {
+    ConstEval e1;
+    ConstEval e2;
+
+    // PLUS
+    e1 = new ConstEval(DatumFactory.createInt4(34));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
+    assertEquals(e1, e2);
+    
+    BinaryEval plus1 = new BinaryEval(EvalType.PLUS, e1, e2);
+    BinaryEval plus2 = new BinaryEval(EvalType.PLUS, e2, e1);
+    assertEquals(plus1, plus2);
+    
+    ConstEval e3 = new ConstEval(DatumFactory.createInt4(29));
+    BinaryEval plus3 = new BinaryEval(EvalType.PLUS, e1, e3);
+    assertFalse(plus1.equals(plus3));
+    
+    // LTH
+    ConstEval e4 = new ConstEval(DatumFactory.createInt4(9));
+    ConstEval e5 = new ConstEval(DatumFactory.createInt4(34));
+    BinaryEval compExpr1 = new BinaryEval(EvalType.LTH, e4, e5);
+    assertCloneEqual(compExpr1);
+    
+    ConstEval e6 = new ConstEval(DatumFactory.createInt4(9));
+    ConstEval e7 = new ConstEval(DatumFactory.createInt4(34));
+    BinaryEval compExpr2 = new BinaryEval(EvalType.LTH, e6, e7);
+    assertCloneEqual(compExpr2);
+    
+    assertTrue(compExpr1.equals(compExpr2));
+  }
+  
+  @Test
+  public final void testJson() throws CloneNotSupportedException {
+    ConstEval e1;
+    ConstEval e2;
+
+    // 29 > (34 + 5) + (5 + 34)
+    e1 = new ConstEval(DatumFactory.createInt4(34));
+    e2 = new ConstEval(DatumFactory.createInt4(5));
+    assertCloneEqual(e1); 
+    
+    BinaryEval plus1 = new BinaryEval(EvalType.PLUS, e1, e2);
+    assertCloneEqual(plus1);
+    BinaryEval plus2 = new BinaryEval(EvalType.PLUS, e2, e1);
+    assertCloneEqual(plus2);
+    BinaryEval plus3 = new BinaryEval(EvalType.PLUS, plus2, plus1);
+    assertCloneEqual(plus3);
+    
+    ConstEval e3 = new ConstEval(DatumFactory.createInt4(29));
+    BinaryEval gth = new BinaryEval(EvalType.GTH, e3, plus3);
+    assertCloneEqual(gth);
+    
+    String json = gth.toJson();
+    EvalNode eval = CoreGsonHelper.fromJson(json, EvalNode.class);
+    assertCloneEqual(eval);
+    
+    assertEquals(gth.getType(), eval.getType());
+    assertEquals(e3.getType(), eval.getLeftExpr().getType());
+    assertEquals(plus3.getType(), eval.getRightExpr().getType());
+    assertEquals(plus3.getLeftExpr(), eval.getRightExpr().getLeftExpr());
+    assertEquals(plus3.getRightExpr(), eval.getRightExpr().getRightExpr());
+    assertEquals(plus2.getLeftExpr(), eval.getRightExpr().getLeftExpr().getLeftExpr());
+    assertEquals(plus2.getRightExpr(), eval.getRightExpr().getLeftExpr().getRightExpr());
+    assertEquals(plus1.getLeftExpr(), eval.getRightExpr().getRightExpr().getLeftExpr());
+    assertEquals(plus1.getRightExpr(), eval.getRightExpr().getRightExpr().getRightExpr());
+  }
+  
+  private void assertCloneEqual(EvalNode eval) throws CloneNotSupportedException {
+    EvalNode copy = (EvalNode) eval.clone();
+    assertEquals(eval, copy);
+    assertFalse(eval == copy);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
new file mode 100644
index 0000000..d916bbc
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
@@ -0,0 +1,359 @@
+/**
+ * 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 org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.algebra.OpType;
+import org.apache.tajo.algebra.Selection;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+import static org.apache.tajo.common.TajoDataTypes.Type.INT4;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestEvalTreeUtil {
+  static TajoTestingCluster util;
+  static CatalogService catalog = null;
+  static EvalNode expr1;
+  static EvalNode expr2;
+  static EvalNode expr3;
+  static SQLAnalyzer analyzer;
+  static LogicalPlanner planner;
+  static Session session = LocalTajoTestingUtility.createDummySession();
+
+  public static class TestSum extends GeneralFunction {
+    private Integer x;
+    private Integer y;
+
+    public TestSum() {
+      super(new Column[] { new Column("arg1", INT4),
+          new Column("arg2", INT4) });
+    }
+
+    @Override
+    public Datum eval(Tuple params) {
+      x =  params.get(0).asInt4();
+      y =  params.get(1).asInt4();
+      return DatumFactory.createInt4(x + y);
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    util = new TajoTestingCluster();
+    util.startCatalogCluster();
+    catalog = util.getMiniCatalogCluster().getCatalog();
+    for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+      catalog.createFunction(funcDesc);
+    }
+    catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse");
+    catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
+
+    Schema schema = new Schema();
+    schema.addColumn("name", TajoDataTypes.Type.TEXT);
+    schema.addColumn("score", TajoDataTypes.Type.INT4);
+    schema.addColumn("age", TajoDataTypes.Type.INT4);
+
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
+    TableDesc desc = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "people"), schema, meta,
+        CommonTestingUtil.getTestDir());
+    catalog.createTable(desc);
+
+    FunctionDesc funcMeta = new FunctionDesc("test_sum", TestSum.class,
+        FunctionType.GENERAL,
+        CatalogUtil.newSimpleDataType(TajoDataTypes.Type.INT4),
+        CatalogUtil.newSimpleDataTypeArray(TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT4));
+    catalog.createFunction(funcMeta);
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+
+    String[] QUERIES = {
+        "select name, score, age from people where score > 30", // 0
+        "select name, score, age from people where score * age", // 1
+        "select name, score, age from people where test_sum(score * age, 50)", // 2
+    };
+
+    expr1 = getRootSelection(QUERIES[0]);
+    expr2 = getRootSelection(QUERIES[1]);
+    expr3 = getRootSelection(QUERIES[2]);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownCatalogCluster();
+  }
+
+  public static Target [] getRawTargets(String query) {
+    Expr expr = analyzer.parse(query);
+    LogicalPlan plan = null;
+    try {
+      plan = planner.createPlan(session, expr);
+    } catch (PlanningException e) {
+      e.printStackTrace();
+    }
+
+    return plan.getRootBlock().getRawTargets();
+  }
+
+  public static EvalNode getRootSelection(String query) throws PlanningException {
+    Expr block = analyzer.parse(query);
+    LogicalPlan plan = null;
+    try {
+      plan = planner.createPlan(session, block);
+    } catch (PlanningException e) {
+      e.printStackTrace();
+    }
+
+    Selection selection = plan.getRootBlock().getSingletonExpr(OpType.Filter);
+    return planner.getExprAnnotator().createEvalNode(plan, plan.getRootBlock(), selection.getQual());
+  }
+
+  @Test
+  public final void testChangeColumnRef() throws CloneNotSupportedException {
+    EvalNode copy = (EvalNode)expr1.clone();
+    EvalTreeUtil.changeColumnRef(copy, "default.people.score", "newscore");
+    Set<Column> set = EvalTreeUtil.findUniqueColumns(copy);
+    assertEquals(1, set.size());
+    assertTrue(set.contains(new Column("newscore", TajoDataTypes.Type.INT4)));
+
+    copy = (EvalNode)expr2.clone();
+    EvalTreeUtil.changeColumnRef(copy, "default.people.age", "sum_age");
+    set = EvalTreeUtil.findUniqueColumns(copy);
+    assertEquals(2, set.size());
+    assertTrue(set.contains(new Column("default.people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("sum_age", TajoDataTypes.Type.INT4)));
+
+    copy = (EvalNode)expr3.clone();
+    EvalTreeUtil.changeColumnRef(copy, "default.people.age", "sum_age");
+    set = EvalTreeUtil.findUniqueColumns(copy);
+    assertEquals(2, set.size());
+    assertTrue(set.contains(new Column("default.people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("sum_age", TajoDataTypes.Type.INT4)));
+  }
+
+  @Test
+  public final void testFindAllRefColumns() {    
+    Set<Column> set = EvalTreeUtil.findUniqueColumns(expr1);
+    assertEquals(1, set.size());
+    assertTrue(set.contains(new Column("default.people.score", TajoDataTypes.Type.INT4)));
+    
+    set = EvalTreeUtil.findUniqueColumns(expr2);
+    assertEquals(2, set.size());
+    assertTrue(set.contains(new Column("default.people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("default.people.age", TajoDataTypes.Type.INT4)));
+    
+    set = EvalTreeUtil.findUniqueColumns(expr3);
+    assertEquals(2, set.size());
+    assertTrue(set.contains(new Column("default.people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("default.people.age", TajoDataTypes.Type.INT4)));
+  }
+  
+  public static final String [] QUERIES = {
+    "select 3 + 4 as plus, (3.5 * 2) as mul", // 0
+    "select (score + 3) < 4, age > 5 from people", // 1
+    "select score from people where score > 7", // 2
+    "select score from people where (10 * 2) * (score + 2) > 20 + 30 + 10", // 3
+    "select score from people where 10 * 2 > score * 10", // 4
+    "select score from people where score < 10 and 4 < score", // 5
+    "select score from people where score < 10 and 4 < score and age > 5", // 6
+    "select score from people where (score > 1 and score < 3) or (7 < score and score < 10)", // 7
+  };
+  
+  @Test
+  public final void testGetSchemaFromTargets() throws InternalException {
+    Target [] targets = getRawTargets(QUERIES[0]);
+    Schema schema = EvalTreeUtil.getSchemaByTargets(null, targets);
+    Column col1 = schema.getColumn(0);
+    Column col2 = schema.getColumn(1);
+    assertEquals("plus", col1.getSimpleName());
+    assertEquals(TajoDataTypes.Type.INT4, col1.getDataType().getType());
+    assertEquals("mul", col2.getSimpleName());
+    assertEquals(TajoDataTypes.Type.FLOAT8, col2.getDataType().getType());
+  }
+
+  @Test
+  public final void testGetContainExprs() throws CloneNotSupportedException, PlanningException {
+    Expr expr = analyzer.parse(QUERIES[1]);
+    LogicalPlan plan = planner.createPlan(session, expr, true);
+    Target [] targets = plan.getRootBlock().getRawTargets();
+    Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4);
+    Collection<EvalNode> exprs =
+        EvalTreeUtil.getContainExpr(targets[0].getEvalTree(), col1);
+    EvalNode node = exprs.iterator().next();
+    assertEquals(EvalType.LTH, node.getType());
+    assertEquals(EvalType.PLUS, node.getLeftExpr().getType());
+    assertEquals(new ConstEval(DatumFactory.createInt4(4)), node.getRightExpr());
+
+    Column col2 = new Column("default.people.age", TajoDataTypes.Type.INT4);
+    exprs = EvalTreeUtil.getContainExpr(targets[1].getEvalTree(), col2);
+    node = exprs.iterator().next();
+    assertEquals(EvalType.GTH, node.getType());
+    assertEquals("default.people.age", node.getLeftExpr().getName());
+    assertEquals(new ConstEval(DatumFactory.createInt4(5)), node.getRightExpr());
+  }
+  
+  @Test
+  public final void testGetCNF() throws PlanningException {
+    // "select score from people where score < 10 and 4 < score "
+    EvalNode node = getRootSelection(QUERIES[5]);
+    EvalNode [] cnf = AlgebraicUtil.toConjunctiveNormalFormArray(node);
+    
+    Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4);
+    
+    assertEquals(2, cnf.length);
+    EvalNode first = cnf[0];
+    EvalNode second = cnf[1];
+    
+    FieldEval field = first.getLeftExpr();
+    assertEquals(col1, field.getColumnRef());
+    assertEquals(EvalType.LTH, first.getType());
+    assertEquals(10, first.getRightExpr().eval(null,  null).asInt4());
+    
+    field = second.getRightExpr();
+    assertEquals(col1, field.getColumnRef());
+    assertEquals(EvalType.LTH, second.getType());
+    assertEquals(4, second.getLeftExpr().eval(null,  null).asInt4());
+  }
+  
+  @Test
+  public final void testTransformCNF2Singleton() throws PlanningException {
+    // "select score from people where score < 10 and 4 < score "
+    EvalNode node = getRootSelection(QUERIES[6]);
+    EvalNode [] cnf1 = AlgebraicUtil.toConjunctiveNormalFormArray(node);
+    assertEquals(3, cnf1.length);
+    
+    EvalNode conj = AlgebraicUtil.createSingletonExprFromCNF(cnf1);
+    EvalNode [] cnf2 = AlgebraicUtil.toConjunctiveNormalFormArray(conj);
+    
+    Set<EvalNode> set1 = Sets.newHashSet(cnf1);
+    Set<EvalNode> set2 = Sets.newHashSet(cnf2);
+    assertEquals(set1, set2);
+  }
+
+  @Test
+  public final void testGetDNF() throws PlanningException {
+    // "select score from people where score > 1 and score < 3 or score > 7 and score < 10", // 7
+    EvalNode node = getRootSelection(QUERIES[7]);
+    EvalNode [] cnf = AlgebraicUtil.toDisjunctiveNormalFormArray(node);
+    assertEquals(2, cnf.length);
+
+    assertEquals("default.people.score (INT4) > 1 AND default.people.score (INT4) < 3", cnf[0].toString());
+    assertEquals("7 < default.people.score (INT4) AND default.people.score (INT4) < 10", cnf[1].toString());
+  }
+  
+  @Test
+  public final void testSimplify() throws PlanningException {
+    Target [] targets = getRawTargets(QUERIES[0]);
+    EvalNode node = AlgebraicUtil.eliminateConstantExprs(targets[0].getEvalTree());
+    assertEquals(EvalType.CONST, node.getType());
+    assertEquals(7, node.eval(null, null).asInt4());
+    node = AlgebraicUtil.eliminateConstantExprs(targets[1].getEvalTree());
+    assertEquals(EvalType.CONST, node.getType());
+    assertTrue(7.0d == node.eval(null, null).asFloat8());
+
+    Expr expr = analyzer.parse(QUERIES[1]);
+    LogicalPlan plan = planner.createPlan(session, expr, true);
+    targets = plan.getRootBlock().getRawTargets();
+    Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4);
+    Collection<EvalNode> exprs =
+        EvalTreeUtil.getContainExpr(targets[0].getEvalTree(), col1);
+    node = exprs.iterator().next();
+  }
+  
+  @Test
+  public final void testConatainSingleVar() throws PlanningException {
+    EvalNode node = getRootSelection(QUERIES[2]);
+    assertEquals(true, AlgebraicUtil.containSingleVar(node));
+    node = getRootSelection(QUERIES[3]);
+    assertEquals(true, AlgebraicUtil.containSingleVar(node));
+  }
+  
+  @Test
+  public final void testTranspose() throws PlanningException {
+    Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4);
+    EvalNode node = getRootSelection(QUERIES[3]);
+    // we expect that score < 3
+    EvalNode transposed = AlgebraicUtil.transpose(node, col1);
+    assertEquals(EvalType.GTH, transposed.getType());
+    FieldEval field = transposed.getLeftExpr();
+    assertEquals(col1, field.getColumnRef());
+    assertEquals(1, transposed.getRightExpr().eval(null, null).asInt4());
+
+    node = getRootSelection(QUERIES[4]);
+    // we expect that score < 3
+    transposed = AlgebraicUtil.transpose(node, col1);
+    assertEquals(EvalType.LTH, transposed.getType());
+    field = transposed.getLeftExpr();
+    assertEquals(col1, field.getColumnRef());
+    assertEquals(2, transposed.getRightExpr().eval(null, null).asInt4());
+  }
+
+  @Test
+  public final void testFindDistinctAggFunctions() throws PlanningException {
+    String query = "select sum(score) + max(age) from people";
+    Expr expr = analyzer.parse(query);
+    LogicalPlan plan = planner.createPlan(session, expr);
+    GroupbyNode groupByNode = plan.getRootBlock().getNode(NodeType.GROUP_BY);
+    EvalNode [] aggEvals = groupByNode.getAggFunctions();
+
+    List<AggregationFunctionCallEval> list = new ArrayList<AggregationFunctionCallEval>();
+    for (int i = 0; i < aggEvals.length; i++) {
+      list.addAll(EvalTreeUtil.findDistinctAggFunction(aggEvals[i]));
+    }
+    assertEquals(2, list.size());
+
+    Set<String> result = Sets.newHashSet("max", "sum");
+    for (AggregationFunctionCallEval eval : list) {
+      assertTrue(result.contains(eval.getName()));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
new file mode 100644
index 0000000..7811e69
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
@@ -0,0 +1,396 @@
+/**
+ * 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.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.BOOLEAN;
+import static org.apache.tajo.common.TajoDataTypes.Type.INT4;
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+public class TestPredicates extends ExprTestBase {
+
+  //////////////////////////////////////////////////////////////////
+  // Logical Operator
+  //////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testAnd() throws IOException {
+    testSimpleEval("select true;", new String[] {"t"});
+
+    testSimpleEval("select true and true;", new String[] {"t"});
+    testSimpleEval("select true and false;", new String[] {"f"});
+    testSimpleEval("select false and true;", new String[] {"f"});
+    testSimpleEval("select false and false;", new String[] {"f"});
+  }
+
+  @Test
+  public void testOr() throws IOException {
+    testSimpleEval("select true or true;", new String[] {"t"});
+    testSimpleEval("select true or false;", new String[] {"t"});
+    testSimpleEval("select false or true;", new String[] {"t"});
+    testSimpleEval("select false or false;", new String[] {"f"});
+  }
+
+  @Test
+  public void testLogicalOperatorPrecedence() throws IOException {
+    testSimpleEval("select true or (false or false) or false;", new String[] {"t"});
+    testSimpleEval("select false or (true or false) or false;", new String[] {"t"});
+    testSimpleEval("select false or (false or true) or false;", new String[] {"t"});
+    testSimpleEval("select false or (false or false) or true;", new String[] {"t"});
+
+    testSimpleEval("select true and (false or false) or false;", new String[] {"f"});
+    testSimpleEval("select false and (true or false) or false;", new String[] {"f"});
+    testSimpleEval("select false and (false or true) or false;", new String[] {"f"});
+    testSimpleEval("select false and (false or false) or true;", new String[] {"t"});
+
+    testSimpleEval("select true or (false and false) or false;", new String[] {"t"});
+    testSimpleEval("select false or (true and false) or false;", new String[] {"f"});
+    testSimpleEval("select false or (false and true) or false;", new String[] {"f"});
+    testSimpleEval("select false or (false and true) or true;", new String[] {"t"});
+
+    testSimpleEval("select true or (false or false) and false;", new String[] {"t"});
+    testSimpleEval("select false or (true or false) and false;", new String[] {"f"});
+    testSimpleEval("select false or (false or true) and false;", new String[] {"f"});
+    testSimpleEval("select false or (false or false) and true;", new String[] {"f"});
+  }
+
+  @Test
+  public void testNot() throws IOException {
+
+    testSimpleEval("select true;", new String[] {"t"});
+    testSimpleEval("select not true;", new String[] {"f"});
+    testSimpleEval("select (true);", new String[] {"t"});
+    testSimpleEval("select not (true);", new String[] {"f"});
+    testSimpleEval("select not (not (true));", new String[] {"t"});
+
+    testSimpleEval("select (not (1 > null)) is null;", new String[] {"t"});
+
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    schema1.addColumn("col3", INT4);
+
+    testEval(schema1,
+        "table1", "123,123,456,-123",
+        "select col1 = col2, col1 = col3 from table1",
+        new String[]{"t", "f"});
+  }
+
+  @Test
+  public void testParenthesizedValues() throws IOException {
+    testSimpleEval("select ((true));", new String[] {"t"});
+    testSimpleEval("select ((((true))));", new String[] {"t"});
+    testSimpleEval("select not(not(not(false)));", new String[] {"t"});
+  }
+
+  //////////////////////////////////////////////////////////////////
+  // Comparison Predicate
+  //////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testComparisonEqual() throws IOException {
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    schema1.addColumn("col3", INT4);
+    schema1.addColumn("col4", INT4);
+    testEval(schema1,
+        "table1", "123,123,456,-123",
+        "select col1 = col2, col1 = col3, col1 = col4 from table1",
+        new String[]{"t", "f", "f"});
+    testEval(schema1,
+        "table1", "123,123,,",
+        "select col1 = col2, (col1 = col3) is null, (col3 = col2) is null from table1",
+        new String[]{"t", "t", "t"});
+  }
+
+  @Test
+  public void testComparisonNotEqual() throws IOException {
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    schema1.addColumn("col3", INT4);
+    schema1.addColumn("col4", INT4);
+    testEval(schema1,
+        "table1", "123,123,456,-123",
+        "select col1 <> col2, col1 <> col3, col1 <> col4 from table1",
+        new String[]{"f", "t", "t"});
+    testEval(schema1,
+        "table1", "123,123,,",
+        "select col1 <> col2, (col1 <> col3) is null, (col3 <> col2) is null from table1",
+        new String[]{"f", "t", "t"});
+  }
+
+  @Test
+  public void testComparisonLessThan() throws IOException {
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    schema1.addColumn("col3", INT4);
+    schema1.addColumn("col4", INT4);
+    testEval(schema1,
+        "table1", "123,123,456,-123",
+        "select col1 < col2, col1 < col3, col1 < col4 from table1",
+        new String[]{"f", "t", "f"});
+    testEval(schema1,
+        "table1", "123,456,,",
+        "select col1 < col2, (col1 = col3) is null, (col4 = col1) is null from table1",
+        new String[]{"t", "t", "t"});
+  }
+
+  @Test
+  public void testComparisonLessThanEqual() throws IOException {
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    schema1.addColumn("col3", INT4);
+    schema1.addColumn("col4", INT4);
+    testEval(schema1,
+        "table1", "123,123,456,-123",
+        "select col1 <= col2, col1 <= col3, col1 <= col4 from table1",
+        new String[]{"t", "t", "f"});
+    testEval(schema1,
+        "table1", "123,456,,",
+        "select col1 <= col2, (col1 <= col3) is null, (col4 <= col1) is null from table1",
+        new String[]{"t", "t", "t"});
+  }
+
+  @Test
+  public void testComparisonGreaterThan() throws IOException {
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    schema1.addColumn("col3", INT4);
+    schema1.addColumn("col4", INT4);
+    testEval(schema1,
+        "table1", "123,123,456,-123",
+        "select col1 > col2, col3 > col2, col1 > col4 from table1",
+        new String[]{"f", "t", "t"});
+    testEval(schema1,
+        "table1", "123,456,,",
+        "select col2 > col1, col1 > col2, (col1 > col3) is null, (col4 > col1) is null from table1",
+        new String[]{"t", "f", "t", "t"});
+  }
+
+  @Test
+  public void testComparisonGreaterThanEqual() throws IOException {
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    schema1.addColumn("col3", INT4);
+    schema1.addColumn("col4", INT4);
+    testEval(schema1,
+        "table1", "123,123,456,-123",
+        "select col1 >= col2, col3 >= col2, col1 >= col4 from table1",
+        new String[]{"t", "t", "t"});
+    testEval(schema1,
+        "table1", "123,456,,",
+        "select col2 >= col1, col1 >= col2, (col1 >= col3) is null, (col4 >= col1) is null from table1",
+        new String[]{"t", "f", "t", "t"});
+  }
+
+  //////////////////////////////////////////////////////////////////
+  // Between Predicate
+  //////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testBetween() throws IOException {
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", TEXT);
+    schema2.addColumn("col2", TEXT);
+    schema2.addColumn("col3", TEXT);
+
+    // constant checker
+    testEval(schema2, "table1", "b,a,c", "select col1 between 'a' and 'c' from table1", new String[]{"t"});
+    testEval(schema2, "table1", "b,a,c", "select col1 between 'c' and 'a' from table1", new String[]{"f"});
+    testEval(schema2, "table1", "b,a,c", "select col1 between symmetric 'c' and 'a' from table1", new String[]{"t"});
+    testEval(schema2, "table1", "d,a,c", "select col1 between 'a' and 'c' from table1", new String[]{"f"});
+
+    // tests for inclusive
+    testEval(schema2, "table1", "a,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"});
+    testEval(schema2, "table1", "b,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"});
+    testEval(schema2, "table1", "c,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"});
+    testEval(schema2, "table1", "d,a,c", "select col1 between col2 and col3 from table1", new String[]{"f"});
+
+    // tests for asymmetric and symmetric
+    testEval(schema2, "table1", "b,a,c", "select col1 between col3 and col2 from table1", new String[]{"f"});
+    testEval(schema2, "table1", "b,a,c", "select col1 between symmetric col3 and col2 from table1", new String[]{"t"});
+  }
+
+  @Test
+  public void testBetween2() throws IOException { // for TAJO-249
+    Schema schema3 = new Schema();
+    schema3.addColumn("date_a", INT4);
+    schema3.addColumn("date_b", INT4);
+    schema3.addColumn("date_c", INT4);
+    schema3.addColumn("date_d", INT4);
+
+    String query = "select " +
+        "case " +
+        "when date_a BETWEEN 20130705 AND 20130715 AND ((date_b BETWEEN 20100101 AND 20120601) OR date_b > 20130715) " +
+        "AND (date_c < 20120601 OR date_c > 20130715) AND date_d > 20130715" +
+        "then 1 else 0 end from table1";
+
+    testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"});
+    testEval(schema3, "table1", "20130716,20100102,20120525,20130716", query, new String [] {"0"});
+
+    // date_b
+    testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"});
+    testEval(schema3, "table1", "20130715,20120602,20120525,20130716", query, new String [] {"0"});
+    testEval(schema3, "table1", "20130715,20091201,20120525,20130716", query, new String [] {"0"});
+    testEval(schema3, "table1", "20130715,20130716,20120525,20130716", query, new String [] {"1"});
+
+    // date_c
+    testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"});
+    testEval(schema3, "table1", "20130715,20100102,20120602,20130716", query, new String [] {"0"});
+
+    testEval(schema3, "table1", "20130715,20100102,20130716,20130716", query, new String [] {"1"});
+    testEval(schema3, "table1", "20130715,20100102,20130714,20130716", query, new String [] {"0"});
+
+    // date_d
+    testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"});
+    testEval(schema3, "table1", "20130715,20100102,20120525,20130705", query, new String [] {"0"});
+  }
+
+  //////////////////////////////////////////////////////////////////
+  // In Predicate
+  //////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testInPredicateWithConstant() throws IOException {
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", TEXT);
+    schema2.addColumn("col2", TEXT);
+    schema2.addColumn("col3", TEXT);
+
+    testEval(schema2, "table1", "a,b,c", "select col1 in ('a'), col2 in ('a', 'c') from table1", new String[]{"t","f"});
+    testEval(schema2, "table1", "a,,c", "select col1 in ('a','b','c'), (col2 in ('a', 'c')) is null from table1",
+        new String[]{"t","t"});
+
+    testEval(schema2,
+        "table1",
+        "2014-03-21,2015-04-01,2016-04-01",
+        "select substr(col1,1,4) in ('2014','2015','2016'), substr(col1,6,2)::int4 in (1,2,3) from table1",
+        new String[]{"t", "t"});
+
+    // null handling test
+    testEval(schema2,
+        "table1",
+        "2014-03-21,,2015-04-01",
+        "select (substr(col2,1,4)::int4 in (2014,2015,2016)) is null from table1",
+        new String[]{"t"});
+  }
+
+  @Test
+  public void testInPredicateWithSimpleExprs() throws IOException {
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", TEXT);
+    schema2.addColumn("col2", INT4);
+    schema2.addColumn("col3", TEXT);
+
+    testEval(schema2, "table1", "abc,2,3", "select col1 in ('a'||'b'||'c'), col2 in (1 + 1, 2 * 10, 2003) from table1",
+        new String[]{"t","t"});
+
+    testEval(schema2, "table1", "abc,2,3", "select col1 in ('a'||'b'), col2 in ('1'::int, '2'::int, 3) from table1",
+        new String[]{"f","t"});
+
+    testEval(schema2,
+        "table1",
+        "abc,,3",
+        "select col1 in (reverse('cba')), (col2 in ('1'::int, '2'::int, 3)) is null from table1",
+        new String[]{"t","t"});
+  }
+
+  //////////////////////////////////////////////////////////////////
+  // Null Predicate
+  //////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testIsNullPredicate() throws IOException {
+    Schema schema1 = new Schema();
+    schema1.addColumn("col1", INT4);
+    schema1.addColumn("col2", INT4);
+    testEval(schema1, "table1", "123,", "select col1 is null, col2 is null as a from table1",
+        new String[]{"f", "t"});
+    testEval(schema1, "table1", "123,", "select col1 is not null, col2 is not null as a from table1",
+        new String[]{"t", "f"});
+  }
+
+  @Test
+  public void testIsNullPredicateWithFunction() throws IOException {
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", TEXT);
+    schema2.addColumn("col2", TEXT);
+    testEval(schema2, "table1", "_123,", "select ltrim(col1, '_') is null, upper(col2) is null as a from table1",
+        new String[]{"f", "t"});
+
+    testEval(schema2, "table1", "_123,",
+        "select ltrim(col1, '_') is not null, upper(col2) is not null as a from table1", new String[]{"t", "f"});
+  }
+
+  //////////////////////////////////////////////////////////////////
+  // Boolean Test
+  //////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testBooleanTest() throws IOException {
+    testSimpleEval("select 1 < 3 is true", new String [] {"t"});
+    testSimpleEval("select 1 < 3 is not true", new String [] {"f"});
+    testSimpleEval("select 1 < 3 is false", new String [] {"f"});
+    testSimpleEval("select 1 < 3 is not false", new String [] {"t"});
+
+    testSimpleEval("select not (1 < 3 is true)", new String [] {"f"});
+    testSimpleEval("select not (1 < 3 is not true)", new String [] {"t"});
+    testSimpleEval("select not (1 < 3 is false)", new String [] {"t"});
+    testSimpleEval("select not (1 < 3 is not false)", new String [] {"f"});
+
+    testSimpleEval("select 1 > 3 is true", new String [] {"f"});
+    testSimpleEval("select 1 > 3 is not true", new String [] {"t"});
+    testSimpleEval("select 1 > 3 is false", new String [] {"t"});
+    testSimpleEval("select 1 > 3 is not false", new String [] {"f"});
+
+    testSimpleEval("select not (1 > 3 is true)", new String [] {"t"});
+    testSimpleEval("select not (1 > 3 is not true)", new String [] {"f"});
+    testSimpleEval("select not (1 > 3 is false)", new String [] {"f"});
+    testSimpleEval("select not (1 > 3 is not false)", new String [] {"t"});
+  }
+
+  @Test
+  public void testBooleanTestOnTable() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", BOOLEAN);
+    schema.addColumn("col2", BOOLEAN);
+    testEval(schema, "table1", "t,f", "select col1 is true, col2 is false from table1", new String [] {"t", "t"});
+    testEval(schema, "table1", "t,f", "select col1 is not true, col2 is not false from table1",
+        new String [] {"f", "f"});
+    testEval(schema, "table1", "t,f", "select not col1 is not true, not col2 is not false from table1",
+        new String [] {"t", "t"});
+  }
+
+  @Test
+  public void testCreateTableWithUnsupportedStoreType() throws IOException {
+    testSimpleEval("create table table1 (name text, age int) using RAW;",
+        new String[] {"Wrong query statement or query plan: create table table1 (name text, age int) using RAW"},
+        false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
new file mode 100644
index 0000000..c9c8dd4
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
@@ -0,0 +1,53 @@
+/**
+ * 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.junit.Test;
+
+import java.io.IOException;
+
+public class TestSQLDateTimeTypes extends ExprTestBase {
+
+  @Test
+  public void testTimestamp() throws IOException {
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37';", new String[]{"1970-01-17 10:09:37"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.5';", new String[]{"1970-01-17 10:09:37.5"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.01';", new String[]{"1970-01-17 10:09:37.01"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.003';", new String[]{"1970-01-17 10:09:37.003"});
+  }
+
+  @Test
+  public void testToTimestamp() throws IOException {
+    testSimpleEval("select to_char(TIMESTAMP '1970-01-17 10:09:37', 'yyyy-MM-dd HH:mm:ss');",
+        new String[]{"1970-01-17 10:09:37"});
+  }
+
+  @Test
+  public void testTimeLiteral() throws IOException {
+    testSimpleEval("select TIME '10:09:37';",
+        new String[]{"10:09:37"});
+  }
+
+  @Test
+  public void testDateLiteral() throws IOException {
+    testSimpleEval("select DATE '1970-01-17';",
+        new String[]{"1970-01-17"});
+  }
+
+}


[31/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java
new file mode 100644
index 0000000..ff1f7b3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java
@@ -0,0 +1,335 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+
+public class MergeFullOuterJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  private JoinNode joinNode;
+  private EvalNode joinQual;
+
+  // temporal tuples and states for nested loop join
+  private FrameTuple frameTuple;
+  private Tuple leftTuple = null;
+  private Tuple rightTuple = null;
+  private Tuple outTuple = null;
+  private Tuple leftNext = null;
+
+  private List<Tuple> leftTupleSlots;
+  private List<Tuple> rightTupleSlots;
+
+  private JoinTupleComparator joincomparator = null;
+  private TupleComparator[] tupleComparator = null;
+
+  private final static int INITIAL_TUPLE_SLOT = 10000;
+
+  private boolean end = false;
+
+  // projection
+  private Projector projector;
+
+  private int rightNumCols;
+  private int leftNumCols;
+  private int posRightTupleSlots = -1;
+  private int posLeftTupleSlots = -1;
+  boolean endInPopulationStage = false;
+  private boolean initRightDone = false;
+
+  public MergeFullOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftChild,
+                                PhysicalExec rightChild, SortSpec[] leftSortKey, SortSpec[] rightSortKey) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), leftChild, rightChild);
+    Preconditions.checkArgument(plan.hasJoinQual(), "Sort-merge join is only used for the equi-join, " +
+        "but there is no join condition");
+    this.joinNode = plan;
+    this.joinQual = plan.getJoinQual();
+
+    this.leftTupleSlots = new ArrayList<Tuple>(INITIAL_TUPLE_SLOT);
+    this.rightTupleSlots = new ArrayList<Tuple>(INITIAL_TUPLE_SLOT);
+    SortSpec[][] sortSpecs = new SortSpec[2][];
+    sortSpecs[0] = leftSortKey;
+    sortSpecs[1] = rightSortKey;
+
+    this.joincomparator = new JoinTupleComparator(leftChild.getSchema(),
+        rightChild.getSchema(), sortSpecs);
+    this.tupleComparator = PlannerUtil.getComparatorsFromJoinQual(
+        plan.getJoinQual(), leftChild.getSchema(), rightChild.getSchema());
+
+    // for projection
+    this.projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+
+    leftNumCols = leftChild.getSchema().size();
+    rightNumCols = rightChild.getSchema().size();
+  }
+
+  public JoinNode getPlan(){
+    return this.joinNode;
+  }
+
+  public Tuple next() throws IOException {
+    Tuple previous;
+
+    for (;;) {
+      boolean newRound = false;
+      if((posRightTupleSlots == -1) && (posLeftTupleSlots == -1)) {
+        newRound = true;
+      }
+      if ((posRightTupleSlots == rightTupleSlots.size()) && (posLeftTupleSlots == leftTupleSlots.size())) {
+        newRound = true;
+      }
+
+      if(newRound == true){
+
+        if (end) {
+
+          ////////////////////////////////////////////////////////////////////////
+          // FINALIZING STAGE
+          ////////////////////////////////////////////////////////////////////////
+          // the finalizing stage, where remaining tuples on the right are
+          // transformed into left-padded results while tuples on the left
+          // are transformed into right-padded results
+
+          // before exit, a left-padded tuple should be built for all remaining
+          // right side and a right-padded tuple should be built for all remaining
+          // left side
+
+          if (initRightDone == false) {
+            // maybe the left operand was empty => the right one didn't have the chance to initialize
+            rightTuple = rightChild.next();
+            initRightDone = true;
+          }
+
+          if((leftTuple == null) && (rightTuple == null)) {
+            return null;
+          }
+
+          if((leftTuple == null) && (rightTuple != null)){
+            // output a tuple with the nulls padded leftTuple
+            Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(leftNumCols);
+            frameTuple.set(nullPaddedTuple, rightTuple);
+            projector.eval(frameTuple, outTuple);
+            // we simulate we found a match, which is exactly the null padded one
+            rightTuple = rightChild.next();
+            return outTuple;
+          }
+
+          if((leftTuple != null) && (rightTuple == null)){
+            // output a tuple with the nulls padded leftTuple
+            Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols);
+            frameTuple.set(leftTuple, nullPaddedTuple);
+            projector.eval(frameTuple, outTuple);
+            // we simulate we found a match, which is exactly the null padded one
+            leftTuple = leftChild.next();
+            return outTuple;
+          }
+        } // if end
+
+        ////////////////////////////////////////////////////////////////////////
+        // INITIALIZING STAGE
+        ////////////////////////////////////////////////////////////////////////
+        // initializing stage, reading the first tuple on each side
+        if (leftTuple == null) {
+          leftTuple = leftChild.next();
+          if( leftTuple == null){
+            end = true;
+            continue;
+          }
+        }
+        if (rightTuple == null) {
+          rightTuple = rightChild.next();
+          initRightDone = true;
+          if (rightTuple == null) {
+            end = true;
+            continue;
+          }
+        }
+
+        // reset tuple slots for a new round
+        leftTupleSlots.clear();
+        rightTupleSlots.clear();
+        posRightTupleSlots = -1;
+        posLeftTupleSlots = -1;
+
+        ////////////////////////////////////////////////////////////////////////
+        // Comparison and Move Forward Stage
+        ////////////////////////////////////////////////////////////////////////
+        // advance alternatively on each side until a match is found
+        int cmp;
+        while (!end && ((cmp = joincomparator.compare(leftTuple, rightTuple)) != 0)) {
+
+          if (cmp > 0) {
+
+            //before getting a new tuple from the right,  a leftnullpadded tuple should be built
+            //output a tuple with the nulls padded leftTuple
+            Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(leftNumCols);
+            frameTuple.set(nullPaddedTuple, rightTuple);
+            projector.eval(frameTuple, outTuple);
+            // BEFORE RETURN, MOVE FORWARD
+            rightTuple = rightChild.next();
+            if(rightTuple == null) {
+              end = true;
+            }
+
+            return outTuple;
+
+          } else if (cmp < 0) {
+            // before getting a new tuple from the left,  a rightnullpadded tuple should be built
+            // output a tuple with the nulls padded rightTuple
+            Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols);
+            frameTuple.set(leftTuple, nullPaddedTuple);
+            projector.eval(frameTuple, outTuple);
+            // we simulate we found a match, which is exactly the null padded one
+            // BEFORE RETURN, MOVE FORWARD
+            leftTuple = leftChild.next();
+            if(leftTuple == null) {
+              end = true;
+            }
+
+            return outTuple;
+
+          } // if (cmp < 0)
+        } //while
+
+
+        ////////////////////////////////////////////////////////////////////////
+        // SLOTS POPULATION STAGE
+        ////////////////////////////////////////////////////////////////////////
+        // once a match is found, retain all tuples with this key in tuple slots
+        // on each side
+        if(!end) {
+          endInPopulationStage = false;
+
+          boolean endLeft = false;
+          boolean endRight = false;
+
+          previous = new VTuple(leftTuple);
+          do {
+            leftTupleSlots.add(new VTuple(leftTuple));
+            leftTuple = leftChild.next();
+            if(leftTuple == null) {
+              endLeft = true;
+            }
+
+
+          } while ((endLeft != true) && (tupleComparator[0].compare(previous, leftTuple) == 0));
+          posLeftTupleSlots = 0;
+
+
+          previous = new VTuple(rightTuple);
+          do {
+            rightTupleSlots.add(new VTuple(rightTuple));
+            rightTuple = rightChild.next();
+            if(rightTuple == null) {
+              endRight = true;
+            }
+
+          } while ((endRight != true) && (tupleComparator[1].compare(previous, rightTuple) == 0) );
+          posRightTupleSlots = 0;
+
+          if ((endLeft == true) || (endRight == true)) {
+            end = true;
+            endInPopulationStage = true;
+          }
+
+        } // if end false
+      } // if newRound
+
+
+      ////////////////////////////////////////////////////////////////////////
+      // RESULTS STAGE
+      ////////////////////////////////////////////////////////////////////////
+      // now output result matching tuples from the slots
+      // if either we haven't reached end on neither side, or we did reach end
+      // on one(or both) sides but that happened in the slots population step
+      // (i.e. refers to next round)
+      if(!end || (end && endInPopulationStage)){
+        if(posLeftTupleSlots == 0){
+          leftNext = new VTuple (leftTupleSlots.get(posLeftTupleSlots));
+          posLeftTupleSlots = posLeftTupleSlots + 1;
+        }
+
+        if(posRightTupleSlots <= (rightTupleSlots.size() -1)) {
+          Tuple aTuple = new VTuple(rightTupleSlots.get(posRightTupleSlots));
+          posRightTupleSlots = posRightTupleSlots + 1;
+          frameTuple.set(leftNext, aTuple);
+          joinQual.eval(inSchema, frameTuple);
+          projector.eval(frameTuple, outTuple);
+          return outTuple;
+        } else {
+          // right (inner) slots reached end and should be rewind if there are still tuples in the outer slots
+          if(posLeftTupleSlots <= (leftTupleSlots.size()-1)) {
+            //rewind the right slots position
+            posRightTupleSlots = 0;
+            Tuple aTuple = new VTuple(rightTupleSlots.get(posRightTupleSlots));
+            posRightTupleSlots = posRightTupleSlots + 1;
+            leftNext = new VTuple (leftTupleSlots.get(posLeftTupleSlots));
+            posLeftTupleSlots = posLeftTupleSlots + 1;
+
+            frameTuple.set(leftNext, aTuple);
+            joinQual.eval(inSchema, frameTuple);
+            projector.eval(frameTuple, outTuple);
+            return outTuple;
+          }
+        }
+      } // the second if end false
+    } // for
+  }
+
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+    leftTupleSlots.clear();
+    rightTupleSlots.clear();
+    posRightTupleSlots = -1;
+    posLeftTupleSlots = -1;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    leftTupleSlots.clear();
+    rightTupleSlots.clear();
+    leftTupleSlots = null;
+    rightTupleSlots = null;
+    joinNode = null;
+    joinQual = null;
+    projector = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java
new file mode 100644
index 0000000..470e1c9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java
@@ -0,0 +1,193 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class MergeJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  private JoinNode joinNode;
+  private EvalNode joinQual;
+
+  // temporal tuples and states for nested loop join
+  private FrameTuple frameTuple;
+  private Tuple outerTuple = null;
+  private Tuple innerTuple = null;
+  private Tuple outTuple = null;
+  private Tuple outerNext = null;
+
+  private List<Tuple> outerTupleSlots;
+  private List<Tuple> innerTupleSlots;
+  private Iterator<Tuple> outerIterator;
+  private Iterator<Tuple> innerIterator;
+
+  private JoinTupleComparator joincomparator = null;
+  private TupleComparator[] tupleComparator = null;
+
+  private final static int INITIAL_TUPLE_SLOT = 10000;
+
+  private boolean end = false;
+
+  // projection
+  private Projector projector;
+
+  public MergeJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer,
+      PhysicalExec inner, SortSpec[] outerSortKey, SortSpec[] innerSortKey) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), outer, inner);
+    Preconditions.checkArgument(plan.hasJoinQual(), "Sort-merge join is only used for the equi-join, " +
+        "but there is no join condition");
+    this.joinNode = plan;
+    this.joinQual = plan.getJoinQual();
+
+    this.outerTupleSlots = new ArrayList<Tuple>(INITIAL_TUPLE_SLOT);
+    this.innerTupleSlots = new ArrayList<Tuple>(INITIAL_TUPLE_SLOT);
+    SortSpec[][] sortSpecs = new SortSpec[2][];
+    sortSpecs[0] = outerSortKey;
+    sortSpecs[1] = innerSortKey;
+
+    this.joincomparator = new JoinTupleComparator(outer.getSchema(),
+        inner.getSchema(), sortSpecs);
+    this.tupleComparator = PlannerUtil.getComparatorsFromJoinQual(
+        plan.getJoinQual(), outer.getSchema(), inner.getSchema());
+    this.outerIterator = outerTupleSlots.iterator();
+    this.innerIterator = innerTupleSlots.iterator();
+    
+    // for projection
+    this.projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+  }
+
+  public JoinNode getPlan(){
+    return this.joinNode;
+  }
+
+  public Tuple next() throws IOException {
+    Tuple previous;
+
+    for (;;) {
+      if (!outerIterator.hasNext() && !innerIterator.hasNext()) {
+        if(end){
+          return null;
+        }
+
+        if(outerTuple == null){
+          outerTuple = leftChild.next();
+        }
+        if(innerTuple == null){
+          innerTuple = rightChild.next();
+        }
+
+        outerTupleSlots.clear();
+        innerTupleSlots.clear();
+
+        int cmp;
+        while ((cmp = joincomparator.compare(outerTuple, innerTuple)) != 0) {
+          if (cmp > 0) {
+            innerTuple = rightChild.next();
+          } else if (cmp < 0) {
+            outerTuple = leftChild.next();
+          }
+          if (innerTuple == null || outerTuple == null) {
+            return null;
+          }
+        }
+
+        try {
+          previous = outerTuple.clone();
+          do {
+            outerTupleSlots.add(outerTuple.clone());
+            outerTuple = leftChild.next();
+            if (outerTuple == null) {
+              end = true;
+              break;
+            }
+          } while (tupleComparator[0].compare(previous, outerTuple) == 0);
+          outerIterator = outerTupleSlots.iterator();
+          outerNext = outerIterator.next();
+
+          previous = innerTuple.clone();
+          do {
+            innerTupleSlots.add(innerTuple.clone());
+            innerTuple = rightChild.next();
+            if (innerTuple == null) {
+              end = true;
+              break;
+            }
+          } while (tupleComparator[1].compare(previous, innerTuple) == 0);
+          innerIterator = innerTupleSlots.iterator();
+        } catch (CloneNotSupportedException e) {
+
+        }
+      }
+
+      if(!innerIterator.hasNext()){
+        outerNext = outerIterator.next();
+        innerIterator = innerTupleSlots.iterator();
+      }
+
+      frameTuple.set(outerNext, innerIterator.next());
+
+      if (joinQual.eval(inSchema, frameTuple).isTrue()) {
+        projector.eval(frameTuple, outTuple);
+        return outTuple;
+      }
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+    outerTupleSlots.clear();
+    innerTupleSlots.clear();
+    outerIterator = outerTupleSlots.iterator();
+    innerIterator = innerTupleSlots.iterator();
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+
+    outerTupleSlots.clear();
+    innerTupleSlots.clear();
+    outerTupleSlots = null;
+    innerTupleSlots = null;
+    outerIterator = null;
+    innerIterator = null;
+    joinQual = null;
+    projector = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java
new file mode 100644
index 0000000..6e5900e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java
@@ -0,0 +1,104 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class NLJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  private JoinNode plan;
+  private EvalNode joinQual;
+
+
+  // temporal tuples and states for nested loop join
+  private boolean needNewOuter;
+  private FrameTuple frameTuple;
+  private Tuple outerTuple = null;
+  private Tuple innerTuple = null;
+  private Tuple outTuple = null;
+
+  // projection
+  private final Projector projector;
+
+  public NLJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer,
+      PhysicalExec inner) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), outer, inner);
+    this.plan = plan;
+
+    if (plan.hasJoinQual()) {
+      this.joinQual = plan.getJoinQual();
+    }
+
+    // for projection
+    projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    needNewOuter = true;
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+  }
+
+  public JoinNode getPlan() {
+    return this.plan;
+  }
+
+  public Tuple next() throws IOException {
+    for (;;) {
+      if (needNewOuter) {
+        outerTuple = leftChild.next();
+        if (outerTuple == null) {
+          return null;
+        }
+        needNewOuter = false;
+      }
+
+      innerTuple = rightChild.next();
+      if (innerTuple == null) {
+        needNewOuter = true;
+        rightChild.rescan();
+        continue;
+      }
+
+      frameTuple.set(outerTuple, innerTuple);
+      if (joinQual != null) {
+        if (joinQual.eval(inSchema, frameTuple).isTrue()) {
+          projector.eval(frameTuple, outTuple);
+          return outTuple;
+        }
+      } else {
+        projector.eval(frameTuple, outTuple);
+        return outTuple;
+      }
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+    needNewOuter = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java
new file mode 100644
index 0000000..5c17c40
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java
@@ -0,0 +1,122 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class NLLeftOuterJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  private JoinNode plan;
+  private EvalNode joinQual;
+
+  // temporal tuples and states for nested loop join
+  private boolean needNextRightTuple;
+  private FrameTuple frameTuple;
+  private Tuple leftTuple = null;
+  private Tuple rightTuple = null;
+  private Tuple outTuple = null;
+
+  // projection
+  private final Projector projector;
+
+  private boolean foundAtLeastOneMatch;
+  private int rightNumCols;
+
+  public NLLeftOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftChild,
+                             PhysicalExec rightChild) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), leftChild, rightChild);
+    this.plan = plan;
+
+    if (plan.hasJoinQual()) {
+      this.joinQual = plan.getJoinQual();
+    }
+
+    // for projection
+    projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    needNextRightTuple = true;
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+
+    foundAtLeastOneMatch = false;
+    rightNumCols = rightChild.getSchema().size();
+  }
+
+  public JoinNode getPlan() {
+    return this.plan;
+  }
+
+  public Tuple next() throws IOException {
+    for (;;) {
+      if (needNextRightTuple) {
+        leftTuple = leftChild.next();
+        if (leftTuple == null) {
+          return null;
+        }
+        needNextRightTuple = false;
+        // a new tuple from the left child has initially no matches on the right operand
+        foundAtLeastOneMatch = false;
+      }
+      rightTuple = rightChild.next();
+
+      if (rightTuple == null) {
+        // the scan of the right operand is finished with no matches found
+        if(foundAtLeastOneMatch == false){
+          //output a tuple with the nulls padded rightTuple
+          Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols);
+          frameTuple.set(leftTuple, nullPaddedTuple);
+          projector.eval(frameTuple, outTuple);
+          // we simulate we found a match, which is exactly the null padded one
+          foundAtLeastOneMatch = true;
+          needNextRightTuple = true;
+          rightChild.rescan();
+          return outTuple;
+        } else {
+          needNextRightTuple = true;
+          rightChild.rescan();
+          continue;
+        }
+      }
+
+      frameTuple.set(leftTuple, rightTuple);
+      ;
+      if (joinQual.eval(inSchema, frameTuple).isTrue()) {
+        projector.eval(frameTuple, outTuple);
+        foundAtLeastOneMatch = true;
+        return outTuple;
+      }
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+    needNextRightTuple = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
new file mode 100644
index 0000000..7f86ba2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
@@ -0,0 +1,143 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.collect.Lists;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+
+/**
+ * A Scanner that reads multiple partitions
+ */
+public class PartitionMergeScanExec extends PhysicalExec {
+  private final ScanNode plan;
+  private SeqScanExec currentScanner = null;
+
+  private CatalogProtos.FragmentProto [] fragments;
+
+  private List<SeqScanExec> scanners = Lists.newArrayList();
+  private Iterator<SeqScanExec> iterator;
+
+  private AbstractStorageManager sm;
+
+  private float progress;
+  protected TableStats inputStats;
+
+  public PartitionMergeScanExec(TaskAttemptContext context, AbstractStorageManager sm,
+                                ScanNode plan, CatalogProtos.FragmentProto[] fragments) throws IOException {
+    super(context, plan.getInSchema(), plan.getOutSchema());
+
+    this.plan = plan;
+    this.fragments = fragments;
+    this.sm = sm;
+
+    inputStats = new TableStats();
+  }
+
+  public void init() throws IOException {
+    for (CatalogProtos.FragmentProto fragment : fragments) {
+      scanners.add(new SeqScanExec(context, sm, (ScanNode) PlannerUtil.clone(null, plan),
+          new CatalogProtos.FragmentProto[] {fragment}));
+    }
+    progress = 0.0f;
+    rescan();
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple;
+    while (currentScanner != null) {
+      tuple = currentScanner.next();
+
+      if (tuple != null) {
+        return tuple;
+      }
+
+      if (iterator.hasNext()) {
+        if (currentScanner != null) {
+          currentScanner.close();
+        }
+        currentScanner = iterator.next();
+        currentScanner.init();
+      } else {
+        break;
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    if (scanners.size() > 0) {
+      iterator = scanners.iterator();
+      currentScanner = iterator.next();
+      currentScanner.init();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    for (SeqScanExec scanner : scanners) {
+      scanner.close();
+      TableStats scannerTableStsts = scanner.getInputStats();
+      if (scannerTableStsts != null) {
+        inputStats.merge(scannerTableStsts);
+      }
+    }
+    iterator = null;
+    progress = 1.0f;
+  }
+
+  public String getTableName() {
+    return plan.getTableName();
+  }
+
+  @Override
+  public float getProgress() {
+    if (iterator != null) {
+      float progressSum = 0.0f;
+      for (SeqScanExec scanner : scanners) {
+        progressSum += scanner.getProgress();
+      }
+      if (progressSum > 0) {
+        // get a average progress - divide progress summary by the number of scanners
+        return progressSum / (float)(scanners.size());
+      } else {
+        return 0.0f;
+      }
+    } else {
+      return progress;
+    }
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    return inputStats;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java
new file mode 100644
index 0000000..6d544a7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * 
+ */
+package org.apache.tajo.engine.planner.physical;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.storage.Tuple;
+
+public abstract class Partitioner {
+  protected final int [] partitionKeyIds;
+  protected final int numPartitions;
+  
+  public Partitioner(final int [] keyList, final int numPartitions) {
+    Preconditions.checkArgument(keyList != null, 
+        "Partition keys must be given");
+    Preconditions.checkArgument(keyList.length >= 0,
+        "At least one partition key must be specified.");
+    // In outer join, zero can be passed into this value because of empty tables.
+    // So, we should allow zero.
+    Preconditions.checkArgument(numPartitions >= 0,
+        "The number of partitions must be positive: %s", numPartitions);
+    this.partitionKeyIds = keyList;
+    this.numPartitions = numPartitions;    
+  }
+  
+  public abstract int getPartition(Tuple tuple);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java
new file mode 100644
index 0000000..e30a10b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java
@@ -0,0 +1,79 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SchemaObject;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.UUID;
+
+public abstract class PhysicalExec implements SchemaObject {
+  protected final TaskAttemptContext context;
+  protected Schema inSchema;
+  protected Schema outSchema;
+  protected int outColumnNum;
+
+  public PhysicalExec(final TaskAttemptContext context, final Schema inSchema,
+                      final Schema outSchema) {
+    this.context = context;
+    this.inSchema = inSchema;
+    this.outSchema = outSchema;
+    this.outColumnNum = outSchema.size();
+  }
+
+  public final Schema getSchema() {
+    return outSchema;
+  }
+
+  public abstract void init() throws IOException;
+
+  public abstract Tuple next() throws IOException;
+
+  public abstract void rescan() throws IOException;
+
+  public abstract void close() throws IOException;
+
+  public abstract float getProgress();
+
+  protected void info(Log log, String message) {
+    log.info("["+ context.getTaskId() + "] " + message);
+  }
+
+  protected void warn(Log log, String message) {
+    log.warn("[" + context.getTaskId() + "] " + message);
+  }
+
+  protected void fatal(Log log, String message) {
+    log.fatal("[" + context.getTaskId() + "] " + message);
+  }
+
+  protected Path getExecutorTmpDir() {
+    return new Path(UUID.randomUUID().toString());
+  }
+
+  public TableStats getInputStats() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java
new file mode 100644
index 0000000..738db62
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java
@@ -0,0 +1,107 @@
+/**
+ * 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.planner.physical;
+
+import java.util.Stack;
+
+public interface PhysicalExecutorVisitor<CONTEXT, RESULT> {
+
+  RESULT visitBNLJoin(CONTEXT context, BNLJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitBSTIndexScan(CONTEXT context, BSTIndexScanExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitEvalExpr(CONTEXT context, EvalExprExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitExternalSort(CONTEXT context, ExternalSortExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHashAggregate(CONTEXT context, HashAggregateExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHashBasedColPartitionStore(CONTEXT context, HashBasedColPartitionStoreExec exec,
+                                         Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHashFullOuterJoin(CONTEXT context, HashFullOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHashJoin(CONTEXT context, HashJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHashLeftAntiJoin(CONTEXT context, HashLeftAntiJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHashLeftOuterJoin(CONTEXT context, HashLeftOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitLeftHashSemiJoin(CONTEXT context, HashLeftSemiJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHashShuffleFileWrite(CONTEXT context, HashShuffleFileWriteExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitHaving(CONTEXT context, HavingExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitLimit(CONTEXT context, LimitExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitMemSort(CONTEXT context, MemSortExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitMergeFullOuterJoin(CONTEXT context, MergeFullOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitMergeJoin(CONTEXT context, MergeJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitNLJoin(CONTEXT context, NLJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitNLLeftOuterJoin(CONTEXT context, NLLeftOuterJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitProjection(CONTEXT context, ProjectionExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitRangeShuffleFileWrite(CONTEXT context, RangeShuffleFileWriteExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitRightOuterMergeJoin(CONTEXT context, RightOuterMergeJoinExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitSelection(CONTEXT context, SelectionExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitSeqScan(CONTEXT context, SeqScanExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitSortAggregate(CONTEXT context, SortAggregateExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitSortBasedColPartitionStore(CONTEXT context, SortBasedColPartitionStoreExec exec,
+                                         Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+
+  RESULT visitStoreTable(CONTEXT context, StoreTableExec exec, Stack<PhysicalExec> stack)
+      throws PhysicalPlanningException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
new file mode 100644
index 0000000..fdd1839
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
@@ -0,0 +1,39 @@
+/**
+ * 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.planner.physical;
+
+import java.util.Stack;
+
+public class PhysicalPlanUtil {
+  public static <T extends PhysicalExec> T findExecutor(PhysicalExec plan, Class<? extends PhysicalExec> clazz)
+      throws PhysicalPlanningException {
+    return (T) new FindVisitor().visit(plan, new Stack<PhysicalExec>(), clazz);
+  }
+
+  private static class FindVisitor extends BasicPhysicalExecutorVisitor<Class<? extends PhysicalExec>, PhysicalExec> {
+    public PhysicalExec visit(PhysicalExec exec, Stack<PhysicalExec> stack, Class<? extends PhysicalExec> target)
+        throws PhysicalPlanningException {
+      if (target.isAssignableFrom(exec.getClass())) {
+        return exec;
+      } else {
+        return super.visit(exec, stack, target);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java
new file mode 100644
index 0000000..62add1e
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java
@@ -0,0 +1,31 @@
+/**
+ * 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.planner.physical;
+
+import java.io.IOException;
+
+public class PhysicalPlanningException extends IOException {
+  public PhysicalPlanningException(String message) {
+    super(message);
+  }
+
+  public PhysicalPlanningException(Exception ioe) {
+    super(ioe);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ProjectionExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ProjectionExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ProjectionExec.java
new file mode 100644
index 0000000..ee6ef1d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ProjectionExec.java
@@ -0,0 +1,69 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.Projectable;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class ProjectionExec extends UnaryPhysicalExec {
+  private Projectable plan;
+
+  // for projection
+  private Tuple outTuple;
+  private Projector projector;
+  
+  public ProjectionExec(TaskAttemptContext context, Projectable plan,
+      PhysicalExec child) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+    this.plan = plan;
+  }
+
+  public void init() throws IOException {
+    super.init();
+
+    this.outTuple = new VTuple(outSchema.size());
+    this.projector = new Projector(inSchema, outSchema, this.plan.getTargets());
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple = child.next();
+
+    if (tuple ==  null) {
+      return null;
+    }
+
+    projector.eval(tuple, outTuple);
+    return outTuple;
+  }
+
+  @Override
+  public void close() throws IOException{
+    super.close();
+    plan = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
new file mode 100644
index 0000000..68379d1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
@@ -0,0 +1,131 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+/**
+ * <code>RangeShuffleFileWriteExec</code> is a physical executor to store intermediate data into a number of
+ * file outputs associated with shuffle key ranges. The file outputs are stored with index files on local disks.
+ * <code>RangeShuffleFileWriteExec</code> is implemented with an assumption that input tuples are sorted in an
+ * specified order of shuffle keys.
+ */
+public class RangeShuffleFileWriteExec extends UnaryPhysicalExec {
+  private static Log LOG = LogFactory.getLog(RangeShuffleFileWriteExec.class);
+  private final SortSpec[] sortSpecs;
+  private int [] indexKeys = null;
+  private Schema keySchema;
+
+  private BSTIndex.BSTIndexWriter indexWriter;
+  private TupleComparator comp;
+  private FileAppender appender;
+  private TableMeta meta;
+
+  public RangeShuffleFileWriteExec(final TaskAttemptContext context, final AbstractStorageManager sm,
+                                   final PhysicalExec child, final Schema inSchema, final Schema outSchema,
+                                   final SortSpec[] sortSpecs) throws IOException {
+    super(context, inSchema, outSchema, child);
+    this.sortSpecs = sortSpecs;
+  }
+
+  public void init() throws IOException {
+    super.init();
+
+    indexKeys = new int[sortSpecs.length];
+    keySchema = PlannerUtil.sortSpecsToSchema(sortSpecs);
+
+    Column col;
+    for (int i = 0 ; i < sortSpecs.length; i++) {
+      col = sortSpecs[i].getSortKey();
+      indexKeys[i] = inSchema.getColumnId(col.getQualifiedName());
+    }
+
+    BSTIndex bst = new BSTIndex(new TajoConf());
+    this.comp = new TupleComparator(keySchema, sortSpecs);
+    Path storeTablePath = new Path(context.getWorkDir(), "output");
+    LOG.info("Output data directory: " + storeTablePath);
+    this.meta = CatalogUtil.newTableMeta(context.getDataChannel() != null ?
+        context.getDataChannel().getStoreType() : CatalogProtos.StoreType.RAW);
+    FileSystem fs = new RawLocalFileSystem();
+    fs.mkdirs(storeTablePath);
+    this.appender = (FileAppender) StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta,
+        outSchema, new Path(storeTablePath, "output"));
+    this.appender.enableStats();
+    this.appender.init();
+    this.indexWriter = bst.getIndexWriter(new Path(storeTablePath, "index"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    this.indexWriter.setLoadNum(100);
+    this.indexWriter.open();
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple;
+    Tuple keyTuple;
+    Tuple prevKeyTuple = null;
+    long offset;
+
+
+    while((tuple = child.next()) != null) {
+      offset = appender.getOffset();
+      appender.addTuple(tuple);
+      keyTuple = new VTuple(keySchema.size());
+      RowStoreUtil.project(tuple, keyTuple, indexKeys);
+      if (prevKeyTuple == null || !prevKeyTuple.equals(keyTuple)) {
+        indexWriter.write(keyTuple, offset);
+        prevKeyTuple = keyTuple;
+      }
+    }
+
+    return null;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+  }
+
+  public void close() throws IOException {
+    super.close();
+
+    appender.flush();
+    IOUtils.cleanup(LOG, appender);
+    indexWriter.flush();
+    IOUtils.cleanup(LOG, indexWriter);
+
+    // Collect statistics data
+    context.setResultStats(appender.getStats());
+    context.addShuffleFileOutput(0, context.getTaskId().toString());
+    appender = null;
+    indexWriter = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java
new file mode 100644
index 0000000..c70174a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java
@@ -0,0 +1,346 @@
+/**
+ * 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.planner.physical;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.storage.FrameTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class RightOuterMergeJoinExec extends BinaryPhysicalExec {
+  // from logical plan
+  private JoinNode joinNode;
+  private EvalNode joinQual;
+
+  // temporal tuples and states for nested loop join
+  private FrameTuple frameTuple;
+  private Tuple leftTuple = null;
+  private Tuple rightTuple = null;
+  private Tuple outTuple = null;
+  private Tuple nextLeft = null;
+
+  private List<Tuple> leftTupleSlots;
+  private List<Tuple> innerTupleSlots;
+
+  private JoinTupleComparator joinComparator = null;
+  private TupleComparator[] tupleComparator = null;
+
+  private final static int INITIAL_TUPLE_SLOT = 10000;
+
+  private boolean end = false;
+
+  // projection
+  private Projector projector;
+
+  private int rightNumCols;
+  private int leftNumCols;
+  private int posRightTupleSlots = -1;
+  private int posLeftTupleSlots = -1;
+  private boolean endInPopulationStage = false;
+  private boolean initRightDone = false;
+
+  public RightOuterMergeJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer,
+                                 PhysicalExec inner, SortSpec[] outerSortKey, SortSpec[] innerSortKey) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), outer, inner);
+    Preconditions.checkArgument(plan.hasJoinQual(), "Sort-merge join is only used for the equi-join, " +
+        "but there is no join condition");
+    this.joinNode = plan;
+    this.joinQual = plan.getJoinQual();
+
+    this.leftTupleSlots = new ArrayList<Tuple>(INITIAL_TUPLE_SLOT);
+    this.innerTupleSlots = new ArrayList<Tuple>(INITIAL_TUPLE_SLOT);
+    SortSpec[][] sortSpecs = new SortSpec[2][];
+    sortSpecs[0] = outerSortKey;
+    sortSpecs[1] = innerSortKey;
+
+    this.joinComparator = new JoinTupleComparator(outer.getSchema(), inner.getSchema(), sortSpecs);
+    this.tupleComparator = PlannerUtil.getComparatorsFromJoinQual(
+        plan.getJoinQual(), outer.getSchema(), inner.getSchema());
+
+    // for projection
+    this.projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    // for join
+    frameTuple = new FrameTuple();
+    outTuple = new VTuple(outSchema.size());
+
+    leftNumCols = outer.getSchema().size();
+  }
+
+  public JoinNode getPlan() {
+    return this.joinNode;
+  }
+
+  /**
+   * creates a tuple of a given size filled with NULL values in all fields
+   */
+  private Tuple createNullPaddedTuple(int columnNum){
+    VTuple tuple = new VTuple(columnNum);
+    for (int i = 0; i < columnNum; i++) {
+      tuple.put(i, DatumFactory.createNullDatum());
+    }
+    return tuple;
+  }
+
+  /**
+   *
+   * Right outer merge join consists of four stages
+   * <ul>
+   *   <li>initialization stage: </li>
+   *   <li>finalizing stage: </li>
+   * </ul>
+   *
+   * @return
+   * @throws IOException
+   */
+  public Tuple next() throws IOException {
+    Tuple previous;
+
+    for (;;) {
+      boolean newRound = false;
+      if((posRightTupleSlots == -1) && (posLeftTupleSlots == -1)) {
+        newRound = true;
+      }
+      if ((posRightTupleSlots == innerTupleSlots.size()) && (posLeftTupleSlots == leftTupleSlots.size())) {
+        newRound = true;
+      }
+
+      if (newRound) {
+
+        //////////////////////////////////////////////////////////////////////
+        // BEGIN FINALIZING STAGE
+        //////////////////////////////////////////////////////////////////////
+
+        // The finalizing stage, where remaining tuples on the only right are transformed into left-padded results
+        if (end) {
+          if (initRightDone == false) {
+            // maybe the left operand was empty => the right one didn't have the chance to initialize
+            rightTuple = rightChild.next();
+            initRightDone = true;
+          }
+
+          if(rightTuple == null) {
+            return null;
+          } else {
+            // output a tuple with the nulls padded leftTuple
+            Tuple nullPaddedTuple = createNullPaddedTuple(leftNumCols);
+            frameTuple.set(nullPaddedTuple, rightTuple);
+            projector.eval(frameTuple, outTuple);
+
+            // we simulate we found a match, which is exactly the null padded one
+            rightTuple = rightChild.next();
+
+            return outTuple;
+          }
+        }
+        //////////////////////////////////////////////////////////////////////
+        // END FINALIZING STAGE
+        //////////////////////////////////////////////////////////////////////
+
+
+        //////////////////////////////////////////////////////////////////////
+        // BEGIN INITIALIZATION STAGE
+        //////////////////////////////////////////////////////////////////////
+
+        // This stage reads the first tuple on each side
+        if (leftTuple == null) {
+          leftTuple = leftChild.next();
+
+          if (leftTuple == null) {
+            end = true;
+            continue;
+          }
+        }
+
+        if(rightTuple == null){
+          rightTuple = rightChild.next();
+
+          if(rightTuple != null){
+            initRightDone = true;
+          }
+          else {
+            initRightDone = true;
+            end = true;
+            continue;
+          }
+        }
+        //////////////////////////////////////////////////////////////////////
+        // END INITIALIZATION STAGE
+        //////////////////////////////////////////////////////////////////////
+
+        // reset tuple slots for a new round
+        leftTupleSlots.clear();
+        innerTupleSlots.clear();
+        posRightTupleSlots = -1;
+        posLeftTupleSlots = -1;
+
+
+        //////////////////////////////////////////////////////////////////////
+        // BEGIN MOVE FORWARDING STAGE
+        //////////////////////////////////////////////////////////////////////
+
+        // This stage moves forward a tuple cursor on each side relation until a match
+        // is found
+        int cmp;
+        while ((end != true) && ((cmp = joinComparator.compare(leftTuple, rightTuple)) != 0)) {
+
+          // if right is lower than the left tuple, it means that all right tuples s.t. cmp <= 0 are
+          // matched tuples.
+          if (cmp > 0) {
+            // before getting a new tuple from the right,  a left null padded tuple should be built
+            // output a tuple with the nulls padded left tuple
+            Tuple nullPaddedTuple = createNullPaddedTuple(leftNumCols);
+            frameTuple.set(nullPaddedTuple, rightTuple);
+            projector.eval(frameTuple, outTuple);
+
+            // we simulate we found a match, which is exactly the null padded one
+            // BEFORE RETURN, MOVE FORWARD
+            rightTuple = rightChild.next();
+            if(rightTuple == null) {
+              end = true;
+            }
+            return outTuple;
+
+          } else if (cmp < 0) {
+            // If the left tuple is lower than the right tuple, just move forward the left tuple cursor.
+            leftTuple = leftChild.next();
+            if(leftTuple == null) {
+              end = true;
+              // in original algorithm we had return null ,
+              // but now we need to continue the end processing phase for remaining unprocessed right tuples
+            }
+          } // if (cmp<0)
+        } // while
+        //////////////////////////////////////////////////////////////////////
+        // END MOVE FORWARDING STAGE
+        //////////////////////////////////////////////////////////////////////
+
+        // once a match is found, retain all tuples with this key in tuple slots on each side
+        if(!end) {
+          endInPopulationStage = false;
+
+          boolean endOuter = false;
+          boolean endInner = false;
+
+          previous = new VTuple(leftTuple);
+          do {
+            leftTupleSlots.add(new VTuple(leftTuple));
+            leftTuple = leftChild.next();
+            if( leftTuple == null) {
+              endOuter = true;
+            }
+          } while ((endOuter != true) && (tupleComparator[0].compare(previous, leftTuple) == 0));
+          posLeftTupleSlots = 0;
+
+          previous = new VTuple(rightTuple);
+
+          do {
+            innerTupleSlots.add(new VTuple(rightTuple));
+            rightTuple = rightChild.next();
+            if(rightTuple == null) {
+              endInner = true;
+            }
+
+          } while ((endInner != true) && (tupleComparator[1].compare(previous, rightTuple) == 0) );
+          posRightTupleSlots = 0;
+
+          if ((endOuter == true) || (endInner == true)) {
+            end = true;
+            endInPopulationStage = true;
+          }
+        } // if end false
+      } // if newRound
+
+
+      // Now output result matching tuples from the slots
+      // if either we haven't reached end on neither side, or we did reach end on one(or both) sides
+      // but that happened in the slots population step (i.e. refers to next round)
+
+      if ((end == false) || ((end == true) && (endInPopulationStage == true))){
+
+        if(posLeftTupleSlots == 0){
+          nextLeft = new VTuple (leftTupleSlots.get(posLeftTupleSlots));
+          posLeftTupleSlots = posLeftTupleSlots + 1;
+        }
+
+
+        if(posRightTupleSlots <= (innerTupleSlots.size() -1)) {
+
+          Tuple aTuple = new VTuple(innerTupleSlots.get(posRightTupleSlots));
+          posRightTupleSlots = posRightTupleSlots + 1;
+
+          frameTuple.set(nextLeft, aTuple);
+          joinQual.eval(inSchema, frameTuple);
+          projector.eval(frameTuple, outTuple);
+          return outTuple;
+
+        } else {
+          // right (inner) slots reached end and should be rewind if there are still tuples in the outer slots
+          if(posLeftTupleSlots <= (leftTupleSlots.size() - 1)) {
+            //rewind the right slots position
+            posRightTupleSlots = 0;
+            Tuple aTuple = new VTuple(innerTupleSlots.get(posRightTupleSlots));
+            posRightTupleSlots = posRightTupleSlots + 1;
+            nextLeft = new VTuple (leftTupleSlots.get(posLeftTupleSlots));
+            posLeftTupleSlots = posLeftTupleSlots + 1;
+
+            frameTuple.set(nextLeft, aTuple);
+            joinQual.eval(inSchema, frameTuple);
+            projector.eval(frameTuple, outTuple);
+            return outTuple;
+          }
+        }
+      } // the second if end false
+    } // for
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+    leftTupleSlots.clear();
+    innerTupleSlots.clear();
+    posRightTupleSlots = -1;
+    posLeftTupleSlots = -1;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    leftTupleSlots.clear();
+    innerTupleSlots.clear();
+    leftTupleSlots = null;
+    innerTupleSlots = null;
+    joinNode = null;
+    joinQual = null;
+    projector = null;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java
new file mode 100644
index 0000000..2e676e9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java
@@ -0,0 +1,49 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.logical.SelectionNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class SelectionExec extends UnaryPhysicalExec  {
+  private final EvalNode qual;
+
+  public SelectionExec(TaskAttemptContext context,
+                       SelectionNode plan,
+                       PhysicalExec child) {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+    this.qual = plan.getQual();
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple;
+    while ((tuple = child.next()) != null) {
+      if (qual.eval(inSchema, tuple).isTrue()) {
+        return tuple;
+      }
+    }
+
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
new file mode 100644
index 0000000..6dbcc3f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
@@ -0,0 +1,316 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.engine.eval.ConstEval;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.eval.EvalTreeUtil;
+import org.apache.tajo.engine.eval.FieldEval;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.Target;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.engine.utils.*;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.*;
+
+
+public class SeqScanExec extends PhysicalExec {
+  private ScanNode plan;
+
+  private Scanner scanner = null;
+
+  private EvalNode qual = null;
+
+  private CatalogProtos.FragmentProto [] fragments;
+
+  private Projector projector;
+
+  private TableStats inputStats;
+
+  private TupleCacheKey cacheKey;
+
+  private boolean cacheRead = false;
+
+  public SeqScanExec(TaskAttemptContext context, AbstractStorageManager sm,
+                     ScanNode plan, CatalogProtos.FragmentProto [] fragments) throws IOException {
+    super(context, plan.getInSchema(), plan.getOutSchema());
+
+    this.plan = plan;
+    this.qual = plan.getQual();
+    this.fragments = fragments;
+
+    if (plan.isBroadcastTable()) {
+      cacheKey = new TupleCacheKey(
+          context.getTaskId().getQueryUnitId().getExecutionBlockId().toString(), plan.getTableName());
+    }
+  }
+
+  /**
+   * This method rewrites an input schema of column-partitioned table because
+   * there are no actual field values in data file in a column-partitioned table.
+   * So, this method removes partition key columns from the input schema.
+   *
+   * TODO - This implementation assumes that a fragment is always FileFragment.
+   * In the column partitioned table, a path has an important role to
+   * indicate partition keys. In this time, it is right. Later, we have to fix it.
+   */
+  private void rewriteColumnPartitionedTableSchema() throws IOException {
+    PartitionMethodDesc partitionDesc = plan.getTableDesc().getPartitionMethod();
+    Schema columnPartitionSchema = SchemaUtil.clone(partitionDesc.getExpressionSchema());
+    String qualifier = inSchema.getColumn(0).getQualifier();
+    columnPartitionSchema.setQualifier(qualifier);
+
+    // Remove partition key columns from an input schema.
+    this.inSchema = plan.getTableDesc().getSchema();
+
+    List<FileFragment> fileFragments = FragmentConvertor.convert(FileFragment.class, fragments);
+
+    // Get a partition key value from a given path
+    Tuple partitionRow =
+        TupleUtil.buildTupleFromPartitionPath(columnPartitionSchema, fileFragments.get(0).getPath(), false);
+
+    // Targets or search conditions may contain column references.
+    // However, actual values absent in tuples. So, Replace all column references by constant datum.
+    for (Column column : columnPartitionSchema.toArray()) {
+      FieldEval targetExpr = new FieldEval(column);
+      Datum datum = targetExpr.eval(columnPartitionSchema, partitionRow);
+      ConstEval constExpr = new ConstEval(datum);
+
+      for (Target target : plan.getTargets()) {
+        if (target.getEvalTree().equals(targetExpr)) {
+          if (!target.hasAlias()) {
+            target.setAlias(target.getEvalTree().getName());
+          }
+          target.setExpr(constExpr);
+        } else {
+          EvalTreeUtil.replace(target.getEvalTree(), targetExpr, constExpr);
+        }
+      }
+
+      if (plan.hasQual()) {
+        EvalTreeUtil.replace(plan.getQual(), targetExpr, constExpr);
+      }
+    }
+  }
+
+  public void init() throws IOException {
+    Schema projected;
+
+    if (fragments != null
+        && plan.getTableDesc().hasPartition()
+        && plan.getTableDesc().getPartitionMethod().getPartitionType() == CatalogProtos.PartitionType.COLUMN) {
+      rewriteColumnPartitionedTableSchema();
+    }
+
+    if (plan.hasTargets()) {
+      projected = new Schema();
+      Set<Column> columnSet = new HashSet<Column>();
+
+      if (plan.hasQual()) {
+        columnSet.addAll(EvalTreeUtil.findUniqueColumns(qual));
+      }
+
+      for (Target t : plan.getTargets()) {
+        columnSet.addAll(EvalTreeUtil.findUniqueColumns(t.getEvalTree()));
+      }
+
+      for (Column column : inSchema.getColumns()) {
+        if (columnSet.contains(column)) {
+          projected.addColumn(column);
+        }
+      }
+    } else {
+      projected = outSchema;
+    }
+
+    if (cacheKey != null) {
+      TupleCache tupleCache = TupleCache.getInstance();
+      if (tupleCache.isBroadcastCacheReady(cacheKey)) {
+        openCacheScanner();
+      } else {
+        if (TupleCache.getInstance().lockBroadcastScan(cacheKey)) {
+          scanAndAddCache(projected);
+          openCacheScanner();
+        } else {
+          Object lockMonitor = tupleCache.getLockMonitor();
+          synchronized (lockMonitor) {
+            try {
+              lockMonitor.wait(20 * 1000);
+            } catch (InterruptedException e) {
+            }
+          }
+          if (tupleCache.isBroadcastCacheReady(cacheKey)) {
+            openCacheScanner();
+          } else {
+            initScanner(projected);
+          }
+        }
+      }
+    } else {
+      initScanner(projected);
+    }
+  }
+
+  private void initScanner(Schema projected) throws IOException {
+    this.projector = new Projector(inSchema, outSchema, plan.getTargets());
+
+    if (fragments != null) {
+      if (fragments.length > 1) {
+        this.scanner = new MergeScanner(context.getConf(), plan.getPhysicalSchema(), plan.getTableDesc().getMeta(),
+            FragmentConvertor.<FileFragment>convert(context.getConf(), plan.getTableDesc().getMeta().getStoreType(),
+                fragments), projected
+        );
+      } else {
+        this.scanner = StorageManagerFactory.getStorageManager(
+            context.getConf()).getScanner(plan.getTableDesc().getMeta(), plan.getPhysicalSchema(), fragments[0],
+            projected);
+      }
+      scanner.init();
+    }
+  }
+
+  private void openCacheScanner() throws IOException {
+    Scanner cacheScanner = TupleCache.getInstance().openCacheScanner(cacheKey, plan.getPhysicalSchema());
+    if (cacheScanner != null) {
+      scanner = cacheScanner;
+      cacheRead = true;
+    }
+  }
+
+  private void scanAndAddCache(Schema projected) throws IOException {
+    initScanner(projected);
+
+    List<Tuple> broadcastTupleCacheList = new ArrayList<Tuple>();
+    while (true) {
+      Tuple tuple = next();
+      if (tuple != null) {
+        broadcastTupleCacheList.add(tuple);
+      } else {
+        break;
+      }
+    }
+
+    scanner.close();
+    scanner = null;
+
+    TupleCache.getInstance().addBroadcastCache(cacheKey, broadcastTupleCacheList);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (fragments == null) {
+      return null;
+    }
+
+    Tuple tuple;
+    Tuple outTuple = new VTuple(outColumnNum);
+
+    if (!plan.hasQual()) {
+      if ((tuple = scanner.next()) != null) {
+        if (cacheRead) {
+          return tuple;
+        }
+        projector.eval(tuple, outTuple);
+        outTuple.setOffset(tuple.getOffset());
+        return outTuple;
+      } else {
+        return null;
+      }
+    } else {
+      while ((tuple = scanner.next()) != null) {
+        if (cacheRead) {
+          return tuple;
+        }
+        if (qual.eval(inSchema, tuple).isTrue()) {
+          projector.eval(tuple, outTuple);
+          return outTuple;
+        }
+      }
+      return null;
+    }
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    scanner.reset();
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.cleanup(null, scanner);
+    if (scanner != null) {
+      try {
+        TableStats stat = scanner.getInputStats();
+        if (stat != null) {
+          inputStats = (TableStats)(stat.clone());
+        }
+      } catch (CloneNotSupportedException e) {
+        e.printStackTrace();
+      }
+    }
+    scanner = null;
+    plan = null;
+    qual = null;
+    projector = null;
+  }
+
+  public String getTableName() {
+    return plan.getTableName();
+  }
+
+  @Override
+  public float getProgress() {
+    if (scanner == null) {
+      return 1.0f;
+    } else {
+      return scanner.getProgress();
+    }
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    if (scanner != null) {
+      return scanner.getInputStats();
+    } else {
+      return inputStats;
+    }
+  }
+
+  @Override
+  public String toString() {
+    if (scanner != null) {
+      return "SeqScanExec:" + plan.getTableName() + "," + scanner.getClass().getName();
+    } else {
+      return "SeqScanExec:" + plan.getTableName();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java
new file mode 100644
index 0000000..629889d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.tajo.engine.function.FunctionContext;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+/**
+ * This is the sort-based aggregation operator.
+ *
+ * <h3>Implementation</h3>
+ * Sort Aggregation has two states while running.
+ *
+ * <h4>Aggregate state</h4>
+ * If lastkey is null or lastkey is equivalent to the current key, sort aggregation is changed to this state.
+ * In this state, this operator aggregates measure values via aggregation functions.
+ *
+ * <h4>Finalize state</h4>
+ * If currentKey is different from the last key, it computes final aggregation results, and then
+ * it makes an output tuple.
+ */
+public class SortAggregateExec extends AggregationExec {
+  private Tuple lastKey = null;
+  private boolean finished = false;
+  private FunctionContext contexts[];
+
+  public SortAggregateExec(TaskAttemptContext context, GroupbyNode plan, PhysicalExec child) throws IOException {
+    super(context, plan, child);
+    contexts = new FunctionContext[plan.getAggFunctions().length];
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple currentKey;
+    Tuple tuple;
+    Tuple outputTuple = null;
+
+    while(!context.isStopped() && (tuple = child.next()) != null) {
+
+      // get a key tuple
+      currentKey = new VTuple(groupingKeyIds.length);
+      for(int i = 0; i < groupingKeyIds.length; i++) {
+        currentKey.put(i, tuple.get(groupingKeyIds[i]));
+      }
+
+      /** Aggregation State */
+      if (lastKey == null || lastKey.equals(currentKey)) {
+        if (lastKey == null) {
+          for(int i = 0; i < aggFunctionsNum; i++) {
+            contexts[i] = aggFunctions[i].newContext();
+            aggFunctions[i].merge(contexts[i], inSchema, tuple);
+          }
+          lastKey = currentKey;
+        } else {
+          // aggregate
+          for (int i = 0; i < aggFunctionsNum; i++) {
+            aggFunctions[i].merge(contexts[i], inSchema, tuple);
+          }
+        }
+
+      } else { /** Finalization State */
+        // finalize aggregate and return
+        outputTuple = new VTuple(outSchema.size());
+        int tupleIdx = 0;
+
+        for(; tupleIdx < groupingKeyNum; tupleIdx++) {
+          outputTuple.put(tupleIdx, lastKey.get(tupleIdx));
+        }
+        for(int aggFuncIdx = 0; aggFuncIdx < aggFunctionsNum; tupleIdx++, aggFuncIdx++) {
+          outputTuple.put(tupleIdx, aggFunctions[aggFuncIdx].terminate(contexts[aggFuncIdx]));
+        }
+
+        for(int evalIdx = 0; evalIdx < aggFunctionsNum; evalIdx++) {
+          contexts[evalIdx] = aggFunctions[evalIdx].newContext();
+          aggFunctions[evalIdx].merge(contexts[evalIdx], inSchema, tuple);
+        }
+
+        lastKey = currentKey;
+        return outputTuple;
+      }
+    } // while loop
+
+    if (!finished) {
+      outputTuple = new VTuple(outSchema.size());
+      int tupleIdx = 0;
+      for(; tupleIdx < groupingKeyNum; tupleIdx++) {
+        outputTuple.put(tupleIdx, lastKey.get(tupleIdx));
+      }
+      for(int aggFuncIdx = 0; aggFuncIdx < aggFunctionsNum; tupleIdx++, aggFuncIdx++) {
+        outputTuple.put(tupleIdx, aggFunctions[aggFuncIdx].terminate(contexts[aggFuncIdx]));
+      }
+      finished = true;
+    }
+    return outputTuple;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    super.rescan();
+
+    lastKey = null;
+    finished = false;
+  }
+}


[40/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
new file mode 100644
index 0000000..b8f3311
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
@@ -0,0 +1,739 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.algebra.*;
+
+import java.util.Stack;
+
+public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTEXT, RESULT> {
+
+  /**
+   * The prehook is called before each expression is visited.
+   */
+  public void preHook(CONTEXT ctx, Stack<Expr> stack, Expr expr) throws PlanningException {
+  }
+
+
+  /**
+   * The posthook is called before each expression is visited.
+   */
+  public RESULT postHook(CONTEXT ctx, Stack<Expr> stack, Expr expr, RESULT current) throws PlanningException {
+    return current;
+  }
+
+  /**
+   * visit visits each relational operator expression recursively.
+   *
+   * @param stack The stack contains the upper operators' type.
+   * @param expr The visiting relational operator
+   */
+  public RESULT visit(CONTEXT ctx, Stack<Expr> stack, Expr expr) throws PlanningException {
+    preHook(ctx, stack, expr);
+
+    RESULT current;
+
+    switch (expr.getType()) {
+
+    case Projection:
+      current = visitProjection(ctx, stack, (Projection) expr);
+      break;
+    case Limit:
+      current = visitLimit(ctx, stack, (Limit) expr);
+      break;
+    case Sort:
+      current = visitSort(ctx, stack, (Sort) expr);
+      break;
+    case Having:
+      current = visitHaving(ctx, stack, (Having) expr);
+      break;
+    case Aggregation:
+      current = visitGroupBy(ctx, stack, (Aggregation) expr);
+      break;
+    case Join:
+      current = visitJoin(ctx, stack, (Join) expr);
+      break;
+    case Filter:
+      current = visitFilter(ctx, stack, (Selection) expr);
+      break;
+    case Union:
+      current = visitUnion(ctx, stack, (SetOperation) expr);
+      break;
+    case Except:
+      current = visitExcept(ctx, stack, (SetOperation) expr);
+      break;
+    case Intersect:
+      current = visitIntersect(ctx, stack, (SetOperation) expr);
+      break;
+    case SimpleTableSubQuery:
+      current = visitSimpleTableSubQuery(ctx, stack, (SimpleTableSubQuery) expr);
+      break;
+    case TablePrimaryTableSubQuery:
+      current = visitTableSubQuery(ctx, stack, (TablePrimarySubQuery) expr);
+      break;
+    case RelationList:
+      current = visitRelationList(ctx, stack, (RelationList) expr);
+      break;
+    case Relation:
+      current = visitRelation(ctx, stack, (Relation) expr);
+      break;
+    case ScalarSubQuery:
+      current = visitScalarSubQuery(ctx, stack, (ScalarSubQuery) expr);
+      break;
+    case Explain:
+      current = visitExplain(ctx, stack, (Explain) expr);
+      break;
+
+    case CreateDatabase:
+      current = visitCreateDatabase(ctx, stack, (CreateDatabase) expr);
+      break;
+    case DropDatabase:
+      current = visitDropDatabase(ctx, stack, (DropDatabase) expr);
+      break;
+    case CreateTable:
+      current = visitCreateTable(ctx, stack, (CreateTable) expr);
+      break;
+    case DropTable:
+      current = visitDropTable(ctx, stack, (DropTable) expr);
+      break;
+    case AlterTablespace:
+      current = visitAlterTablespace(ctx, stack, (AlterTablespace) expr);
+      break;
+    case AlterTable:
+      current = visitAlterTable(ctx, stack, (AlterTable) expr);
+      break;
+
+    case Insert:
+      current = visitInsert(ctx, stack, (Insert) expr);
+      break;
+
+    case And:
+      current = visitAnd(ctx, stack, (BinaryOperator) expr);
+      break;
+    case Or:
+      current = visitOr(ctx, stack, (BinaryOperator) expr);
+      break;
+    case Not:
+      current = visitNot(ctx, stack, (NotExpr) expr);
+      break;
+
+    case Equals:
+      current = visitEquals(ctx, stack, (BinaryOperator) expr);
+      break;
+    case NotEquals:
+      current = visitNotEquals(ctx, stack, (BinaryOperator) expr);
+      break;
+    case LessThan:
+      current = visitLessThan(ctx, stack, (BinaryOperator) expr);
+      break;
+    case LessThanOrEquals:
+      current = visitLessThanOrEquals(ctx, stack, (BinaryOperator) expr);
+      break;
+    case GreaterThan:
+      current = visitGreaterThan(ctx, stack, (BinaryOperator) expr);
+      break;
+    case GreaterThanOrEquals:
+      current = visitGreaterThanOrEquals(ctx, stack, (BinaryOperator) expr);
+      break;
+
+    // Other Predicates
+    case Between:
+      current = visitBetween(ctx, stack, (BetweenPredicate) expr);
+      break;
+    case CaseWhen:
+      current = visitCaseWhen(ctx, stack, (CaseWhenPredicate) expr);
+      break;
+    case IsNullPredicate:
+      current = visitIsNullPredicate(ctx, stack, (IsNullPredicate) expr);
+      break;
+    case InPredicate:
+      current = visitInPredicate(ctx, stack, (InPredicate) expr);
+      break;
+    case ValueList:
+      current = visitValueListExpr(ctx, stack, (ValueListExpr) expr);
+      break;
+    case ExistsPredicate:
+      current = visitExistsPredicate(ctx, stack, (ExistsPredicate) expr);
+      break;
+
+    // String Operator or Pattern Matching Predicates
+    case LikePredicate:
+      current = visitLikePredicate(ctx, stack, (PatternMatchPredicate) expr);
+      break;
+    case SimilarToPredicate:
+      current = visitSimilarToPredicate(ctx, stack, (PatternMatchPredicate) expr);
+      break;
+    case Regexp:
+      current = visitRegexpPredicate(ctx, stack, (PatternMatchPredicate) expr);
+      break;
+    case Concatenate:
+      current = visitConcatenate(ctx, stack, (BinaryOperator) expr);
+      break;
+
+    // Arithmetic Operators
+    case Plus:
+      current = visitPlus(ctx, stack, (BinaryOperator) expr);
+      break;
+    case Minus:
+      current = visitMinus(ctx, stack, (BinaryOperator) expr);
+      break;
+    case Multiply:
+      current = visitMultiply(ctx, stack, (BinaryOperator) expr);
+      break;
+    case Divide:
+      current = visitDivide(ctx, stack, (BinaryOperator) expr);
+      break;
+    case Modular:
+      current = visitModular(ctx, stack, (BinaryOperator) expr);
+      break;
+
+    // Other Expressions
+    case Sign:
+      current = visitSign(ctx, stack, (SignedExpr) expr);
+      break;
+    case Column:
+      current = visitColumnReference(ctx, stack, (ColumnReferenceExpr) expr);
+      break;
+    case Target:
+      current = visitTargetExpr(ctx, stack, (NamedExpr) expr);
+      break;
+    case Function:
+      current = visitFunction(ctx, stack, (FunctionExpr) expr);
+      break;
+    case Asterisk:
+      current = visitQualifiedAsterisk(ctx, stack, (QualifiedAsteriskExpr) expr);
+      break;
+
+
+    case CountRowsFunction:
+      current = visitCountRowsFunction(ctx, stack, (CountRowsFunctionExpr) expr);
+      break;
+    case GeneralSetFunction:
+      current = visitGeneralSetFunction(ctx, stack, (GeneralSetFunctionExpr) expr);
+      break;
+
+    case DataType:
+      current = visitDataType(ctx, stack, (DataTypeExpr) expr);
+      break;
+    case Cast:
+      current = visitCastExpr(ctx, stack, (CastExpr) expr);
+      break;
+    case Literal:
+      current = visitLiteral(ctx, stack, (LiteralValue) expr);
+      break;
+    case NullLiteral:
+      current = visitNullLiteral(ctx, stack, (NullLiteral) expr);
+      break;
+    case DateLiteral:
+      current = visitDateLiteral(ctx, stack, (DateLiteral) expr);
+      break;
+    case TimeLiteral:
+      current = visitTimeLiteral(ctx, stack, (TimeLiteral) expr);
+      break;
+    case TimestampLiteral:
+      current = visitTimestampLiteral(ctx, stack, (TimestampLiteral) expr);
+      break;
+
+
+
+    default:
+      throw new PlanningException("Cannot support this type algebra \"" + expr.getType() + "\"");
+    }
+
+    // skip postHook against only one relation
+    if (expr.getType() == OpType.RelationList) {
+      RelationList relationList = (RelationList)expr;
+      if (relationList.size() == 1 && relationList.getRelations()[0].getType() == OpType.Relation) {
+        return current;
+      }
+    }
+
+    postHook(ctx, stack, expr, current);
+    return current;
+  }
+
+  private RESULT visitDefaultUnaryExpr(CONTEXT ctx, Stack<Expr> stack, UnaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    RESULT child = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return child;
+  }
+
+  private RESULT visitDefaultBinaryExpr(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr)
+      throws PlanningException {
+    stack.push(expr);
+    RESULT child = visit(ctx, stack, expr.getLeft());
+    visit(ctx, stack, expr.getRight());
+    stack.pop();
+    return child;
+  }
+
+  @Override
+  public RESULT visitProjection(CONTEXT ctx, Stack<Expr> stack, Projection expr) throws PlanningException {
+    stack.push(expr);
+    try {
+      for (NamedExpr target : expr.getNamedExprs()) {
+        visit(ctx, stack, target);
+      }
+      if (expr.hasChild()) {
+        return visit(ctx, stack, expr.getChild());
+      }
+    } finally {
+      stack.pop();
+    }
+    return null;
+  }
+
+  @Override
+  public RESULT visitLimit(CONTEXT ctx, Stack<Expr> stack, Limit expr) throws PlanningException {
+    stack.push(expr);
+    visit(ctx, stack, expr.getFetchFirstNum());
+    RESULT result = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitSort(CONTEXT ctx, Stack<Expr> stack, Sort expr) throws PlanningException {
+    stack.push(expr);
+    for (Sort.SortSpec sortSpec : expr.getSortSpecs()) {
+      visit(ctx, stack, sortSpec.getKey());
+    }
+    RESULT result = visit(ctx, stack, expr.getChild());
+    return result;
+  }
+
+  @Override
+  public RESULT visitHaving(CONTEXT ctx, Stack<Expr> stack, Having expr) throws PlanningException {
+    stack.push(expr);
+    visit(ctx, stack, expr.getQual());
+    RESULT result = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitGroupBy(CONTEXT ctx, Stack<Expr> stack, Aggregation expr) throws PlanningException {
+    stack.push(expr);
+
+    for (org.apache.tajo.algebra.Aggregation.GroupElement groupElement : expr.getGroupSet()) {
+      for (Expr groupingSet : groupElement.getGroupingSets()) {
+        visit(ctx, stack, groupingSet);
+      }
+    }
+
+    RESULT result = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitJoin(CONTEXT ctx, Stack<Expr> stack, Join expr) throws PlanningException {
+    stack.push(expr);
+    visit(ctx, stack, expr.getQual());
+    visit(ctx, stack, expr.getLeft());
+    RESULT result = visit(ctx, stack, expr.getRight());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitFilter(CONTEXT ctx, Stack<Expr> stack, Selection expr) throws PlanningException {
+    stack.push(expr);
+    visit(ctx, stack, expr.getQual());
+    RESULT result = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitUnion(CONTEXT ctx, Stack<Expr> stack, SetOperation expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitExcept(CONTEXT ctx, Stack<Expr> stack, SetOperation expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitIntersect(CONTEXT ctx, Stack<Expr> stack, SetOperation expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitSimpleTableSubQuery(CONTEXT ctx, Stack<Expr> stack, SimpleTableSubQuery expr)
+      throws PlanningException {
+    return visitDefaultUnaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitTableSubQuery(CONTEXT ctx, Stack<Expr> stack, TablePrimarySubQuery expr)
+      throws PlanningException {
+    stack.push(expr);
+    RESULT child = visit(ctx, stack, expr.getSubQuery());
+    stack.pop();
+    return child;
+  }
+
+  @Override
+  public RESULT visitRelationList(CONTEXT ctx, Stack<Expr> stack, RelationList expr) throws PlanningException {
+    stack.push(expr);
+    RESULT child = null;
+    for (Expr e : expr.getRelations()) {
+      child = visit(ctx, stack, e);
+    }
+    stack.pop();
+    return child;
+  }
+
+  @Override
+  public RESULT visitRelation(CONTEXT ctx, Stack<Expr> stack, Relation expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitScalarSubQuery(CONTEXT ctx, Stack<Expr> stack, ScalarSubQuery expr) throws PlanningException {
+    return visitDefaultUnaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitExplain(CONTEXT ctx, Stack<Expr> stack, Explain expr) throws PlanningException {
+    stack.push(expr);
+    RESULT child = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return child;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Data Definition Language Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitCreateDatabase(CONTEXT ctx, Stack<Expr> stack, CreateDatabase expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitDropDatabase(CONTEXT ctx, Stack<Expr> stack, DropDatabase expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitCreateTable(CONTEXT ctx, Stack<Expr> stack, CreateTable expr) throws PlanningException {
+    stack.push(expr);
+    RESULT child = null;
+    if (expr.hasSubQuery()) {
+      child = visit(ctx, stack, expr.getSubQuery());
+    }
+    stack.pop();
+    return child;
+  }
+
+  @Override
+  public RESULT visitDropTable(CONTEXT ctx, Stack<Expr> stack, DropTable expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitAlterTablespace(CONTEXT ctx, Stack<Expr> stack, AlterTablespace expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitAlterTable(CONTEXT ctx, Stack<Expr> stack, AlterTable expr) throws PlanningException {
+    return null;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Insert or Update Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public RESULT visitInsert(CONTEXT ctx, Stack<Expr> stack, Insert expr) throws PlanningException {
+    stack.push(expr);
+    RESULT child = visit(ctx, stack, expr.getSubQuery());
+    stack.pop();
+    return child;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Logical Operator Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitAnd(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitOr(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitNot(CONTEXT ctx, Stack<Expr> stack, NotExpr expr) throws PlanningException {
+    return visitDefaultUnaryExpr(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Comparison Predicates Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  @Override
+  public RESULT visitEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitNotEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitLessThan(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitLessThanOrEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitGreaterThan(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitGreaterThanOrEquals(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr)
+      throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Other Predicates Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitBetween(CONTEXT ctx, Stack<Expr> stack, BetweenPredicate expr) throws PlanningException {
+    stack.push(expr);
+    RESULT result = visit(ctx, stack, expr.predicand());
+    visit(ctx, stack, expr.begin());
+    visit(ctx, stack, expr.end());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitCaseWhen(CONTEXT ctx, Stack<Expr> stack, CaseWhenPredicate expr) throws PlanningException {
+    stack.push(expr);
+    RESULT result = null;
+    for (CaseWhenPredicate.WhenExpr when : expr.getWhens()) {
+      result = visit(ctx, stack, when.getCondition());
+      visit(ctx, stack, when.getResult());
+    }
+    if (expr.hasElseResult()) {
+      visit(ctx, stack, expr.getElseResult());
+    }
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitIsNullPredicate(CONTEXT ctx, Stack<Expr> stack, IsNullPredicate expr) throws PlanningException {
+    return visitDefaultUnaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitInPredicate(CONTEXT ctx, Stack<Expr> stack, InPredicate expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitValueListExpr(CONTEXT ctx, Stack<Expr> stack, ValueListExpr expr) throws PlanningException {
+    stack.push(expr);
+    RESULT result = null;
+    for (Expr value : expr.getValues()) {
+      result = visit(ctx, stack, value);
+    }
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitExistsPredicate(CONTEXT ctx, Stack<Expr> stack, ExistsPredicate expr) throws PlanningException {
+    return visitDefaultUnaryExpr(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // String Operator or Pattern Matching Predicates Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  @Override
+  public RESULT visitLikePredicate(CONTEXT ctx, Stack<Expr> stack, PatternMatchPredicate expr)
+      throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitSimilarToPredicate(CONTEXT ctx, Stack<Expr> stack, PatternMatchPredicate expr)
+      throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitRegexpPredicate(CONTEXT ctx, Stack<Expr> stack, PatternMatchPredicate expr)
+      throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitConcatenate(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Arithmetic Operators
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitPlus(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitMinus(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitMultiply(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitDivide(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitModular(CONTEXT ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitDefaultBinaryExpr(ctx, stack, expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Other Expressions
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitSign(CONTEXT ctx, Stack<Expr> stack, SignedExpr expr) throws PlanningException {
+    return visitDefaultUnaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitColumnReference(CONTEXT ctx, Stack<Expr> stack, ColumnReferenceExpr expr)
+      throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitTargetExpr(CONTEXT ctx, Stack<Expr> stack, NamedExpr expr) throws PlanningException {
+    return visitDefaultUnaryExpr(ctx, stack, expr);
+  }
+
+  @Override
+  public RESULT visitFunction(CONTEXT ctx, Stack<Expr> stack, FunctionExpr expr) throws PlanningException {
+    stack.push(expr);
+    RESULT result = null;
+    if (expr.hasParams()) {
+      for (Expr param : expr.getParams()) {
+        result = visit(ctx, stack, param);
+      }
+    }
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitQualifiedAsterisk(CONTEXT ctx, Stack<Expr> stack, QualifiedAsteriskExpr expr) throws PlanningException {
+    return null;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // General Set Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitCountRowsFunction(CONTEXT ctx, Stack<Expr> stack, CountRowsFunctionExpr expr)
+      throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitGeneralSetFunction(CONTEXT ctx, Stack<Expr> stack, GeneralSetFunctionExpr expr)
+      throws PlanningException {
+    stack.push(expr);
+    RESULT result = null;
+    for (Expr param : expr.getParams()) {
+      result = visit(ctx, stack, param);
+    }
+    stack.pop();
+    return result;
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Literal Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public RESULT visitDataType(CONTEXT ctx, Stack<Expr> stack, DataTypeExpr expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitCastExpr(CONTEXT ctx, Stack<Expr> stack, CastExpr expr) throws PlanningException {
+    stack.push(expr);
+    RESULT result = visit(ctx, stack, expr.getOperand());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitLiteral(CONTEXT ctx, Stack<Expr> stack, LiteralValue expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitNullLiteral(CONTEXT ctx, Stack<Expr> stack, NullLiteral expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitTimestampLiteral(CONTEXT ctx, Stack<Expr> stack, TimestampLiteral expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitTimeLiteral(CONTEXT ctx, Stack<Expr> stack, TimeLiteral expr) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitDateLiteral(CONTEXT ctx, Stack<Expr> stack, DateLiteral expr) throws PlanningException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java
new file mode 100644
index 0000000..0f758bf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java
@@ -0,0 +1,319 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.engine.planner.logical.*;
+
+import java.util.Stack;
+
+public class BasicLogicalPlanVisitor<CONTEXT, RESULT> implements LogicalPlanVisitor<CONTEXT, RESULT> {
+
+  /**
+   * The prehook is called before each node is visited.
+   */
+  @SuppressWarnings("unused")
+  public void preHook(LogicalPlan plan, LogicalNode node, Stack<LogicalNode> stack, CONTEXT data)
+      throws PlanningException {
+  }
+
+  /**
+   * The posthook is called after each node is visited.
+   */
+  @SuppressWarnings("unused")
+  public void postHook(LogicalPlan plan, LogicalNode node, Stack<LogicalNode> stack, CONTEXT data)
+      throws PlanningException {
+  }
+
+  public CONTEXT visit(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block)
+      throws PlanningException {
+    visit(context, plan, block, block.getRoot(), new Stack<LogicalNode>());
+    return context;
+  }
+
+  /**
+   * visit visits each logicalNode recursively.
+   */
+  public RESULT visit(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, LogicalNode node,
+                      Stack<LogicalNode> stack)
+      throws PlanningException {
+    RESULT current;
+    switch (node.getType()) {
+      case ROOT:
+        current = visitRoot(context, plan, block, (LogicalRootNode) node, stack);
+        break;
+      case EXPRS:
+        return null;
+      case PROJECTION:
+        current = visitProjection(context, plan, block, (ProjectionNode) node, stack);
+        break;
+      case LIMIT:
+        current = visitLimit(context, plan, block, (LimitNode) node, stack);
+        break;
+      case SORT:
+        current = visitSort(context, plan, block, (SortNode) node, stack);
+        break;
+      case HAVING:
+        current = visitHaving(context, plan, block, (HavingNode) node, stack);
+        break;
+      case GROUP_BY:
+        current = visitGroupBy(context, plan, block, (GroupbyNode) node, stack);
+        break;
+      case SELECTION:
+        current = visitFilter(context, plan, block, (SelectionNode) node, stack);
+        break;
+      case JOIN:
+        current = visitJoin(context, plan, block, (JoinNode) node, stack);
+        break;
+      case UNION:
+        current = visitUnion(context, plan, block, (UnionNode) node, stack);
+        break;
+      case EXCEPT:
+        current = visitExcept(context, plan, block, (ExceptNode) node, stack);
+        break;
+      case INTERSECT:
+        current = visitIntersect(context, plan, block, (IntersectNode) node, stack);
+        break;
+      case TABLE_SUBQUERY:
+        current = visitTableSubQuery(context, plan, block, (TableSubQueryNode) node, stack);
+        break;
+      case SCAN:
+        current = visitScan(context, plan, block, (ScanNode) node, stack);
+        break;
+      case PARTITIONS_SCAN:
+        current = visitPartitionedTableScan(context, plan, block, (PartitionedTableScanNode) node, stack);
+        break;
+      case STORE:
+        current = visitStoreTable(context, plan, block, (StoreTableNode) node, stack);
+        break;
+      case INSERT:
+        current = visitInsert(context, plan, block, (InsertNode) node, stack);
+        break;
+      case CREATE_DATABASE:
+        current = visitCreateDatabase(context, plan, block, (CreateDatabaseNode) node, stack);
+        break;
+      case DROP_DATABASE:
+        current = visitDropDatabase(context, plan, block, (DropDatabaseNode) node, stack);
+        break;
+      case CREATE_TABLE:
+        current = visitCreateTable(context, plan, block, (CreateTableNode) node, stack);
+        break;
+      case DROP_TABLE:
+        current = visitDropTable(context, plan, block, (DropTableNode) node, stack);
+        break;
+      case ALTER_TABLESPACE:
+        current = visitAlterTablespace(context, plan, block, (AlterTablespaceNode) node, stack);
+        break;
+      case ALTER_TABLE:
+        current = visitAlterTable(context, plan, block, (AlterTableNode) node, stack);
+        break;
+      default:
+        throw new PlanningException("Unknown logical node type: " + node.getType());
+    }
+
+    return current;
+  }
+
+  @Override
+  public RESULT visitRoot(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, LogicalRootNode node,
+                          Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitProjection(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, ProjectionNode node,
+                                Stack<LogicalNode> stack)
+      throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitLimit(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, LimitNode node,
+                           Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitSort(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, SortNode node,
+                          Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitHaving(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, HavingNode node,
+                            Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitGroupBy(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, GroupbyNode node,
+                             Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitFilter(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, SelectionNode node,
+                            Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitJoin(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode node,
+                          Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getLeftChild(), stack);
+    visit(context, plan, block, node.getRightChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitUnion(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, UnionNode node,
+                           Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    LogicalPlan.QueryBlock leftBlock = plan.getBlock(node.getLeftChild());
+    RESULT result = visit(context, plan, leftBlock, leftBlock.getRoot(), stack);
+    LogicalPlan.QueryBlock rightBlock = plan.getBlock(node.getRightChild());
+    visit(context, plan, rightBlock, rightBlock.getRoot(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitExcept(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, ExceptNode node,
+                            Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getLeftChild(), stack);
+    visit(context, plan, block, node.getRightChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitIntersect(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, IntersectNode node,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getLeftChild(), stack);
+    visit(context, plan, block, node.getRightChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitTableSubQuery(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                   TableSubQueryNode node, Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    LogicalPlan.QueryBlock childBlock = plan.getBlock(node.getSubQuery());
+    RESULT result = visit(context, plan, childBlock, childBlock.getRoot(), new Stack<LogicalNode>());
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitScan(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode node,
+                          Stack<LogicalNode> stack) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitPartitionedTableScan(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                          PartitionedTableScanNode node, Stack<LogicalNode> stack)
+      throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitStoreTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, StoreTableNode node,
+                                Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitInsert(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, InsertNode node,
+                            Stack<LogicalNode> stack) throws PlanningException {
+    stack.push(node);
+    RESULT result = visit(context, plan, block, node.getChild(), stack);
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitCreateDatabase(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                    CreateDatabaseNode node, Stack<LogicalNode> stack) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitDropDatabase(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, DropDatabaseNode node, Stack<LogicalNode> stack) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitCreateTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, CreateTableNode node,
+                                 Stack<LogicalNode> stack) throws PlanningException {
+    RESULT result = null;
+    stack.push(node);
+    if (node.hasSubQuery()) {
+      result = visit(context, plan, block, node.getChild(), stack);
+    }
+    stack.pop();
+    return result;
+  }
+
+  @Override
+  public RESULT visitDropTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, DropTableNode node,
+                               Stack<LogicalNode> stack) {
+    return null;
+  }
+
+  @Override
+  public RESULT visitAlterTablespace(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                     AlterTablespaceNode node, Stack<LogicalNode> stack) throws PlanningException {
+    return null;
+  }
+
+  @Override
+  public RESULT visitAlterTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, AlterTableNode node,
+                                 Stack<LogicalNode> stack) {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java
new file mode 100644
index 0000000..14ac85f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner;
+
+import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.engine.planner.logical.JoinNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+
+import java.util.Stack;
+
+public class BroadcastJoinPlanVisitor extends BasicLogicalPlanVisitor<GlobalPlanner.GlobalPlanContext, LogicalNode> {
+
+  @Override
+  public LogicalNode visitJoin(GlobalPlanner.GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                               JoinNode node, Stack<LogicalNode> stack) throws PlanningException {
+    LogicalNode leftChild = node.getLeftChild();
+    LogicalNode rightChild = node.getRightChild();
+
+    if (isScanNode(leftChild) && isScanNode(rightChild)) {
+      node.setCandidateBroadcast(true);
+      node.getBroadcastTargets().add(leftChild);
+      node.getBroadcastTargets().add(rightChild);
+      return node;
+    }
+
+    if(!isScanNode(leftChild)) {
+      visit(context, plan, block, leftChild, stack);
+    }
+
+    if(!isScanNode(rightChild)) {
+      visit(context, plan, block, rightChild, stack);
+    }
+
+    if(isBroadcastCandidateNode(leftChild) && isBroadcastCandidateNode(rightChild)) {
+      node.setCandidateBroadcast(true);
+      if(leftChild.getType() == NodeType.JOIN) {
+        node.getBroadcastTargets().addAll(((JoinNode)leftChild).getBroadcastTargets());
+      } else {
+        node.getBroadcastTargets().add(leftChild);
+      }
+
+      if(rightChild.getType() == NodeType.JOIN) {
+        node.getBroadcastTargets().addAll(((JoinNode)rightChild).getBroadcastTargets());
+      } else {
+        node.getBroadcastTargets().add(rightChild);
+      }
+    }
+
+    return node;
+  }
+
+  private static boolean isBroadcastCandidateNode(LogicalNode node) {
+    if(node.getType() == NodeType.SCAN ||
+        node.getType() == NodeType.PARTITIONS_SCAN) {
+      return true;
+    }
+
+    if(node.getType() == NodeType.JOIN && ((JoinNode)node).isCandidateBroadcast()) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private static boolean isScanNode(LogicalNode node) {
+    return node.getType() == NodeType.SCAN ||
+        node.getType() == NodeType.PARTITIONS_SCAN;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java
new file mode 100644
index 0000000..9dd8700
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java
@@ -0,0 +1,236 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.engine.planner.logical.*;
+
+import java.util.Stack;
+
+/**
+ * It returns a list of node plan strings.
+ */
+public class ExplainLogicalPlanVisitor extends BasicLogicalPlanVisitor<ExplainLogicalPlanVisitor.Context, LogicalNode> {
+
+  public static class Context {
+    public int maxDepth  = -1;
+    public int depth = 0;
+    public Stack<DepthString> explains = new Stack<DepthString>();
+
+    public void add(int depth, PlanString planString) {
+      maxDepth = Math.max(maxDepth, depth);
+      explains.push(new DepthString(depth, planString));
+    }
+
+    public int getMaxDepth() {
+      return this.maxDepth;
+    }
+
+    public Stack<DepthString> getExplains() {
+      return explains;
+    }
+  }
+
+  public static class DepthString {
+    private int depth;
+    private PlanString planStr;
+
+    DepthString(int depth, PlanString planStr) {
+      this.depth = depth;
+      this.planStr = planStr;
+    }
+
+    public int getDepth() {
+      return depth;
+    }
+
+    public PlanString getPlanString() {
+      return planStr;
+    }
+  }
+
+  public Context getBlockPlanStrings(@Nullable LogicalPlan plan, LogicalNode node) throws PlanningException {
+    Stack<LogicalNode> stack = new Stack<LogicalNode>();
+    Context explainContext = new Context();
+    visit(explainContext, plan, null, node, stack);
+    return explainContext;
+  }
+
+  @Override
+  public LogicalNode visitRoot(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, LogicalRootNode node, Stack<LogicalNode> stack)
+      throws PlanningException {
+    return visit(context, plan, block, node.getChild(), stack);
+  }
+
+  @Override
+  public LogicalNode visitProjection(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                     ProjectionNode node, Stack<LogicalNode> stack)
+      throws PlanningException {
+    return visitUnaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitLimit(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                LimitNode node, Stack<LogicalNode> stack) throws PlanningException {
+    return visitUnaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitSort(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, SortNode node,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    return visitUnaryNode(context, plan, block, node, stack);
+  }
+
+  public LogicalNode visitHaving(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, HavingNode node,
+                                  Stack<LogicalNode> stack) throws PlanningException {
+    return visitUnaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitGroupBy(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, GroupbyNode node,
+                                  Stack<LogicalNode> stack) throws PlanningException {
+    return visitUnaryNode(context, plan, block, node, stack);
+  }
+
+  private LogicalNode visitUnaryNode(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                     UnaryNode node, Stack<LogicalNode> stack) throws PlanningException {
+    context.depth++;
+    stack.push(node);
+    visit(context, plan, block, node.getChild(), stack);
+    context.depth--;
+    context.add(context.depth, node.getPlanString());
+    return node;
+  }
+
+  private LogicalNode visitBinaryNode(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, BinaryNode node,
+                                      Stack<LogicalNode> stack)
+      throws PlanningException {
+    context.depth++;
+    stack.push(node);
+    visit(context, plan, block, node.getLeftChild(), stack);
+    visit(context, plan, block, node.getRightChild(), stack);
+    stack.pop();
+    context.depth--;
+    context.add(context.depth, node.getPlanString());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitFilter(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, SelectionNode node,
+                                 Stack<LogicalNode> stack) throws PlanningException {
+    return visitUnaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitJoin(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, JoinNode node,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    return visitBinaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitUnion(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, UnionNode node,
+                                Stack<LogicalNode> stack) throws PlanningException {
+    return visitBinaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitExcept(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, ExceptNode node,
+                                 Stack<LogicalNode> stack) throws PlanningException {
+    return visitBinaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitIntersect(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, IntersectNode node,
+                                    Stack<LogicalNode> stack) throws PlanningException {
+    return visitBinaryNode(context, plan, block, node, stack);
+  }
+
+  @Override
+  public LogicalNode visitTableSubQuery(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                        TableSubQueryNode node, Stack<LogicalNode> stack) throws PlanningException {
+    context.depth++;
+    stack.push(node);
+    visit(context, plan, block, node.getSubQuery(), new Stack<LogicalNode>());
+    stack.pop();
+    context.depth--;
+    context.add(context.depth, node.getPlanString());
+
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitScan(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, ScanNode node,
+                               Stack<LogicalNode> stack) throws PlanningException {
+    context.add(context.depth, node.getPlanString());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitPartitionedTableScan(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                          PartitionedTableScanNode node, Stack<LogicalNode> stack)
+      throws PlanningException {
+    context.add(context.depth, node.getPlanString());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitStoreTable(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                     StoreTableNode node, Stack<LogicalNode> stack) throws PlanningException {
+    return visitUnaryNode(context, plan, block, node, stack);
+  }
+
+  public LogicalNode visitCreateDatabase(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                         CreateDatabaseNode node, Stack<LogicalNode> stack) throws PlanningException {
+    context.add(context.depth, node.getPlanString());
+    return node;
+  }
+
+  public LogicalNode visitDropDatabase(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block,
+                                         DropDatabaseNode node, Stack<LogicalNode> stack) throws PlanningException {
+    context.add(context.depth, node.getPlanString());
+    return node;
+  }
+
+  @Override
+  public LogicalNode visitInsert(Context context, LogicalPlan plan, LogicalPlan.QueryBlock block, InsertNode node,
+                                 Stack<LogicalNode> stack) throws PlanningException {
+    context.depth++;
+    stack.push(node);
+    super.visitInsert(context, plan, block, node, stack);
+    stack.pop();
+    context.depth--;
+    context.add(context.depth, node.getPlanString());
+    return node;
+  }
+
+  public static String printDepthString(int maxDepth, DepthString planStr) {
+    StringBuilder output = new StringBuilder();
+    String pad = new String(new char[planStr.getDepth() * 3]).replace('\0', ' ');
+    output.append(pad + planStr.getPlanString().getTitle()).append("\n");
+
+    for (String str : planStr.getPlanString().getExplanations()) {
+      output.append(pad).append("  => ").append(str).append("\n");
+    }
+
+    for (String str : planStr.getPlanString().getDetails()) {
+      output.append(pad).append("  => ").append(str).append("\n");
+    }
+    return output.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java
new file mode 100644
index 0000000..1b57b98
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java
@@ -0,0 +1,631 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.FunctionDesc;
+import org.apache.tajo.catalog.exception.NoSuchFunctionException;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.engine.eval.*;
+import org.apache.tajo.engine.function.AggFunction;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.exception.InternalException;
+import org.joda.time.DateTime;
+
+import java.util.Stack;
+
+/**
+ * <code>ExprAnnotator</code> makes an annotated expression called <code>EvalNode</code> from an
+ * {@link org.apache.tajo.algebra.Expr}. It visits descendants recursively from a given expression, and finally
+ * it returns an EvalNode.
+ */
+public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, EvalNode> {
+  private CatalogService catalog;
+
+  public ExprAnnotator(CatalogService catalog) {
+    this.catalog = catalog;
+  }
+
+  static class Context {
+    LogicalPlan plan;
+    LogicalPlan.QueryBlock currentBlock;
+
+    public Context(LogicalPlan plan, LogicalPlan.QueryBlock block) {
+      this.plan = plan;
+      this.currentBlock = block;
+    }
+  }
+
+  public EvalNode createEvalNode(LogicalPlan plan, LogicalPlan.QueryBlock block, Expr expr)
+      throws PlanningException {
+    Context context = new Context(plan, block);
+    return visit(context, new Stack<Expr>(), expr);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Logical Operator Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public EvalNode visitAnd(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.AND, left, right);
+  }
+
+  @Override
+  public EvalNode visitOr(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.OR, left, right);
+  }
+
+  @Override
+  public EvalNode visitNot(Context ctx, Stack<Expr> stack, NotExpr expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode child = visit(ctx, stack, expr.getChild());
+    stack.pop();
+    return new NotEval(child);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Comparison Predicates Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  @Override
+  public EvalNode visitEquals(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitCommonComparison(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitNotEquals(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitCommonComparison(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitLessThan(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitCommonComparison(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitLessThanOrEquals(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitCommonComparison(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitGreaterThan(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    return visitCommonComparison(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitGreaterThanOrEquals(Context ctx, Stack<Expr> stack, BinaryOperator expr)
+      throws PlanningException {
+    return visitCommonComparison(ctx, stack, expr);
+  }
+
+  public EvalNode visitCommonComparison(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    EvalType evalType;
+    switch (expr.getType()) {
+      case Equals:
+        evalType = EvalType.EQUAL;
+        break;
+      case NotEquals:
+        evalType = EvalType.NOT_EQUAL;
+        break;
+      case LessThan:
+        evalType = EvalType.LTH;
+        break;
+      case LessThanOrEquals:
+        evalType = EvalType.LEQ;
+        break;
+      case GreaterThan:
+        evalType = EvalType.GTH;
+        break;
+      case GreaterThanOrEquals:
+        evalType = EvalType.GEQ;
+        break;
+      default:
+      throw new IllegalStateException("Wrong Expr Type: " + expr.getType());
+    }
+
+    return new BinaryEval(evalType, left, right);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Other Predicates Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public EvalNode visitBetween(Context ctx, Stack<Expr> stack, BetweenPredicate between) throws PlanningException {
+    stack.push(between);
+    EvalNode predicand = visit(ctx, stack, between.predicand());
+    EvalNode begin = visit(ctx, stack, between.begin());
+    EvalNode end = visit(ctx, stack, between.end());
+    stack.pop();
+
+    BetweenPredicateEval betweenEval = new BetweenPredicateEval(
+        between.isNot(),
+        between.isSymmetric(),
+        predicand, begin, end);
+    return betweenEval;
+  }
+
+  @Override
+  public EvalNode visitCaseWhen(Context ctx, Stack<Expr> stack, CaseWhenPredicate caseWhen) throws PlanningException {
+    CaseWhenEval caseWhenEval = new CaseWhenEval();
+
+    EvalNode condition;
+    EvalNode result;
+    for (CaseWhenPredicate.WhenExpr when : caseWhen.getWhens()) {
+      condition = visit(ctx, stack, when.getCondition());
+      result = visit(ctx, stack, when.getResult());
+      caseWhenEval.addWhen(condition, result);
+    }
+
+    if (caseWhen.hasElseResult()) {
+      caseWhenEval.setElseResult(visit(ctx, stack, caseWhen.getElseResult()));
+    }
+
+    return caseWhenEval;
+  }
+
+  @Override
+  public EvalNode visitIsNullPredicate(Context ctx, Stack<Expr> stack, IsNullPredicate expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode child = visit(ctx, stack, expr.getPredicand());
+    stack.pop();
+    return new IsNullEval(expr.isNot(), child);
+  }
+
+  @Override
+  public EvalNode visitInPredicate(Context ctx, Stack<Expr> stack, InPredicate expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode lhs = visit(ctx, stack, expr.getLeft());
+    RowConstantEval rowConstantEval = (RowConstantEval) visit(ctx, stack, expr.getInValue());
+    stack.pop();
+    return new InEval(lhs, rowConstantEval, expr.isNot());
+  }
+
+  @Override
+  public EvalNode visitValueListExpr(Context ctx, Stack<Expr> stack, ValueListExpr expr) throws PlanningException {
+    Datum[] values = new Datum[expr.getValues().length];
+    EvalNode [] evalNodes = new EvalNode[expr.getValues().length];
+    for (int i = 0; i < expr.getValues().length; i++) {
+      evalNodes[i] = visit(ctx, stack, expr.getValues()[i]);
+      if (!EvalTreeUtil.checkIfCanBeConstant(evalNodes[i])) {
+        throw new PlanningException("Non constant values cannot be included in IN PREDICATE.");
+      }
+      values[i] = EvalTreeUtil.evaluateImmediately(evalNodes[i]);
+    }
+    return new RowConstantEval(values);
+  }
+
+  @Override
+  public EvalNode visitExistsPredicate(Context ctx, Stack<Expr> stack, ExistsPredicate expr) throws PlanningException {
+    throw new PlanningException("Cannot support EXISTS clause yet");
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // String Operator or Pattern Matching Predicates Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  @Override
+  public EvalNode visitLikePredicate(Context ctx, Stack<Expr> stack, PatternMatchPredicate expr)
+      throws PlanningException {
+    return visitPatternMatchPredicate(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitSimilarToPredicate(Context ctx, Stack<Expr> stack, PatternMatchPredicate expr)
+      throws PlanningException {
+    return visitPatternMatchPredicate(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitRegexpPredicate(Context ctx, Stack<Expr> stack, PatternMatchPredicate expr)
+      throws PlanningException {
+    return visitPatternMatchPredicate(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitConcatenate(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.CONCATENATE, left, right);
+  }
+
+  private EvalNode visitPatternMatchPredicate(Context ctx, Stack<Expr> stack, PatternMatchPredicate expr)
+      throws PlanningException {
+    EvalNode field = visit(ctx, stack, expr.getPredicand());
+    ConstEval pattern = (ConstEval) visit(ctx, stack, expr.getPattern());
+
+    // A pattern is a const value in pattern matching predicates.
+    // In a binary expression, the result is always null if a const value in left or right side is null.
+    if (pattern.getValue() instanceof NullDatum) {
+      return new ConstEval(NullDatum.get());
+    } else {
+      if (expr.getType() == OpType.LikePredicate) {
+        return new LikePredicateEval(expr.isNot(), field, pattern, expr.isCaseInsensitive());
+      } else if (expr.getType() == OpType.SimilarToPredicate) {
+        return new SimilarToPredicateEval(expr.isNot(), field, pattern);
+      } else {
+        return new RegexPredicateEval(expr.isNot(), field, pattern, expr.isCaseInsensitive());
+      }
+    }
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Arithmetic Operators
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public EvalNode visitPlus(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.PLUS, left, right);
+  }
+
+  @Override
+  public EvalNode visitMinus(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.MINUS, left, right);
+  }
+
+  @Override
+  public EvalNode visitMultiply(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.MULTIPLY, left, right);
+  }
+
+  @Override
+  public EvalNode visitDivide(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.DIVIDE, left, right);
+  }
+
+  @Override
+  public EvalNode visitModular(Context ctx, Stack<Expr> stack, BinaryOperator expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode left = visit(ctx, stack, expr.getLeft());
+    EvalNode right = visit(ctx, stack, expr.getRight());
+    stack.pop();
+
+    return new BinaryEval(EvalType.MODULAR, left, right);
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Other Expressions
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public EvalNode visitSign(Context ctx, Stack<Expr> stack, SignedExpr expr) throws PlanningException {
+    stack.push(expr);
+    EvalNode numericExpr = visit(ctx, stack, expr.getChild());
+    stack.pop();
+
+    if (expr.isNegative()) {
+      return new SignedEval(expr.isNegative(), numericExpr);
+    } else {
+      return numericExpr;
+    }
+  }
+
+  @Override
+  public EvalNode visitColumnReference(Context ctx, Stack<Expr> stack, ColumnReferenceExpr expr)
+      throws PlanningException {
+    Column column = ctx.plan.resolveColumn(ctx.currentBlock, expr);
+    return new FieldEval(column);
+  }
+
+  @Override
+  public EvalNode visitTargetExpr(Context ctx, Stack<Expr> stack, NamedExpr expr) throws PlanningException {
+    throw new PlanningException("ExprAnnotator cannot take NamedExpr");
+  }
+
+  @Override
+  public EvalNode visitFunction(Context ctx, Stack<Expr> stack, FunctionExpr expr) throws PlanningException {
+    stack.push(expr); // <--- Push
+
+    // Given parameters
+    Expr[] params = expr.getParams();
+    if (params == null) {
+      params = new Expr[0];
+    }
+
+    EvalNode[] givenArgs = new EvalNode[params.length];
+    TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[params.length];
+
+    for (int i = 0; i < params.length; i++) {
+      givenArgs[i] = visit(ctx, stack, params[i]);
+      paramTypes[i] = givenArgs[i].getValueType();
+    }
+
+    stack.pop(); // <--- Pop
+
+    if (!catalog.containFunction(expr.getSignature(), paramTypes)) {
+      throw new NoSuchFunctionException(expr.getSignature(), paramTypes);
+    }
+
+    FunctionDesc funcDesc = catalog.getFunction(expr.getSignature(), paramTypes);
+
+    try {
+    CatalogProtos.FunctionType functionType = funcDesc.getFuncType();
+    if (functionType == CatalogProtos.FunctionType.GENERAL
+        || functionType == CatalogProtos.FunctionType.UDF) {
+      return new GeneralFunctionEval(funcDesc, (GeneralFunction) funcDesc.newInstance(), givenArgs);
+    } else if (functionType == CatalogProtos.FunctionType.AGGREGATION
+        || functionType == CatalogProtos.FunctionType.UDA) {
+      if (!ctx.currentBlock.hasNode(NodeType.GROUP_BY)) {
+        ctx.currentBlock.setAggregationRequire();
+      }
+      return new AggregationFunctionCallEval(funcDesc, (AggFunction) funcDesc.newInstance(), givenArgs);
+    } else if (functionType == CatalogProtos.FunctionType.DISTINCT_AGGREGATION
+        || functionType == CatalogProtos.FunctionType.DISTINCT_UDA) {
+      throw new PlanningException("Unsupported function: " + funcDesc.toString());
+    } else {
+      throw new PlanningException("Unsupported Function Type: " + functionType.name());
+    }
+    } catch (InternalException e) {
+      throw new PlanningException(e);
+    }
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // General Set Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public EvalNode visitCountRowsFunction(Context ctx, Stack<Expr> stack, CountRowsFunctionExpr expr)
+      throws PlanningException {
+    FunctionDesc countRows = catalog.getFunction("count", CatalogProtos.FunctionType.AGGREGATION,
+        new TajoDataTypes.DataType[] {});
+    if (countRows == null) {
+      throw new NoSuchFunctionException(countRows.getSignature(), new TajoDataTypes.DataType[]{});
+    }
+
+    try {
+      ctx.currentBlock.setAggregationRequire();
+
+      return new AggregationFunctionCallEval(countRows, (AggFunction) countRows.newInstance(),
+          new EvalNode[] {});
+    } catch (InternalException e) {
+      throw new NoSuchFunctionException(countRows.getSignature(), new TajoDataTypes.DataType[]{});
+    }
+  }
+
+  @Override
+  public EvalNode visitGeneralSetFunction(Context ctx, Stack<Expr> stack, GeneralSetFunctionExpr setFunction)
+      throws PlanningException {
+
+    Expr[] params = setFunction.getParams();
+    EvalNode[] givenArgs = new EvalNode[params.length];
+    TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[params.length];
+
+    CatalogProtos.FunctionType functionType = setFunction.isDistinct() ?
+        CatalogProtos.FunctionType.DISTINCT_AGGREGATION : CatalogProtos.FunctionType.AGGREGATION;
+    givenArgs[0] = visit(ctx, stack, params[0]);
+    if (setFunction.getSignature().equalsIgnoreCase("count")) {
+      paramTypes[0] = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.ANY);
+    } else {
+      paramTypes[0] = givenArgs[0].getValueType();
+    }
+
+    if (!catalog.containFunction(setFunction.getSignature(), functionType, paramTypes)) {
+      throw new NoSuchFunctionException(setFunction.getSignature(), paramTypes);
+    }
+
+    FunctionDesc funcDesc = catalog.getFunction(setFunction.getSignature(), functionType, paramTypes);
+    if (!ctx.currentBlock.hasNode(NodeType.GROUP_BY)) {
+      ctx.currentBlock.setAggregationRequire();
+    }
+
+    try {
+      return new AggregationFunctionCallEval(funcDesc, (AggFunction) funcDesc.newInstance(), givenArgs);
+    } catch (InternalException e) {
+      throw new PlanningException(e);
+    }
+  }
+
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Literal Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public EvalNode visitDataType(Context ctx, Stack<Expr> stack, DataTypeExpr expr) throws PlanningException {
+    return super.visitDataType(ctx, stack, expr);
+  }
+
+  @Override
+  public EvalNode visitCastExpr(Context ctx, Stack<Expr> stack, CastExpr expr) throws PlanningException {
+    EvalNode child = super.visitCastExpr(ctx, stack, expr);
+
+    if (child.getType() == EvalType.CONST) { // if it is a casting operation for a constant value
+      ConstEval constEval = (ConstEval) child; // it will be pre-computed and casted to a constant value
+      return new ConstEval(DatumFactory.cast(constEval.getValue(), LogicalPlanner.convertDataType(expr.getTarget())));
+    } else {
+      return new CastEval(child, LogicalPlanner.convertDataType(expr.getTarget()));
+    }
+  }
+
+  @Override
+  public EvalNode visitLiteral(Context ctx, Stack<Expr> stack, LiteralValue expr) throws PlanningException {
+    switch (expr.getValueType()) {
+    case Boolean:
+      return new ConstEval(DatumFactory.createBool(((BooleanLiteral) expr).isTrue()));
+    case String:
+      return new ConstEval(DatumFactory.createText(expr.getValue()));
+    case Unsigned_Integer:
+      return new ConstEval(DatumFactory.createInt4(expr.getValue()));
+    case Unsigned_Large_Integer:
+      return new ConstEval(DatumFactory.createInt8(expr.getValue()));
+    case Unsigned_Float:
+      return new ConstEval(DatumFactory.createFloat8(expr.getValue()));
+    default:
+      throw new RuntimeException("Unsupported type: " + expr.getValueType());
+    }
+  }
+
+  @Override
+  public EvalNode visitNullLiteral(Context ctx, Stack<Expr> stack, NullLiteral expr) throws PlanningException {
+    return new ConstEval(NullDatum.get());
+  }
+
+  @Override
+  public EvalNode visitDateLiteral(Context context, Stack<Expr> stack, DateLiteral expr) throws PlanningException {
+    DateValue dateValue = expr.getDate();
+    int [] dates = dateToIntArray(dateValue.getYears(), dateValue.getMonths(), dateValue.getDays());
+    return new ConstEval(new DateDatum(dates[0], dates[1], dates[2]));
+  }
+
+  @Override
+  public EvalNode visitTimestampLiteral(Context ctx, Stack<Expr> stack, TimestampLiteral expr)
+      throws PlanningException {
+    DateValue dateValue = expr.getDate();
+    TimeValue timeValue = expr.getTime();
+
+    int [] dates = dateToIntArray(dateValue.getYears(),
+        dateValue.getMonths(),
+        dateValue.getDays());
+    int [] times = timeToIntArray(timeValue.getHours(),
+        timeValue.getMinutes(),
+        timeValue.getSeconds(),
+        timeValue.getSecondsFraction());
+    DateTime dateTime;
+    if (timeValue.hasSecondsFraction()) {
+      dateTime = new DateTime(dates[0], dates[1], dates[2], times[0], times[1], times[2], times[3]);
+    } else {
+      dateTime = new DateTime(dates[0], dates[1], dates[2], times[0], times[1], times[2]);
+    }
+
+    return new ConstEval(new TimestampDatum(dateTime));
+  }
+
+  @Override
+  public EvalNode visitTimeLiteral(Context ctx, Stack<Expr> stack, TimeLiteral expr) throws PlanningException {
+    TimeValue timeValue = expr.getTime();
+    int [] times = timeToIntArray(timeValue.getHours(),
+        timeValue.getMinutes(),
+        timeValue.getSeconds(),
+        timeValue.getSecondsFraction());
+
+    TimeDatum datum;
+    if (timeValue.hasSecondsFraction()) {
+      datum = new TimeDatum(times[0], times[1], times[2], times[3]);
+    } else {
+      datum = new TimeDatum(times[0], times[1], times[2]);
+    }
+    return new ConstEval(datum);
+  }
+
+  public static int [] dateToIntArray(String years, String months, String days)
+      throws PlanningException {
+    int year = Integer.valueOf(years);
+    int month = Integer.valueOf(months);
+    int day = Integer.valueOf(days);
+
+    if (!(1 <= year && year <= 9999)) {
+      throw new PlanningException(String.format("Years (%d) must be between 1 and 9999 integer value", year));
+    }
+
+    if (!(1 <= month && month <= 12)) {
+      throw new PlanningException(String.format("Months (%d) must be between 1 and 12 integer value", month));
+    }
+
+    if (!(1<= day && day <= 31)) {
+      throw new PlanningException(String.format("Days (%d) must be between 1 and 31 integer value", day));
+    }
+
+    int [] results = new int[3];
+    results[0] = year;
+    results[1] = month;
+    results[2] = day;
+
+    return results;
+  }
+
+  public static int [] timeToIntArray(String hours, String minutes, String seconds, String fractionOfSecond)
+      throws PlanningException {
+    int hour = Integer.valueOf(hours);
+    int minute = Integer.valueOf(minutes);
+    int second = Integer.valueOf(seconds);
+    int fraction = 0;
+    if (fractionOfSecond != null) {
+      fraction = Integer.valueOf(fractionOfSecond);
+    }
+
+    if (!(0 <= hour && hour <= 23)) {
+      throw new PlanningException(String.format("Hours (%d) must be between 0 and 24 integer value", hour));
+    }
+
+    if (!(0 <= minute && minute <= 59)) {
+      throw new PlanningException(String.format("Minutes (%d) must be between 0 and 59 integer value", minute));
+    }
+
+    if (!(0 <= second && second <= 59)) {
+      throw new PlanningException(String.format("Seconds (%d) must be between 0 and 59 integer value", second));
+    }
+
+    if (fraction != 0) {
+      if (!(0 <= fraction && fraction <= 999)) {
+        throw new PlanningException(String.format("Seconds (%d) must be between 0 and 999 integer value", fraction));
+      }
+    }
+
+    int [] results = new int[4];
+    results[0] = hour;
+    results[1] = minute;
+    results[2] = second;
+    results[3] = fraction;
+
+    return results;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprFinder.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprFinder.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprFinder.java
new file mode 100644
index 0000000..89eed91
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprFinder.java
@@ -0,0 +1,74 @@
+/**
+ * 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.planner;
+
+import org.apache.tajo.algebra.BinaryOperator;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.algebra.OpType;
+import org.apache.tajo.algebra.UnaryOperator;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.Stack;
+
+class ExprFinder extends SimpleAlgebraVisitor<ExprFinder.Context, Object> {
+
+  static class Context {
+    Set<Expr> set = new HashSet<Expr>();
+    OpType targetType;
+
+    Context(OpType type) {
+      this.targetType = type;
+    }
+  }
+
+  public static <T extends Expr> Set<T> finds(Expr expr, OpType type) {
+    Context context = new Context(type);
+    ExprFinder finder = new ExprFinder();
+    Stack<Expr> stack = new Stack<Expr>();
+    stack.push(expr);
+    try {
+      finder.visit(context, new Stack<Expr>(), expr);
+    } catch (PlanningException e) {
+      throw new RuntimeException(e);
+    }
+    stack.pop();
+    return (Set<T>) context.set;
+  }
+
+  public Object visit(Context ctx, Stack<Expr> stack, Expr expr) throws PlanningException {
+    if (expr instanceof UnaryOperator) {
+      preHook(ctx, stack, expr);
+      visitUnaryOperator(ctx, stack, (UnaryOperator) expr);
+      postHook(ctx, stack, expr, null);
+    } else if (expr instanceof BinaryOperator) {
+      preHook(ctx, stack, expr);
+      visitBinaryOperator(ctx, stack, (BinaryOperator) expr);
+      postHook(ctx, stack, expr, null);
+    } else {
+      super.visit(ctx, stack, expr);
+    }
+
+    if (ctx.targetType == expr.getType()) {
+      ctx.set.add(expr);
+    }
+
+    return null;
+  }
+}


[50/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ffe96cc..1bc79c9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -649,6 +649,9 @@ Release 0.8.0 - unreleased
 
   TASKS
 
+    TAJO-752: Escalate sub modules in tajo-core into the top-level modules.
+    (hyunsik)
+
     TAJO-753: Clean up of maven dependencies. (jinho)
 
     TAJO-730: Update Tajo site to reflect graduation. (hyunsik)

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bd3358a..964d984 100644
--- a/pom.xml
+++ b/pom.xml
@@ -88,6 +88,7 @@
     <module>tajo-jdbc</module>
     <module>tajo-dist</module>
     <module>tajo-storage</module>
+    <module>tajo-yarn-pullserver</module>
   </modules>
 
   <build>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-client/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-client/pom.xml b/tajo-client/pom.xml
index 6a14382..5662678 100644
--- a/tajo-client/pom.xml
+++ b/tajo-client/pom.xml
@@ -114,7 +114,7 @@
                 <argument>-Isrc/main/proto/</argument>
                 <argument>--proto_path=../tajo-common/src/main/proto</argument>
                 <argument>--proto_path=../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
-                <argument>--proto_path=../tajo-core/tajo-core-backend/src/main/proto</argument>
+                <argument>--proto_path=../tajo-core/src/main/proto</argument>
                 <argument>--java_out=target/generated-sources/proto</argument>
                 <argument>src/main/proto/ClientProtos.proto</argument>
                 <argument>src/main/proto/TajoMasterClientProtocol.proto</argument>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
index 05a921e..52711c9 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
@@ -110,14 +110,15 @@ public class TajoDump {
     }
 
     PrintWriter writer = new PrintWriter(System.out);
-    dump(client, userInfo, baseDatabaseName, isDumpingAllDatabases, writer);
+    dump(client, userInfo, baseDatabaseName, isDumpingAllDatabases, true, writer);
 
     System.exit(0);
   }
 
   public static void dump(TajoClient client, UserGroupInformation userInfo, String baseDatabaseName,
-                   boolean isDumpingAllDatabases, PrintWriter out) throws SQLException, ServiceException {
-    printHeader(out, userInfo);
+                   boolean isDumpingAllDatabases, boolean includeDate, PrintWriter out)
+      throws SQLException, ServiceException {
+    printHeader(out, userInfo, includeDate);
 
     if (isDumpingAllDatabases) {
       // sort database names in an ascending lexicographic order of the names.
@@ -133,12 +134,14 @@ public class TajoDump {
     out.flush();
   }
 
-  private static void printHeader(PrintWriter writer, UserGroupInformation userInfo) {
+  private static void printHeader(PrintWriter writer, UserGroupInformation userInfo, boolean includeDate) {
     writer.write("--\n");
     writer.write("-- Tajo database dump\n");
     writer.write("--\n");
     writer.write("-- Dump user: " + userInfo.getUserName() + "\n");
-    writer.write("-- Dump date: " + toDateString() + "\n");
+    if (includeDate) {
+      writer.write("-- Dump date: " + toDateString() + "\n");
+    }
     writer.write("--\n");
     writer.write("\n");
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
index 6bbce04..9aa6af9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.IOUtils;
 
 import java.io.*;
+import java.net.URL;
 import java.nio.charset.Charset;
 
 public class FileUtil {
@@ -81,6 +82,10 @@ public class FileUtil {
     return new File(path);
   }
 
+  public static URL getResourcePath(String resource) throws IOException {
+    return ClassLoader.getSystemResource(resource);
+  }
+
   public static String readTextFileFromResource(String resource) throws IOException {
     StringBuilder fileData = new StringBuilder(1000);
     InputStream inputStream = ClassLoader.getSystemResourceAsStream(resource);

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/simple/groupby1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/simple/groupby1.sql b/tajo-core/benchmark/simple/groupby1.sql
new file mode 100644
index 0000000..cb6ff09
--- /dev/null
+++ b/tajo-core/benchmark/simple/groupby1.sql
@@ -0,0 +1,6 @@
+select
+	l_orderkey, sum(l_quantity) as sum_qty, max(l_quantity) as max_qty, min(l_quantity) as min_qty
+from
+	lineitem
+group by
+    l_orderkey
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/simple/groupby2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/simple/groupby2.sql b/tajo-core/benchmark/simple/groupby2.sql
new file mode 100644
index 0000000..7178e20
--- /dev/null
+++ b/tajo-core/benchmark/simple/groupby2.sql
@@ -0,0 +1,6 @@
+select
+	l_orderkey, l_linenumber, sum(l_quantity) as sum_qty, max(l_quantity) as max_qty, min(l_quantity) as min_qty
+from
+	lineitem
+group by
+    l_orderkey, l_linenumber
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/simple/selection1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/simple/selection1.sql b/tajo-core/benchmark/simple/selection1.sql
new file mode 100644
index 0000000..e5fd404
--- /dev/null
+++ b/tajo-core/benchmark/simple/selection1.sql
@@ -0,0 +1,5 @@
+select
+	*
+from
+	lineitem
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/simple/selection2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/simple/selection2.sql b/tajo-core/benchmark/simple/selection2.sql
new file mode 100644
index 0000000..0c120d8
--- /dev/null
+++ b/tajo-core/benchmark/simple/selection2.sql
@@ -0,0 +1,5 @@
+select
+	l_orderkey, l_linenumber
+from
+	lineitem
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/simple/selection3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/simple/selection3.sql b/tajo-core/benchmark/simple/selection3.sql
new file mode 100644
index 0000000..f415c4e
--- /dev/null
+++ b/tajo-core/benchmark/simple/selection3.sql
@@ -0,0 +1,7 @@
+select
+	l_orderkey, l_linenumber
+from
+	lineitem
+where
+    l_shipdate <= '1998-12-01'
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/customer.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/customer.schema b/tajo-core/benchmark/tpch/customer.schema
new file mode 100644
index 0000000..9df7c38
--- /dev/null
+++ b/tajo-core/benchmark/tpch/customer.schema
@@ -0,0 +1,9 @@
+create table customer (
+	c_custkey long,
+	c_name string,
+	c_nationkey long,
+	c_phone string,
+	c_acctbal double,
+	c_mktsegment string,
+	c_comment string
+) 

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/lineitem.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/lineitem.schema b/tajo-core/benchmark/tpch/lineitem.schema
new file mode 100644
index 0000000..c90c11e
--- /dev/null
+++ b/tajo-core/benchmark/tpch/lineitem.schema
@@ -0,0 +1,19 @@
+create table lineitem (
+	l_orderkey long,
+	l_partkey long,
+	l_suppkey long,
+	l_linenumber int,
+	l_quantity double,
+	l_extendedprice double,
+	l_discount double,
+	l_tax double,
+	l_returnflag string,
+	l_linestatus string,
+	l_shipdate string,
+	l_commitdate string,
+	l_receiptdate string,
+	l_shipinstruct string,
+	l_shipmode string,
+	l_comment string
+	/* primary key (l_orderkey, l_linenumber) */
+) 

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/nation.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/nation.schema b/tajo-core/benchmark/tpch/nation.schema
new file mode 100644
index 0000000..7dd9623
--- /dev/null
+++ b/tajo-core/benchmark/tpch/nation.schema
@@ -0,0 +1,6 @@
+create table nation (
+	n_nationkey long /* primary key */,
+	n_name string,
+	n_regionkey long,
+	n_comment string
+) 

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/orders.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/orders.schema b/tajo-core/benchmark/tpch/orders.schema
new file mode 100644
index 0000000..62cb45b
--- /dev/null
+++ b/tajo-core/benchmark/tpch/orders.schema
@@ -0,0 +1,11 @@
+create table orders (
+	o_orderkey long, /* primary key */
+	o_custkey long,
+	o_orderstatus string,
+	o_totalprice double,
+	o_orderdate string,
+	o_orderpriority string,
+	o_clerk string,
+	o_shippriority int,
+	o_comment string
+) 

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/part.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/part.schema b/tajo-core/benchmark/tpch/part.schema
new file mode 100644
index 0000000..dad053b
--- /dev/null
+++ b/tajo-core/benchmark/tpch/part.schema
@@ -0,0 +1,11 @@
+create table part (
+	p_partkey long /* primary key */, 
+	p_name string, 
+	p_mfgr string,
+	p_brand string,
+	p_type string,
+	p_size int,
+	p_container string,
+	p_retailprice double,
+	p_comment string
+)

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/partsupp.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/partsupp.schema b/tajo-core/benchmark/tpch/partsupp.schema
new file mode 100644
index 0000000..4732392
--- /dev/null
+++ b/tajo-core/benchmark/tpch/partsupp.schema
@@ -0,0 +1,8 @@
+create table partsupp (
+	ps_partkey long,
+	ps_suppkey long,
+	ps_availqty int,
+	ps_supplycost double,
+	ps_comment string
+	/* primary key (ps_partkey, ps_suppkey) */
+)

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q1.sql b/tajo-core/benchmark/tpch/q1.sql
new file mode 100644
index 0000000..3c1f6bb
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q1.sql
@@ -0,0 +1,21 @@
+select 
+	l_returnflag,
+	l_linestatus,
+	sum(l_quantity) as sum_qty,
+	sum(l_extendedprice) as sum_base_price,
+	sum(l_extendedprice*(1-l_discount)) as sum_disc_price,
+	sum(l_extendedprice*(1-l_discount)*(1+l_tax)) as sum_charge,
+	avg(l_quantity) as avg_qty,
+	avg(l_extendedprice) as avg_price,
+	avg(l_discount) as avg_disc,
+	count(*) as count_order
+from
+	lineitem
+where
+	l_shipdate <= '1998-09-01'
+group by
+	l_returnflag,
+	l_linestatus
+order by
+	l_returnflag,
+	l_linestatus
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q10.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q10.sql b/tajo-core/benchmark/tpch/q10.sql
new file mode 100644
index 0000000..73111c3
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q10.sql
@@ -0,0 +1,31 @@
+select
+	c_custkey,
+	c_name,
+	sum(l_extendedprice * (1 - l_discount)) as revenue,
+	c_acctbal,
+	n_name,
+	c_address,
+	c_phone,
+	c_comment
+from
+	customer,
+	orders,
+	lineitem,
+	nation
+where
+	c_custkey = o_custkey and
+	l_orderkey = o_orderkey and
+	o_orderdate >= '1993-10-01' and
+	o_orderdate < '1994-01-01' and
+	l_returnflag = 'R' and
+	c_nationkey = n_nationkey
+group by
+	c_custkey,
+	c_name,
+	c_acctbal,
+	c_phone,
+	n_name,
+	c_address,
+	c_comment
+order by
+	revenue desc
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q11.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q11.sql b/tajo-core/benchmark/tpch/q11.sql
new file mode 100644
index 0000000..a36cb66
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q11.sql
@@ -0,0 +1,27 @@
+select
+	ps_partkey,
+	sum(ps_supplycost * ps_availqty) as value
+from
+	partsupp,
+	supplier,
+	nation
+where
+	ps_suppkey = s_suppkey
+	and s_nationkey = n_nationkey
+	and n_name = 'GERMANY'
+group by
+	ps_partkey having
+	sum(ps_supplycost * ps_availqty) > (
+		select
+		sum(ps_supplycost * ps_availqty) * 0.0001
+		from
+		partsupp,
+		supplier,
+		nation
+		where
+		ps_suppkey = s_suppkey
+		and s_nationkey = n_nationkey
+		and n_name = '[NATION]'
+	)
+order by
+	value desc
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q12.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q12.sql b/tajo-core/benchmark/tpch/q12.sql
new file mode 100644
index 0000000..2f26ee9
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q12.sql
@@ -0,0 +1,24 @@
+select
+	l_shipmode,
+	sum(case when o_orderpriority ='1-URGENT' or o_orderpriority ='2-HIGH'
+	  then 1
+	  else 0
+	end) as high_line_count,
+	sum(case when o_orderpriority != '1-URGENT' and o_orderpriority != '2-HIGH'
+	  then 1
+	  else 0
+	end) as low_line_count
+from
+	orders,
+	lineitem
+where
+	o_orderkey = l_orderkey and
+	(l_shipmode = 'MAIL' or l_shipmode = 'SHIP') and
+	l_commitdate < l_receiptdate and
+	l_shipdate < l_commitdate and
+	l_receiptdate >= '1994-01-01' and
+	l_receiptdate < '1995-01-01'
+group by
+	l_shipmode
+order by
+	l_shipmode
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q13.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q13.sql b/tajo-core/benchmark/tpch/q13.sql
new file mode 100644
index 0000000..2ee0102
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q13.sql
@@ -0,0 +1,18 @@
+select
+  c_count, count(*) as custdist
+from (
+  select
+    c_custkey,
+    count(o_orderkey)
+  from
+    customer left outer join orders on
+    c_custkey = o_custkey
+    and o_comment not like '%special%requests%'
+  group by
+    c_custkey
+  ) as c_orders (c_custkey, c_count)
+group by
+  c_count
+order by
+  custdist desc,
+  c_count desc
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q14.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q14.sql b/tajo-core/benchmark/tpch/q14.sql
new file mode 100644
index 0000000..f00e469
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q14.sql
@@ -0,0 +1,13 @@
+select
+	100.00 * sum(case
+	  when p_type like 'PROMO%'
+	  then l_extendedprice*(1-l_discount)
+	  else 0
+	end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue
+from
+	lineitem,
+	part
+where
+	l_partkey = p_partkey and
+	l_shipdate >= '1995-09-01' and
+	l_shipdate < '1995-10-01';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q15.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q15.sql b/tajo-core/benchmark/tpch/q15.sql
new file mode 100644
index 0000000..fc95929
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q15.sql
@@ -0,0 +1,32 @@
+create view revenue[STREAM_ID] (supplier_no, total_revenue) as
+	select
+		l_suppkey,
+		sum(l_extendedprice * (1 - l_discount))
+	from
+		lineitem
+	where
+		l_shipdate >= date '1996-01-01'
+		and l_shipdate < date '1996-01-01' + interval '3' month
+	group by
+		l_suppkey;
+
+select
+	s_suppkey,
+	s_name,
+	s_address,
+	s_phone,
+	total_revenue
+from
+	supplier,
+	revenue[STREAM_ID]
+where
+	s_suppkey = supplier_no
+	and total_revenue = (
+		select
+		max(total_revenue)
+		from
+		revenue[STREAM_ID]
+	)
+order by
+	s_suppkey;
+	drop view revenue[STREAM_ID];
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q16.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q16.sql b/tajo-core/benchmark/tpch/q16.sql
new file mode 100644
index 0000000..18713e4
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q16.sql
@@ -0,0 +1,30 @@
+select
+	p_brand,
+	p_type,
+	p_size,
+	count(distinct ps_suppkey) as supplier_cnt
+from
+	partsupp,
+	part
+where
+	p_partkey = ps_partkey
+	and p_brand <> 'Brand#45'
+	and p_type not like 'MEDIUM POLISHED%'
+	and p_size in (49, 14, 23, 45, 19, 3, 36, 9)
+	and ps_suppkey not in (
+		select
+		s_suppkey
+		from
+		supplier
+		where
+		s_comment like '%Customer%Complaints%'
+	)
+group by
+	p_brand,
+	p_type,
+	p_size
+order by
+	supplier_cnt desc,
+	p_brand,
+	p_type,
+	p_size;

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q17.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q17.sql b/tajo-core/benchmark/tpch/q17.sql
new file mode 100644
index 0000000..d67a399
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q17.sql
@@ -0,0 +1,17 @@
+select
+	sum(l_extendedprice) / 7.0 as avg_yearly
+from
+	lineitem,
+	part
+where
+	p_partkey = l_partkey
+	and p_brand = 'Brand#23'
+	and p_container = 'MED BOX'
+	and l_quantity < (
+		select
+			0.2 * avg(l_quantity)
+		from
+			lineitem
+		where
+			l_partkey = p_partkey
+	)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q18.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q18.sql b/tajo-core/benchmark/tpch/q18.sql
new file mode 100644
index 0000000..43b2787
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q18.sql
@@ -0,0 +1,33 @@
+select
+	c_name,
+	c_custkey,
+	o_orderkey,
+	o_orderdate,
+	o_totalprice,
+	sum(l_quantity)
+from
+	customer,
+	orders,
+	lineitem
+where
+	o_orderkey in (
+		select
+			l_orderkey
+		from
+			lineitem
+		group by
+			l_orderkey 
+		having
+			sum(l_quantity) > [QUANTITY]
+	)
+	and c_custkey = o_custkey
+	and o_orderkey = l_orderkey
+group by
+	c_name,
+	c_custkey,
+	o_orderkey,
+	o_orderdate,
+	o_totalprice
+order by
+	o_totalprice desc,
+	o_orderdate

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q19.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q19.sql b/tajo-core/benchmark/tpch/q19.sql
new file mode 100644
index 0000000..7760e46
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q19.sql
@@ -0,0 +1,35 @@
+select
+	sum(l_extendedprice * (1 - l_discount) ) as revenue
+from
+	lineitem,
+	part
+where
+	(
+		p_partkey = l_partkey
+		and p_brand = 'Brand#12'
+		and p_container in ( 'SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')
+		and l_quantity >= 1 and l_quantity <= 1 + 10
+		and p_size between 1 and 5
+		and l_shipmode in ('AIR', 'AIR REG')
+		and l_shipinstruct = 'DELIVER IN PERSON'
+	)
+or
+	(
+		p_partkey = l_partkey
+		and p_brand = 'Brand#23'
+		and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')
+		and l_quantity >= 10 and l_quantity <= 10 + 10
+		and p_size between 1 and 10
+		and l_shipmode in ('AIR', 'AIR REG')
+		and l_shipinstruct = 'DELIVER IN PERSON'
+	)
+or
+	(
+		p_partkey = l_partkey
+		and p_brand = 'Brand#34'
+		and p_container in ( 'LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')
+		and l_quantity >= 20 and l_quantity <= 20 + 10
+		and p_size between 1 and 15
+		and l_shipmode in ('AIR', 'AIR REG')
+		and l_shipinstruct = 'DELIVER IN PERSON'
+	)

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q2.sql b/tajo-core/benchmark/tpch/q2.sql
new file mode 100644
index 0000000..b7a7cd8
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q2.sql
@@ -0,0 +1,38 @@
+select
+  s_acctbal,
+  s_name,
+  n_name,
+  p_partkey,
+  p_mfgr,
+  s_address,
+  s_phone,
+  s_comment
+from
+  part,
+  supplier,
+  partsupp,
+  nation,
+  region
+where
+  p_partkey = ps_partkey
+  and s_suppkey = ps_suppkey
+  and p_size = 15
+  and p_type like '%BRASS'
+  and s_nationkey = n_nationkey
+  and n_regionkey = r_regionkey
+  and r_name = 'c'
+  and ps_supplycost =
+    (
+      select min(ps_supplycost) from partsupp, supplier, nation, region
+      where 
+	      p_partkey = ps_partkey
+	      and s_suppkey = ps_suppkey
+	      and s_nationkey = n_nationkey
+	      and n_regionkey = r_regionkey
+	      and r_name = 'EUROPE'
+    )
+order by 
+  s_acctbal desc, 
+  n_name, 
+  s_name, 
+  p_partkey
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q20.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q20.sql b/tajo-core/benchmark/tpch/q20.sql
new file mode 100644
index 0000000..11f9919
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q20.sql
@@ -0,0 +1,36 @@
+select
+	s_name,
+	s_address
+from
+	supplier, nation
+	where
+	s_suppkey in (
+		select
+		ps_suppkey
+		from
+		partsupp
+		where
+		ps_partkey in (
+			select
+				p_partkey
+			from
+				part
+			where
+				p_name like 'forest%'
+		)
+		and ps_availqty > (
+			select
+				0.5 * sum(l_quantity)
+			from
+				lineitem
+			where
+				l_partkey = ps_partkey
+				and l_suppkey = ps_suppkey
+				and l_shipdate >= date('1994-01-01')
+				and l_shipdate < date('1994-01-01') + interval '1' year
+		)
+	)
+	and s_nationkey = n_nationkey
+	and n_name = 'CANADA'
+order by
+	s_name

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q21.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q21.sql b/tajo-core/benchmark/tpch/q21.sql
new file mode 100644
index 0000000..66cb0d4
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q21.sql
@@ -0,0 +1,39 @@
+select
+	s_name,
+	count(*) as numwait
+from
+	supplier,
+	lineitem l1,
+	orders,
+	nation
+where
+	s_suppkey = l1.l_suppkey
+	and o_orderkey = l1.l_orderkey
+	and o_orderstatus = 'F'
+	and l1.l_receiptdate > l1.l_commitdate
+	and exists (
+		select
+			*
+		from
+			lineitem l2
+		where
+			l2.l_orderkey = l1.l_orderkey
+			and l2.l_suppkey <> l1.l_suppkey
+	)
+	and not exists (
+		select
+			*
+		from
+			lineitem l3
+		where
+			l3.l_orderkey = l1.l_orderkey
+			and l3.l_suppkey <> l1.l_suppkey
+			and l3.l_receiptdate > l3.l_commitdate
+	)
+	and s_nationkey = n_nationkey
+	and n_name = 'SAUDI ARABIA'
+group by
+	s_name
+order by
+	numwait desc,
+	s_name

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q22.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q22.sql b/tajo-core/benchmark/tpch/q22.sql
new file mode 100644
index 0000000..d315e48
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q22.sql
@@ -0,0 +1,36 @@
+select
+	cntrycode,
+	count(*) as numcust,
+	sum(c_acctbal) as totacctbal
+from (
+	select
+		substring(c_phone from 1 for 2) as cntrycode,
+		c_acctbal
+	from
+		customer
+	where
+		substring(c_phone from 1 for 2) in
+		('13','31','23','29','30','18','17')
+		and c_acctbal > (
+			select
+				avg(c_acctbal)
+			from
+				customer
+			where
+				c_acctbal > 0.00
+				and substring (c_phone from 1 for 2) in
+				('13','31','23','29','30','18','17')
+		)
+		and not exists (
+			select
+				*
+			from
+				orders
+			where
+				o_custkey = c_custkey
+		)
+	) as custsale
+group by
+	cntrycode
+order by
+	cntrycode

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q3.sql b/tajo-core/benchmark/tpch/q3.sql
new file mode 100644
index 0000000..60c8fdb
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q3.sql
@@ -0,0 +1,22 @@
+select 
+	l_orderkey,
+	sum(l_extendedprice*(1-l_discount)) as revenue,
+	o_orderdate,
+	o_shippriority
+from
+	customer,
+	orders,
+	lineitem
+where
+	c_mktsegment = 'BUILDING' and
+	c_custkey = o_custkey and
+	l_orderkey = o_orderkey and
+	o_orderdate < '1995-03-15' and
+	l_shipdate > '1995-03-15'
+group by
+	l_orderkey,
+	o_orderdate,
+	o_shippriority
+order by
+	revenue desc,
+	o_orderdate
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q4.sql b/tajo-core/benchmark/tpch/q4.sql
new file mode 100644
index 0000000..2ab4598
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q4.sql
@@ -0,0 +1,21 @@
+select
+  o_orderpriority,
+  count(*) as order_count
+from 
+  orders
+where 
+  o_orderdate >= '1993-07-01'
+  and o_orderdate < '1993-10-01'
+  and exists (
+    select
+      *
+    from
+      lineitem
+    where
+      l_orderkey = o_orderkey
+      and l_commitdate < l_receiptdate
+  )
+group by 
+  o_orderpriority
+order by 
+  o_orderpriority;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q5.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q5.sql b/tajo-core/benchmark/tpch/q5.sql
new file mode 100644
index 0000000..e7d3d56
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q5.sql
@@ -0,0 +1,24 @@
+select
+	n_name,
+	sum(l_extendedprice * (1 - l_discount)) as revenue
+from
+	customer,
+	orders,
+	lineitem,
+	supplier,
+	nation,
+	region
+where
+	c_custkey = o_custkey and
+	l_orderkey = o_orderkey and
+	l_suppkey = s_suppkey and
+	c_nationkey = s_nationkey and
+	s_nationkey = n_nationkey and
+	n_regionkey = r_regionkey and
+	r_name = 'ASIA' and
+	o_orderdate >= '1994-01-01' and
+	o_orderdate < '1995-01-01'
+group by
+	n_name
+order by
+	revenue desc
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q6.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q6.sql b/tajo-core/benchmark/tpch/q6.sql
new file mode 100644
index 0000000..6427ca7
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q6.sql
@@ -0,0 +1,10 @@
+select
+	sum(l_extendedprice*l_discount) as revenue
+from
+	lineitem
+where
+	l_shipdate >= '1994-01-01' and
+	l_shipdate < '1995-01-01' and
+	l_discount >= 0.05 and
+	l_discount <= 0.07 and
+	l_quantity < 24
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q7.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q7.sql b/tajo-core/benchmark/tpch/q7.sql
new file mode 100644
index 0000000..a7d0f79
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q7.sql
@@ -0,0 +1,38 @@
+select
+  supp_nation,
+  cust_nation,
+  l_year,
+  sum(volume) as revenue
+from (
+  select
+    n1.n_name as supp_nation,
+    n2.n_name as cust_nation,
+    l_shipdate as l_year,
+    l_extendedprice * (1 - l_discount) as volume
+  from
+    supplier,
+    lineitem,
+    orders,
+    customer,
+    nation n1,
+    nation n2
+  where
+    s_suppkey = l_suppkey
+    and o_orderkey = l_orderkey
+    and c_custkey = o_custkey
+    and s_nationkey = n1.n_nationkey
+    and c_nationkey = n2.n_nationkey
+    and (
+      (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY')
+      or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE')
+    )
+    and '1995-01-01' < l_shipdate  and l_shipdate < '1996-12-31'
+  ) as shipping
+group by 
+  supp_nation,
+  cust_nation,
+  l_year
+order by 
+  supp_nation,
+  cust_nation,
+  l_year;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q8.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q8.sql b/tajo-core/benchmark/tpch/q8.sql
new file mode 100644
index 0000000..9e96d4d
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q8.sql
@@ -0,0 +1,37 @@
+select
+  o_year,
+  sum(case
+  when nation = 'BRAZIL'
+  then volume
+  else 0
+  end) / sum(volume) as mkt_share
+from (
+  select
+    o_orderdate as o_year,
+    l_extendedprice * (1-l_discount) as volume,
+    n2.n_name as nation
+  from
+    part,
+    supplier,
+    lineitem,
+    orders,
+    customer,
+    nation n1,
+    nation n2,
+  region
+  where
+    p_partkey = l_partkey
+    and s_suppkey = l_suppkey
+    and l_orderkey = o_orderkey
+    and o_custkey = c_custkey
+    and c_nationkey = n1.n_nationkey
+    and n1.n_regionkey = r_regionkey
+    and r_name = 'AMERICA'
+    and s_nationkey = n2.n_nationkey
+    and '1995-01-01' < o_orderdate  and o_orderdate <'1996-12-31'
+    and p_type = 'ECONOMY ANODIZED STEEL'
+  ) as all_nations
+group by 
+  o_year
+order by 
+  o_year;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/q9.sql
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/q9.sql b/tajo-core/benchmark/tpch/q9.sql
new file mode 100644
index 0000000..06c69ed
--- /dev/null
+++ b/tajo-core/benchmark/tpch/q9.sql
@@ -0,0 +1,31 @@
+select 
+    nation,
+    o_year,
+    sum(amount) as sum_profit
+  from (
+    select
+      n_name as nation,
+      o_orderdate as o_year,
+      l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount
+    from
+      part,
+      supplier,
+      lineitem,
+      partsupp,
+      orders,
+      nation
+    where
+      s_suppkey = l_suppkey
+      and ps_suppkey = l_suppkey
+      and ps_partkey = l_partkey
+      and p_partkey = l_partkey
+      and o_orderkey = l_orderkey
+      and s_nationkey = n_nationkey
+      and p_name like 'green'
+  ) as profit
+group by 
+  nation,
+  o_year
+order by 
+  nation,
+  o_year desc;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/region.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/region.schema b/tajo-core/benchmark/tpch/region.schema
new file mode 100644
index 0000000..e9a0057
--- /dev/null
+++ b/tajo-core/benchmark/tpch/region.schema
@@ -0,0 +1,5 @@
+create table region (
+	r_regionkey long /* primary key */,
+	r_name string,
+	r_comment string
+) 

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/benchmark/tpch/supplier.schema
----------------------------------------------------------------------
diff --git a/tajo-core/benchmark/tpch/supplier.schema b/tajo-core/benchmark/tpch/supplier.schema
new file mode 100644
index 0000000..4fca98b
--- /dev/null
+++ b/tajo-core/benchmark/tpch/supplier.schema
@@ -0,0 +1,9 @@
+create table supplier (
+	s_suppkey long /* primary key */,
+	s_name string,
+	s_address string,
+	s_nationkey long,
+	s_phone string,
+	s_acctbal double,
+	s_comment string
+)

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml
index 709390d..2f38e92 100644
--- a/tajo-core/pom.xml
+++ b/tajo-core/pom.xml
@@ -1,8 +1,12 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <!--
-  Licensed 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
+  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
 
@@ -14,7 +18,7 @@
   -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <artifactId>tajo-project</artifactId>
@@ -23,19 +27,27 @@
     <relativePath>../tajo-project</relativePath>
   </parent>
   <artifactId>tajo-core</artifactId>
-  <packaging>pom</packaging>
+  <packaging>jar</packaging>
   <name>Tajo Core</name>
   <version>0.8.0-SNAPSHOT</version>
-
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    <antlr4.visitor>true</antlr4.visitor>
+    <antlr4.listener>true</antlr4.listener>
+    <metrics.version>3.0.1</metrics.version>
   </properties>
 
-  <modules>
-    <module>tajo-core-backend</module>
-	  <module>tajo-core-pullserver</module>
-  </modules>
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
 
   <build>
     <plugins>
@@ -49,6 +61,88 @@
         </configuration>
       </plugin>
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.12.4</version>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.2</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr4-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>antlr4</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--proto_path=../tajo-client/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/ResourceTrackerProtocol.proto</argument>
+                <argument>src/main/proto/QueryMasterProtocol.proto</argument>
+                <argument>src/main/proto/TajoMasterProtocol.proto</argument>
+                <argument>src/main/proto/TajoWorkerProtocol.proto</argument>
+                <argument>src/main/proto/InternalTypes.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>build-helper-maven-plugin</artifactId>
         <version>1.5</version>
@@ -69,122 +163,517 @@
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-report-plugin</artifactId>
-        <version>2.15</version>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-dependencies</id>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeScope>runtime</includeScope>
+              <outputDirectory>${project.build.directory}/lib</outputDirectory>
+              <overWriteReleases>false</overWriteReleases>
+              <overWriteSnapshots>false</overWriteSnapshots>
+              <overWriteIfNewer>true</overWriteIfNewer>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-pmd-plugin</artifactId>
+        <version>2.7.1</version>
       </plugin>
     </plugins>
   </build>
 
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-algebra</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-yarn-pullserver</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-jdbc</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-rpc</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-tests</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>antlr4</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.dspace.dependencies</groupId>
+      <artifactId>dspace-geoip</artifactId>
+      <version>1.2.3</version>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jdt</groupId>
+      <artifactId>core</artifactId>
+      <version>3.1.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
+      <version>6.1.14</version>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jsp-2.1</artifactId>
+      <version>6.1.14</version>
+    </dependency>
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+      <version>${metrics.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-jvm</artifactId>
+      <version>${metrics.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>info.ganglia.gmetric4j</groupId>
+      <artifactId>gmetric4j</artifactId>
+      <version>1.0.3</version>
+    </dependency>
+  </dependencies>
+
   <profiles>
     <profile>
-      <id>src</id>
+      <id>docs</id>
       <activation>
         <activeByDefault>false</activeByDefault>
       </activation>
       <build>
         <plugins>
           <plugin>
-            <artifactId>maven-source-plugin</artifactId>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
             <executions>
               <execution>
-                <id>attach-sources</id>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
                 <goals>
-                  <!-- avoid warning about recursion -->
-                  <goal>jar-no-fork</goal>
+                  <goal>jar</goal>
                 </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
               </execution>
             </executions>
           </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-report-plugin</artifactId>
+            <version>2.15</version>
+            <configuration>
+              <aggregate>true</aggregate>
+            </configuration>
+          </plugin>
         </plugins>
       </build>
     </profile>
     <profile>
-      <id>dist</id>
+      <id>src</id>
       <activation>
         <activeByDefault>false</activeByDefault>
-        <property>
-          <name>tar|rpm|deb</name>
-        </property>
       </activation>
       <build>
         <plugins>
           <plugin>
             <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-dependency-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>copy-dependencies</id>
-                <phase>prepare-package</phase>
-                <goals>
-                  <goal>copy-dependencies</goal>
-                </goals>
-                <configuration>
-                  <includeScope>runtime</includeScope>
-                  <excludeGroupIds>org.apache.tajo</excludeGroupIds>
-                  <outputDirectory>${project.build.directory}/lib</outputDirectory>
-                  <overWriteReleases>false</overWriteReleases>
-                  <overWriteSnapshots>false</overWriteSnapshots>
-                  <overWriteIfNewer>true</overWriteIfNewer>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <artifactId>maven-source-plugin</artifactId>
             <executions>
               <execution>
-                <id>dist</id>
-                <phase>prepare-package</phase>
+                <!-- builds source jars and attaches them to the project for publishing -->
+                <id>hadoop-java-sources</id>
+                <phase>package</phase>
                 <goals>
-                  <goal>run</goal>
+                  <goal>jar-no-fork</goal>
                 </goals>
-                <configuration>
-                  <target>
-                    <echo file="${project.build.directory}/dist-layout-stitching.sh">
-                      run() {
-                      echo "\$ ${@}"
-                      "${@}"
-                      res=$?
-                      if [ $res != 0 ]; then
-                      echo
-                      echo "Failed!"
-                      echo
-                      exit $res
-                      fi
-                      }
-
-                      ROOT=`cd ${basedir}/..;pwd`
-                      echo
-                      echo "Current directory `pwd`"
-                      echo
-                      run rm -rf ${project.artifactId}-${project.version}
-                      run mkdir ${project.artifactId}-${project.version}
-                      run cd ${project.artifactId}-${project.version}
-                      run cp -r ${basedir}/${project.artifactId}-pullserver/target/${project.artifactId}-pullserver-${project.version}*.jar .
-                      run cp -r ${basedir}/${project.artifactId}-backend/target/${project.artifactId}-backend-${project.version}*.jar .
-                      run cp -r ${basedir}/${project.artifactId}-backend/target/lib .
-                      echo
-                      echo "Tajo Core dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
-                      echo
-                    </echo>
-                    <exec executable="sh" dir="${project.build.directory}" failonerror="true">
-                      <arg line="./dist-layout-stitching.sh"/>
-                    </exec>
-                  </target>
-                </configuration>
               </execution>
             </executions>
           </plugin>
         </plugins>
       </build>
     </profile>
+    <profile>
+      <id>hcatalog-0.12.0</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.thrift</groupId>
+          <artifactId>libfb303</artifactId>
+          <version>0.9.0</version>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.thrift</groupId>
+          <artifactId>libthrift</artifactId>
+          <version>0.9.0</version>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.tajo</groupId>
+          <artifactId>tajo-hcatalog</artifactId>
+          <scope>test</scope>
+          <version>${tajo.version}</version>
+          <exclusions>
+            <exclusion>
+              <groupId>com.google.protobuf</groupId>
+              <artifactId>protobuf-java</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hive</groupId>
+          <artifactId>hive-exec</artifactId>
+          <version>0.12.0</version>
+          <scope>provided</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-common</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-contrib</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-hbase-handler</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-metastore</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-serde</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-shims</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-testutils</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.thrift</groupId>
+              <artifactId>libfb303</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.thrift</groupId>
+              <artifactId>libthrift</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.jolbox</groupId>
+              <artifactId>bonecp</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hbase</groupId>
+              <artifactId>hbase</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.google.protobuf</groupId>
+              <artifactId>protobuf-java</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hive</groupId>
+          <artifactId>hive-metastore</artifactId>
+          <version>0.12.0</version>
+          <scope>provided</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-common</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-serde</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-shimss</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.thrift</groupId>
+              <artifactId>libfb303</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.thrift</groupId>
+              <artifactId>libthrift</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.jolbox</groupId>
+              <artifactId>bonecp</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.google.protobuf</groupId>
+              <artifactId>protobuf-java</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hive</groupId>
+          <artifactId>hive-cli</artifactId>
+          <version>0.12.0</version>
+          <scope>provided</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-common</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-exec</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-metastore</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-serde</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-service</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-shims</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.jolbox</groupId>
+              <artifactId>bonecp</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.google.protobuf</groupId>
+              <artifactId>protobuf-java</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>jline</groupId>
+              <artifactId>jline</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hive.hcatalog</groupId>
+          <artifactId>hcatalog-core</artifactId>
+          <version>0.12.0</version>
+          <scope>provided</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-cli</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-common</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-exec</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-metastore</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-serde</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-service</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.apache.hive</groupId>
+              <artifactId>hive-shims</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.jolbox</groupId>
+              <artifactId>bonecp</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.google.protobuf</groupId>
+              <artifactId>protobuf-java</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
   </profiles>
 
   <reporting>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+        <version>2.4</version>
+        <configuration>
+          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-report-plugin</artifactId>
         <version>2.15</version>
       </plugin>

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4 b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
new file mode 100644
index 0000000..f7b76ef
--- /dev/null
+++ b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
@@ -0,0 +1,390 @@
+/**
+   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.
+*/
+lexer grammar HiveQLLexer;
+
+
+// Keywords
+
+KW_TRUE : 'TRUE';
+KW_FALSE : 'FALSE';
+KW_ALL : 'ALL';
+KW_AND : 'AND';
+KW_OR : 'OR';
+KW_NOT : 'NOT' | '!';
+KW_LIKE : 'LIKE';
+
+KW_IF : 'IF';
+KW_EXISTS : 'EXISTS';
+
+KW_ASC : 'ASC';
+KW_DESC : 'DESC';
+KW_ORDER : 'ORDER';
+KW_GROUP : 'GROUP';
+KW_BY : 'BY';
+KW_HAVING : 'HAVING';
+KW_WHERE : 'WHERE';
+KW_FROM : 'FROM';
+KW_AS : 'AS';
+KW_SELECT : 'SELECT';
+KW_DISTINCT : 'DISTINCT';
+KW_INSERT : 'INSERT';
+KW_OVERWRITE : 'OVERWRITE';
+KW_OUTER : 'OUTER';
+KW_UNIQUEJOIN : 'UNIQUEJOIN';
+KW_PRESERVE : 'PRESERVE';
+KW_JOIN : 'JOIN';
+KW_LEFT : 'LEFT';
+KW_RIGHT : 'RIGHT';
+KW_FULL : 'FULL';
+KW_ON : 'ON';
+KW_PARTITION : 'PARTITION';
+KW_PARTITIONS : 'PARTITIONS';
+KW_TABLE: 'TABLE';
+KW_TABLES: 'TABLES';
+KW_COLUMNS: 'COLUMNS';
+KW_INDEX: 'INDEX';
+KW_INDEXES: 'INDEXES';
+KW_REBUILD: 'REBUILD';
+KW_FUNCTIONS: 'FUNCTIONS';
+KW_SHOW: 'SHOW';
+KW_MSCK: 'MSCK';
+KW_REPAIR: 'REPAIR';
+KW_DIRECTORY: 'DIRECTORY';
+KW_LOCAL: 'LOCAL';
+KW_TRANSFORM : 'TRANSFORM';
+KW_USING: 'USING';
+KW_CLUSTER: 'CLUSTER';
+KW_DISTRIBUTE: 'DISTRIBUTE';
+KW_SORT: 'SORT';
+KW_UNION: 'UNION';
+KW_LOAD: 'LOAD';
+KW_EXPORT: 'EXPORT';
+KW_IMPORT: 'IMPORT';
+KW_DATA: 'DATA';
+KW_INPATH: 'INPATH';
+KW_IS: 'IS';
+KW_NULL: 'NULL';
+KW_CREATE: 'CREATE';
+KW_EXTERNAL: 'EXTERNAL';
+KW_ALTER: 'ALTER';
+KW_CHANGE: 'CHANGE';
+KW_COLUMN: 'COLUMN';
+KW_FIRST: 'FIRST';
+KW_AFTER: 'AFTER';
+KW_DESCRIBE: 'DESCRIBE';
+KW_DROP: 'DROP';
+KW_RENAME: 'RENAME';
+KW_IGNORE: 'IGNORE';
+KW_PROTECTION: 'PROTECTION';
+KW_TO: 'TO';
+KW_COMMENT: 'COMMENT';
+KW_BOOLEAN: 'BOOLEAN';
+KW_TINYINT: 'TINYINT';
+KW_SMALLINT: 'SMALLINT';
+KW_INT: 'INT';
+KW_BIGINT: 'BIGINT';
+KW_FLOAT: 'FLOAT';
+KW_DOUBLE: 'DOUBLE';
+KW_DATE: 'DATE';
+KW_DATETIME: 'DATETIME';
+KW_TIMESTAMP: 'TIMESTAMP';
+KW_DECIMAL: 'DECIMAL';
+KW_STRING: 'STRING';
+KW_ARRAY: 'ARRAY';
+KW_STRUCT: 'STRUCT';
+KW_MAP: 'MAP';
+KW_UNIONTYPE: 'UNIONTYPE';
+KW_REDUCE: 'REDUCE';
+KW_PARTITIONED: 'PARTITIONED';
+KW_CLUSTERED: 'CLUSTERED';
+KW_SORTED: 'SORTED';
+KW_INTO: 'INTO';
+KW_BUCKETS: 'BUCKETS';
+KW_ROW: 'ROW';
+KW_ROWS: 'ROWS';
+KW_FORMAT: 'FORMAT';
+KW_DELIMITED: 'DELIMITED';
+KW_FIELDS: 'FIELDS';
+KW_TERMINATED: 'TERMINATED';
+KW_ESCAPED: 'ESCAPED';
+KW_COLLECTION: 'COLLECTION';
+KW_ITEMS: 'ITEMS';
+KW_KEYS: 'KEYS';
+KW_KEY_TYPE: '$KEY$';
+KW_LINES: 'LINES';
+KW_STORED: 'STORED';
+KW_FILEFORMAT: 'FILEFORMAT';
+KW_SEQUENCEFILE: 'SEQUENCEFILE';
+KW_TEXTFILE: 'TEXTFILE';
+KW_RCFILE: 'RCFILE';
+KW_ORCFILE: 'ORC';
+KW_INPUTFORMAT: 'INPUTFORMAT';
+KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
+KW_INPUTDRIVER: 'INPUTDRIVER';
+KW_OUTPUTDRIVER: 'OUTPUTDRIVER';
+KW_OFFLINE: 'OFFLINE';
+KW_ENABLE: 'ENABLE';
+KW_DISABLE: 'DISABLE';
+KW_READONLY: 'READONLY';
+KW_NO_DROP: 'NO_DROP';
+KW_LOCATION: 'LOCATION';
+KW_TABLESAMPLE: 'TABLESAMPLE';
+KW_BUCKET: 'BUCKET';
+KW_OUT: 'OUT';
+KW_OF: 'OF';
+KW_PERCENT: 'PERCENT';
+KW_CAST: 'CAST';
+KW_ADD: 'ADD';
+KW_REPLACE: 'REPLACE';
+KW_RLIKE: 'RLIKE';
+KW_REGEXP: 'REGEXP';
+KW_TEMPORARY: 'TEMPORARY';
+KW_FUNCTION: 'FUNCTION';
+KW_EXPLAIN: 'EXPLAIN';
+KW_EXTENDED: 'EXTENDED';
+KW_FORMATTED: 'FORMATTED';
+KW_PRETTY: 'PRETTY';
+KW_DEPENDENCY: 'DEPENDENCY';
+KW_SERDE: 'SERDE';
+KW_WITH: 'WITH';
+KW_DEFERRED: 'DEFERRED';
+KW_SERDEPROPERTIES: 'SERDEPROPERTIES';
+KW_DBPROPERTIES: 'DBPROPERTIES';
+KW_LIMIT: 'LIMIT';
+KW_SET: 'SET';
+KW_UNSET: 'UNSET';
+KW_TBLPROPERTIES: 'TBLPROPERTIES';
+KW_IDXPROPERTIES: 'IDXPROPERTIES';
+KW_VALUE_TYPE: '$VALUE$';
+KW_ELEM_TYPE: '$ELEM$';
+KW_CASE: 'CASE';
+KW_WHEN: 'WHEN';
+KW_THEN: 'THEN';
+KW_ELSE: 'ELSE';
+KW_END: 'END';
+KW_MAPJOIN: 'MAPJOIN';
+KW_STREAMTABLE: 'STREAMTABLE';
+KW_HOLD_DDLTIME: 'HOLD_DDLTIME';
+KW_CLUSTERSTATUS: 'CLUSTERSTATUS';
+KW_UTC: 'UTC';
+KW_UTCTIMESTAMP: 'UTC_TMESTAMP';
+KW_LONG: 'LONG';
+KW_DELETE: 'DELETE';
+KW_PLUS: 'PLUS';
+KW_MINUS: 'MINUS';
+KW_FETCH: 'FETCH';
+KW_INTERSECT: 'INTERSECT';
+KW_VIEW: 'VIEW';
+KW_IN: 'IN';
+KW_DATABASE: 'DATABASE';
+KW_DATABASES: 'DATABASES';
+KW_MATERIALIZED: 'MATERIALIZED';
+KW_SCHEMA: 'SCHEMA';
+KW_SCHEMAS: 'SCHEMAS';
+KW_GRANT: 'GRANT';
+KW_REVOKE: 'REVOKE';
+KW_SSL: 'SSL';
+KW_UNDO: 'UNDO';
+KW_LOCK: 'LOCK';
+KW_LOCKS: 'LOCKS';
+KW_UNLOCK: 'UNLOCK';
+KW_SHARED: 'SHARED';
+KW_EXCLUSIVE: 'EXCLUSIVE';
+KW_PROCEDURE: 'PROCEDURE';
+KW_UNSIGNED: 'UNSIGNED';
+KW_WHILE: 'WHILE';
+KW_READ: 'READ';
+KW_READS: 'READS';
+KW_PURGE: 'PURGE';
+KW_RANGE: 'RANGE';
+KW_ANALYZE: 'ANALYZE';
+KW_BEFORE: 'BEFORE';
+KW_BETWEEN: 'BETWEEN';
+KW_BOTH: 'BOTH';
+KW_BINARY: 'BINARY';
+KW_CROSS: 'CROSS';
+KW_CONTINUE: 'CONTINUE';
+KW_CURSOR: 'CURSOR';
+KW_TRIGGER: 'TRIGGER';
+KW_RECORDREADER: 'RECORDREADER';
+KW_RECORDWRITER: 'RECORDWRITER';
+KW_SEMI: 'SEMI';
+KW_LATERAL: 'LATERAL';
+KW_TOUCH: 'TOUCH';
+KW_ARCHIVE: 'ARCHIVE';
+KW_UNARCHIVE: 'UNARCHIVE';
+KW_COMPUTE: 'COMPUTE';
+KW_STATISTICS: 'STATISTICS';
+KW_USE: 'USE';
+KW_OPTION: 'OPTION';
+KW_CONCATENATE: 'CONCATENATE';
+KW_SHOW_DATABASE: 'SHOW_DATABASE';
+KW_UPDATE: 'UPDATE';
+KW_RESTRICT: 'RESTRICT';
+KW_CASCADE: 'CASCADE';
+KW_SKEWED: 'SKEWED';
+KW_ROLLUP: 'ROLLUP';
+KW_CUBE: 'CUBE';
+KW_DIRECTORIES: 'DIRECTORIES';
+KW_FOR: 'FOR';
+KW_WINDOW: 'WINDOW';
+KW_UNBOUNDED: 'UNBOUNDED';
+KW_PRECEDING: 'PRECEDING';
+KW_FOLLOWING: 'FOLLOWING';
+KW_CURRENT: 'CURRENT';
+KW_LESS: 'LESS';
+KW_MORE: 'MORE';
+KW_OVER: 'OVER';
+KW_GROUPING: 'GROUPING';
+KW_SETS: 'SETS';
+KW_TRUNCATE: 'TRUNCATE';
+KW_NOSCAN: 'NOSCAN';
+KW_PARTIALSCAN: 'PARTIALSCAN';
+KW_USER: 'USER';
+KW_ROLE: 'ROLE';
+KW_INNER: 'INNER';
+
+// Operators
+// NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.
+
+DOT : '.'; // generated as a part of Number rule
+COLON : ':' ;
+COMMA : ',' ;
+SEMICOLON : ';' ;
+
+LPAREN : '(' ;
+RPAREN : ')' ;
+LSQUARE : '[' ;
+RSQUARE : ']' ;
+LCURLY : '{';
+RCURLY : '}';
+
+EQUAL : '=' | '==';
+EQUAL_NS : '<=>';
+NOTEQUAL : '<>' | '!=';
+LESSTHANOREQUALTO : '<=';
+LESSTHAN : '<';
+GREATERTHANOREQUALTO : '>=';
+GREATERTHAN : '>';
+
+DIVIDE : '/';
+PLUS : '+';
+MINUS : '-';
+STAR : '*';
+MOD : '%';
+DIV : 'DIV';
+
+AMPERSAND : '&';
+TILDE : '~';
+BITWISEOR : '|';
+BITWISEXOR : '^';
+QUESTION : '?';
+DOLLAR : '$';
+
+// LITERALS
+fragment
+Letter
+    : 'a'..'z' | 'A'..'Z'
+    ;
+
+fragment
+HexDigit
+    : 'a'..'f' | 'A'..'F'
+    ;
+
+fragment
+Digit
+    :
+    '0'..'9'
+    ;
+
+fragment
+Exponent
+    :
+    ('e' | 'E') ( PLUS|MINUS )? (Digit)+
+    ;
+
+fragment
+RegexComponent
+    : 'a'..'z' | 'A'..'Z' | '0'..'9' | '_'
+    | PLUS | STAR | QUESTION | MINUS | DOT
+    | LPAREN | RPAREN | LSQUARE | RSQUARE | LCURLY | RCURLY
+    | BITWISEXOR | BITWISEOR | DOLLAR
+    ;
+
+StringLiteral
+    :
+    ( '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
+    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
+    )+
+    ;
+
+CharSetLiteral
+    :
+    StringLiteral
+    | '0' 'X' (HexDigit|Digit)+
+    ;
+
+BigintLiteral
+    :
+    (Digit)+ 'L'
+    ;
+
+SmallintLiteral
+    :
+    (Digit)+ 'S'
+    ;
+
+TinyintLiteral
+    :
+    (Digit)+ 'Y'
+    ;
+
+DecimalLiteral
+    :
+    Number 'B' 'D'
+    ;
+
+ByteLengthLiteral
+    :
+    (Digit)+ ('b' | 'B' | 'k' | 'K' | 'm' | 'M' | 'g' | 'G')
+    ;
+
+Number
+    :
+    (Digit)+ ( DOT (Digit)* (Exponent)? | Exponent)?
+    ;
+    
+Identifier
+    : (Letter | Digit) (Letter | Digit | '_')* { setText(getText().toLowerCase()); }
+    ;
+
+CharSetName
+    :
+    '_' (Letter | Digit | '_' | '-' | '.' | ':' )+
+    ;
+
+WS  :
+(' '|'\r'|'\t'|'\n') -> skip
+    ;
+
+COMMENT
+  : '--' (~('\n'|'\r'))*  -> skip
+  ;
+
+
+


[02/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
new file mode 100644
index 0000000..87262e8
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
@@ -0,0 +1,166 @@
+/**
+ * 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.planner.physical;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Random;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestSortExec {
+  private static TajoConf conf;
+  private static final String TEST_PATH = "target/test-data/TestPhysicalPlanner";
+  private static TajoTestingCluster util;
+  private static CatalogService catalog;
+  private static SQLAnalyzer analyzer;
+  private static LogicalPlanner planner;
+  private static LogicalOptimizer optimizer;
+  private static AbstractStorageManager sm;
+  private static Path workDir;
+  private static Path tablePath;
+  private static TableMeta employeeMeta;
+
+  private static Random rnd = new Random(System.currentTimeMillis());
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new TajoConf();
+    util = TpchTestBase.getInstance().getTestingCluster();
+    catalog = util.getMaster().getCatalog();
+    workDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    sm = StorageManagerFactory.getStorageManager(conf, workDir);
+
+    Schema schema = new Schema();
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
+
+    employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
+
+    tablePath = StorageUtil.concatPath(workDir, "employee", "table1");
+    sm.getFileSystem().mkdirs(tablePath.getParent());
+
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(employeeMeta, schema, tablePath);
+    appender.init();
+    Tuple tuple = new VTuple(schema.size());
+    for (int i = 0; i < 100; i++) {
+      tuple.put(new Datum[] {
+          DatumFactory.createInt4(rnd.nextInt(5)),
+          DatumFactory.createInt4(rnd.nextInt(10)),
+          DatumFactory.createText("dept_" + rnd.nextInt(10))});
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+
+    TableDesc desc = new TableDesc(
+        CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, employeeMeta,
+        tablePath);
+    catalog.createTable(desc);
+
+    analyzer = new SQLAnalyzer();
+    planner = new LogicalPlanner(catalog);
+    optimizer = new LogicalOptimizer(conf);
+  }
+
+  public static String[] QUERIES = {
+      "select managerId, empId, deptName from employee order by managerId, empId desc" };
+
+  @Test
+  public final void testNext() throws IOException, PlanningException {
+    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employeeMeta, tablePath, Integer.MAX_VALUE);
+    Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestSortExec");
+    TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility
+        .newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
+    ctx.setEnforcer(new Enforcer());
+    Expr context = analyzer.parse(QUERIES[0]);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), context);
+    LogicalNode rootNode = optimizer.optimize(plan);
+
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
+
+    Tuple tuple;
+    Datum preVal = null;
+    Datum curVal;
+    exec.init();
+    while ((tuple = exec.next()) != null) {
+      curVal = tuple.get(0);
+      if (preVal != null) {
+        assertTrue(preVal.lessThanEqual(curVal).asBool());
+      }
+
+      preVal = curVal;
+    }
+    exec.close();
+  }
+
+  @Test
+  /**
+   * TODO - Now, in FSM branch, TestUniformRangePartition is ported to Java.
+   * So, this test is located in here.
+   * Later it should be moved TestUniformPartitions.
+   */
+  public void testTAJO_946() {
+    Schema schema = new Schema();
+    schema.addColumn("l_orderkey", Type.INT8);
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(1);
+    s.put(0, DatumFactory.createInt8(0));
+    Tuple e = new VTuple(1);
+    e.put(0, DatumFactory.createInt8(6000000000l));
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner
+        = new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(967);
+
+    TupleRange prev = null;
+    for (TupleRange r : ranges) {
+      if (prev == null) {
+        prev = r;
+      } else {
+        assertTrue(prev.compareTo(r) < 0);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java
new file mode 100644
index 0000000..57d8b32
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java
@@ -0,0 +1,52 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+
+@Category(IntegrationTest.class)
+public class TestAlterTable extends QueryTestCaseBase {
+  @Test
+  public final void testAlterTableName() throws Exception {
+    List<String> createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "ABC");
+    assertTableExists(createdNames.get(0));
+    executeDDL("alter_table_rename_table_ddl.sql", null);
+    assertTableExists("DEF");
+  }
+
+  @Test
+  public final void testAlterTableColumnName() throws Exception {
+    List<String> createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "XYZ");
+    executeDDL("alter_table_rename_column_ddl.sql", null);
+    assertColumnExists(createdNames.get(0),"renum");
+  }
+
+  @Test
+  public final void testAlterTableAddNewColumn() throws Exception {
+    List<String> createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "EFG");
+    executeDDL("alter_table_add_new_column_ddl.sql", null);
+    assertColumnExists(createdNames.get(0),"cool");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java
new file mode 100644
index 0000000..47e98a9
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java
@@ -0,0 +1,59 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.*;
+
+@Category(IntegrationTest.class)
+public class TestAlterTablespace extends QueryTestCaseBase {
+
+  @Test
+  public final void testAlterLocation() throws Exception {
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      //////////////////////////////////////////////////////////////////////////////
+      // Create two table spaces
+      //////////////////////////////////////////////////////////////////////////////
+
+      assertFalse(catalog.existTablespace("space1"));
+      assertTrue(catalog.createTablespace("space1", "hdfs://xxx.com/warehouse"));
+      assertTrue(catalog.existTablespace("space1"));
+
+      // pre verification
+      CatalogProtos.TablespaceProto space1 = catalog.getTablespace("space1");
+      assertEquals("space1", space1.getSpaceName());
+      assertEquals("hdfs://xxx.com/warehouse", space1.getUri());
+
+      executeString("ALTER TABLESPACE space1 LOCATION 'hdfs://yyy.com/warehouse';");
+
+      // Verify ALTER TABLESPACE space1
+      space1 = catalog.getTablespace("space1");
+      assertEquals("space1", space1.getSpaceName());
+      assertEquals("hdfs://yyy.com/warehouse", space1.getUri());
+
+      assertTrue(catalog.dropTablespace("space1"));
+      assertFalse(catalog.existTablespace("space1"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
new file mode 100644
index 0000000..a4e31e0
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
@@ -0,0 +1,239 @@
+/**
+ * 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.query;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.storage.StorageConstants;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+import java.util.Map;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.apache.tajo.catalog.CatalogUtil.buildFQName;
+import static org.junit.Assert.*;
+
+
+/**
+ * Test CREATE TABLE AS SELECT statements
+ */
+@Category(IntegrationTest.class)
+public class TestCTASQuery extends QueryTestCaseBase {
+
+  public TestCTASQuery() {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test
+  public final void testCtasWithoutTableDefinition() throws Exception {
+    ResultSet res = executeQuery();
+    res.close();
+
+    String tableName = CatalogUtil.normalizeIdentifier("testCtasWithoutTableDefinition");
+    CatalogService catalog = testBase.getTestingCluster().getMaster().getCatalog();
+    String qualifiedTableName = buildFQName(DEFAULT_DATABASE_NAME, tableName);
+    TableDesc desc = catalog.getTableDesc(qualifiedTableName);
+    assertTrue(catalog.existsTable(qualifiedTableName));
+
+    assertTrue(desc.getSchema().contains("default.testctaswithouttabledefinition.col1"));
+    PartitionMethodDesc partitionDesc = desc.getPartitionMethod();
+    assertEquals(partitionDesc.getPartitionType(), CatalogProtos.PartitionType.COLUMN);
+    assertEquals("key", partitionDesc.getExpressionSchema().getColumns().get(0).getSimpleName());
+
+    FileSystem fs = FileSystem.get(testBase.getTestingCluster().getConfiguration());
+    Path path = desc.getPath();
+    assertTrue(fs.isDirectory(path));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=38.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=45.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=49.0")));
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      assertEquals(5, desc.getStats().getNumRows().intValue());
+    }
+
+    ResultSet res2 = executeFile("check1.sql");
+
+    Map<Double, int []> resultRows1 = Maps.newHashMap();
+    resultRows1.put(45.0d, new int[]{3, 2});
+    resultRows1.put(38.0d, new int[]{2, 2});
+
+    int i = 0;
+    while(res2.next()) {
+      assertEquals(resultRows1.get(res2.getDouble(3))[0], res2.getInt(1));
+      assertEquals(resultRows1.get(res2.getDouble(3))[1], res2.getInt(2));
+      i++;
+    }
+    res2.close();
+    assertEquals(2, i);
+  }
+
+  @Test
+  public final void testCtasWithColumnedPartition() throws Exception {
+    ResultSet res = executeQuery();
+    res.close();
+
+    String tableName = CatalogUtil.normalizeIdentifier("testCtasWithColumnedPartition");
+
+    TajoTestingCluster cluster = testBase.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+    PartitionMethodDesc partitionDesc = desc.getPartitionMethod();
+    assertEquals(partitionDesc.getPartitionType(), CatalogProtos.PartitionType.COLUMN);
+    assertEquals("key", partitionDesc.getExpressionSchema().getColumns().get(0).getSimpleName());
+
+    FileSystem fs = FileSystem.get(cluster.getConfiguration());
+    Path path = desc.getPath();
+    assertTrue(fs.isDirectory(path));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=38.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=45.0")));
+    assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=49.0")));
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(5, desc.getStats().getNumRows().intValue());
+    }
+
+    ResultSet res2 = executeFile("check2.sql");
+
+    Map<Double, int []> resultRows1 = Maps.newHashMap();
+    resultRows1.put(45.0d, new int[]{3, 2});
+    resultRows1.put(38.0d, new int[]{2, 2});
+
+    int i = 0;
+    while(res2.next()) {
+      assertEquals(resultRows1.get(res2.getDouble(3))[0], res2.getInt(1));
+      assertEquals(resultRows1.get(res2.getDouble(3))[1], res2.getInt(2));
+      i++;
+    }
+    res2.close();
+    assertEquals(2, i);
+  }
+
+  @Test
+  public final void testCtasWithGroupby() throws Exception {
+    ResultSet res = executeFile("CtasWithGroupby.sql");
+    res.close();
+
+    ResultSet res2 = executeQuery();
+    assertResultSet(res2);
+    res2.close();
+  }
+
+  @Test
+  public final void testCtasWithOrderby() throws Exception {
+    ResultSet res = executeFile("CtasWithOrderby.sql");
+    res.close();
+
+    ResultSet res2 = executeQuery();
+    assertResultSet(res2);
+    res2.close();
+  }
+
+  @Test
+  public final void testCtasWithLimit() throws Exception {
+    ResultSet res = executeFile("CtasWithLimit.sql");
+    res.close();
+
+    ResultSet res2 = executeQuery();
+    assertResultSet(res2);
+    res2.close();
+  }
+
+  @Test
+  public final void testCtasWithUnion() throws Exception {
+    ResultSet res = executeFile("CtasWithUnion.sql");
+    res.close();
+
+    ResultSet res2 = executeQuery();
+    resultSetToString(res2);
+    res2.close();
+  }
+
+  @Test
+  public final void testCtasWithStoreType() throws Exception {
+    ResultSet res = executeFile("CtasWithStoreType.sql");
+    res.close();
+
+    ResultSet res2 = executeQuery();
+    resultSetToString(res2);
+    res2.close();
+
+    TableDesc desc =  client.getTableDesc(CatalogUtil.normalizeIdentifier(res2.getMetaData().getTableName(1)));
+    assertNotNull(desc);
+    assertEquals(CatalogProtos.StoreType.RCFILE, desc.getMeta().getStoreType());
+  }
+
+  @Test
+  public final void testCtasWithOptions() throws Exception {
+    ResultSet res = executeFile("CtasWithOptions.sql");
+    res.close();
+
+    ResultSet res2 = executeQuery();
+    resultSetToString(res2);
+    res2.close();
+
+    TableDesc desc =  client.getTableDesc(CatalogUtil.normalizeIdentifier(res2.getMetaData().getTableName(1)));
+    assertNotNull(desc);
+    assertEquals(CatalogProtos.StoreType.CSV, desc.getMeta().getStoreType());
+
+
+    Options options = desc.getMeta().getOptions();
+    assertNotNull(options);
+    assertEquals(StringEscapeUtils.escapeJava("\u0001"), options.get(StorageConstants.CSVFILE_DELIMITER));
+  }
+
+  @Test
+  public final void testCtasWithManagedTable() throws Exception {
+    ResultSet res = executeFile("CtasWithManagedTable.sql");
+    res.close();
+
+    if (testingCluster.isHCatalogStoreRunning()) {
+      assertTrue(client.existTable("managed_table1"));
+
+      TableDesc desc =  client.getTableDesc("managed_table1");
+
+      assertNotNull(desc);
+      assertEquals("managed_table1", desc.getPath().getName());
+    } else {
+      assertFalse(client.existTable("managed_Table1"));
+      assertTrue(client.existTable("MANAGED_TABLE1"));
+
+      TableDesc desc =  client.getTableDesc("MANAGED_TABLE1");
+
+      assertNotNull(desc);
+      assertEquals("MANAGED_TABLE1", desc.getPath().getName());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java
new file mode 100644
index 0000000..9836a57
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java
@@ -0,0 +1,70 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.junit.Test;
+
+import java.sql.ResultSet;
+
+public class TestCaseByCases extends QueryTestCaseBase {
+
+  public TestCaseByCases() {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test
+  public final void testTAJO415Case() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testTAJO418Case() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  /**
+   * It's an unit test to reproduce TAJO-619 (https://issues.apache.org/jira/browse/TAJO-619).
+   */
+  @Test
+  public final void testTAJO619Case() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testTAJO718Case() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testTAJO739Case() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java
new file mode 100644
index 0000000..453c174
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java
@@ -0,0 +1,75 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+
+@Category(IntegrationTest.class)
+public class TestCreateDatabase extends QueryTestCaseBase {
+
+  @Test
+  public final void testCreateAndDropDatabase() throws Exception {
+    String databaseName = CatalogUtil.normalizeIdentifier("testCreateAndDropDatabase");
+
+    ResultSet res = null;
+    try {
+      res = executeString("CREATE DATABASE testCreateAndDropDatabase;");
+      assertDatabaseExists(databaseName);
+      executeString("DROP DATABASE testCreateAndDropDatabase;");
+      assertDatabaseNotExists(databaseName);
+    } finally {
+      cleanupQuery(res);
+    }
+  }
+
+  @Test
+  public final void testCreateIfNotExists() throws Exception {
+    String databaseName = CatalogUtil.normalizeIdentifier("testCreateIfNotExists");
+
+    assertDatabaseNotExists(databaseName);
+    executeString("CREATE DATABASE " + databaseName + ";").close();
+    assertDatabaseExists(databaseName);
+
+    executeString("CREATE DATABASE IF NOT EXISTS " + databaseName + ";").close();
+    assertDatabaseExists(databaseName);
+
+    executeString("DROP DATABASE " + databaseName + ";").close();
+    assertDatabaseNotExists(databaseName);
+  }
+
+  @Test
+  public final void testDropIfExists() throws Exception {
+    String databaseName = CatalogUtil.normalizeIdentifier("testDropIfExists");
+    assertDatabaseNotExists(databaseName);
+    executeString("CREATE DATABASE " + databaseName + ";").close();
+    assertDatabaseExists(databaseName);
+
+    executeString("DROP DATABASE " + databaseName + ";").close();
+    assertDatabaseNotExists(databaseName);
+
+    executeString("DROP DATABASE IF EXISTS " + databaseName + ";");
+    assertDatabaseNotExists(databaseName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
new file mode 100644
index 0000000..2d289ba
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
@@ -0,0 +1,360 @@
+/**
+ * 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.query;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.StorageUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category(IntegrationTest.class)
+public class TestCreateTable extends QueryTestCaseBase {
+
+  @Test
+  public final void testVariousTypes() throws Exception {
+    List<String> createdNames;
+    if (testingCluster.isHCatalogStoreRunning()) {
+      createdNames = executeDDL("create_table_various_types_for_hcatalog.sql", null);
+    } else {
+      createdNames = executeDDL("create_table_various_types.sql", null);
+    }
+    assertTableExists(createdNames.get(0));
+  }
+
+  @Test
+  public final void testCreateTable1() throws Exception {
+    List<String> createdNames = executeDDL("table1_ddl.sql", "table1", "table1");
+    assertTableExists(createdNames.get(0));
+  }
+
+  @Test
+  public final void testCreateTable2() throws Exception {
+    executeString("CREATE DATABASE D1;").close();
+    executeString("CREATE DATABASE D2;").close();
+
+    executeString("CREATE TABLE D1.table1 (age int);").close();
+    executeString("CREATE TABLE D1.table2 (age int);").close();
+    executeString("CREATE TABLE d2.table3 (age int);").close();
+    executeString("CREATE TABLE d2.table4 (age int);").close();
+
+    assertTableExists("d1.table1");
+    assertTableExists("d1.table2");
+    assertTableNotExists("d2.table1");
+    assertTableNotExists("d2.table2");
+
+    assertTableExists("d2.table3");
+    assertTableExists("d2.table4");
+    assertTableNotExists("d1.table3");
+    assertTableNotExists("d1.table4");
+
+    executeString("DROP TABLE D1.table1");
+    executeString("DROP TABLE D1.table2");
+    executeString("DROP TABLE D2.table3");
+    executeString("DROP TABLE D2.table4");
+
+    assertDatabaseExists("d1");
+    assertDatabaseExists("d2");
+    executeString("DROP DATABASE D1").close();
+    executeString("DROP DATABASE D2").close();
+    assertDatabaseNotExists("d1");
+    assertDatabaseNotExists("d2");
+  }
+
+  private final void assertPathOfCreatedTable(final String databaseName,
+                                              final String originalTableName,
+                                              final String newTableName,
+                                              String createTableStmt) throws Exception {
+    // create one table
+    executeString("CREATE DATABASE " + CatalogUtil.denormalizeIdentifier(databaseName)).close();
+    getClient().existDatabase(CatalogUtil.denormalizeIdentifier(databaseName));
+    final String oldFQTableName = CatalogUtil.buildFQName(databaseName, originalTableName);
+
+    ResultSet res = executeString(createTableStmt);
+    res.close();
+    assertTableExists(oldFQTableName);
+    TableDesc oldTableDesc = client.getTableDesc(oldFQTableName);
+
+
+    // checking the existence of the table directory and validating the path
+    FileSystem fs = testingCluster.getMaster().getStorageManager().getFileSystem();
+    Path warehouseDir = TajoConf.getWarehouseDir(testingCluster.getConfiguration());
+    assertTrue(fs.exists(oldTableDesc.getPath()));
+    assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, originalTableName), oldTableDesc.getPath());
+
+    // Rename
+    client.executeQuery("ALTER TABLE " + CatalogUtil.denormalizeIdentifier(oldFQTableName)
+        + " RENAME to " + CatalogUtil.denormalizeIdentifier(newTableName));
+
+    // checking the existence of the new table directory and validating the path
+    final String newFQTableName = CatalogUtil.buildFQName(databaseName, newTableName);
+    TableDesc newTableDesc = client.getTableDesc(newFQTableName);
+    assertTrue(fs.exists(newTableDesc.getPath()));
+    assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, newTableName), newTableDesc.getPath());
+  }
+
+  @Test
+  public final void testCreatedTableViaCTASAndVerifyPath() throws Exception {
+    assertPathOfCreatedTable("d4", "old_table", "new_mgmt_table",
+        "CREATE TABLE d4.old_table AS SELECT * FROM default.lineitem;");
+  }
+
+  @Test
+  public final void testCreatedTableJustCreatedAndVerifyPath() throws Exception {
+    assertPathOfCreatedTable("d5", "old_table", "new_mgmt_table", "CREATE TABLE d5.old_table (age integer);");
+  }
+
+  @Test
+  public final void testCreatedTableWithQuotedIdentifierAndVerifyPath() throws Exception {
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      assertPathOfCreatedTable("D6", "OldTable", "NewMgmtTable", "CREATE TABLE \"D6\".\"OldTable\" (age integer);");
+    }
+  }
+
+  @Test
+  public final void testCreateTableIfNotExists() throws Exception {
+    executeString("CREATE DATABASE D3;").close();
+
+    assertTableNotExists("d3.table1");
+    executeString("CREATE TABLE D3.table1 (age int);").close();
+    assertTableExists("d3.table1");
+
+    executeString("CREATE TABLE IF NOT EXISTS D3.table1 (age int);").close();
+    assertTableExists("d3.table1");
+
+    executeString("DROP TABLE D3.table1");
+  }
+
+  @Test
+  public final void testDropTableIfExists() throws Exception {
+    executeString("CREATE DATABASE D4;").close();
+
+    assertTableNotExists("d4.table1");
+    executeString("CREATE TABLE d4.table1 (age int);").close();
+    assertTableExists("d4.table1");
+
+    executeString("DROP TABLE d4.table1;").close();
+    assertTableNotExists("d4.table1");
+
+    executeString("DROP TABLE IF EXISTS d4.table1");
+    assertTableNotExists("d4.table1");
+  }
+
+  @Test
+  public final void testDelimitedIdentifierWithNonAsciiCharacters() throws Exception {
+
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      ResultSet res = null;
+      try {
+        List<String> tableNames = executeDDL("quoted_identifier_non_ascii_ddl.sql", "table1", "\"테이블1\"");
+        assertTableExists(tableNames.get(0));
+
+        // SELECT "아이디", "텍스트", "숫자" FROM "테이블1";
+        res = executeFile("quoted_identifier_non_ascii_1.sql");
+        assertResultSet(res, "quoted_identifier_non_ascii_1.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      // SELECT "아이디" as "진짜아이디", "텍스트" as text, "숫자" FROM "테이블1" as "테이블 별명"
+      try {
+        res = executeFile("quoted_identifier_non_ascii_2.sql");
+        assertResultSet(res, "quoted_identifier_non_ascii_2.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      // SELECT "아이디" "진짜아이디", char_length("텍스트") as "길이", "숫자" * 2 FROM "테이블1" "테이블 별명"
+      try {
+        res = executeFile("quoted_identifier_non_ascii_3.sql");
+        assertResultSet(res, "quoted_identifier_non_ascii_3.result");
+      } finally {
+        cleanupQuery(res);
+      }
+    }
+  }
+
+  @Test
+  public final void testDelimitedIdentifierWithMixedCharacters() throws Exception {
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      ResultSet res = null;
+
+      try {
+        List<String> tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table1", "\"TABLE1\"");
+        assertTableExists(tableNames.get(0));
+
+        tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table2", "\"tablE1\"");
+        assertTableExists(tableNames.get(0));
+
+        // SELECT "aGe", "tExt", "Number" FROM "TABLE1";
+        res = executeFile("quoted_identifier_mixed_chars_1.sql");
+        assertResultSet(res, "quoted_identifier_mixed_chars_1.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      try {
+        res = executeFile("quoted_identifier_mixed_chars_2.sql");
+        assertResultSet(res, "quoted_identifier_mixed_chars_2.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      try {
+        res = executeFile("quoted_identifier_mixed_chars_3.sql");
+        assertResultSet(res, "quoted_identifier_mixed_chars_3.result");
+      } finally {
+        cleanupQuery(res);
+      }
+    }
+  }
+
+  @Test
+  public final void testNonreservedKeywordTableNames() throws Exception {
+    List<String> createdNames = null;
+    createdNames = executeDDL("table1_ddl.sql", "table1", "filter");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "first");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "format");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "grouping");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "hash");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "index");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "insert");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "last");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "location");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "max");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "min");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "national");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "nullif");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "overwrite");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "precision");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "range");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "regexp");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "rlike");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "set");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "unknown");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "var_pop");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "var_samp");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "varying");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "zone");
+    assertTableExists(createdNames.get(0));
+
+    createdNames = executeDDL("table1_ddl.sql", "table1", "bigint");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "bit");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "blob");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "bool");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "boolean");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "bytea");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "char");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "date");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "decimal");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "double");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "float");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "float4");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "float8");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "inet4");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "int");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "int1");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "int2");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "int4");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "int8");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "integer");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "nchar");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "numeric");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "nvarchar");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "real");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "smallint");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "text");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "time");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "timestamp");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "timestamptz");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "timetz");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "tinyint");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "varbinary");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "varbit");
+    assertTableExists(createdNames.get(0));
+    createdNames = executeDDL("table1_ddl.sql", "table1", "varchar");
+    assertTableExists(createdNames.get(0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java
new file mode 100644
index 0000000..0020156
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+
+@Category(IntegrationTest.class)
+public class TestDropTable extends QueryTestCaseBase {
+
+  @Test
+  public final void testDropManagedTable() throws Exception {
+    List<String> createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "abc");
+    assertTableExists(createdNames.get(0));
+    executeDDL("drop_table_ddl.sql", null);
+    assertTableNotExists("abc");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
new file mode 100644
index 0000000..9e3c375
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
@@ -0,0 +1,246 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+
+@Category(IntegrationTest.class)
+public class TestGroupByQuery extends QueryTestCaseBase {
+
+  public TestGroupByQuery() {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test
+  public final void testGroupBy() throws Exception {
+    // select count(1) as unique_key from lineitem;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupBy2() throws Exception {
+    // select count(1) as unique_key from lineitem group by l_linenumber;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupBy3() throws Exception {
+    // select l_orderkey as gkey from lineitem group by gkey order by gkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupBy4() throws Exception {
+    // select l_orderkey as gkey, count(1) as unique_key from lineitem group by lineitem.l_orderkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupBy5() throws Exception {
+    // select l_orderkey as gkey, '00' as num from lineitem group by lineitem.l_orderkey order by gkey
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupByNested1() throws Exception {
+    // select l_orderkey + l_partkey as unique_key from lineitem group by l_orderkey + l_partkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupByNested2() throws Exception {
+    // select sum(l_orderkey) + sum(l_partkey) as total from lineitem group by l_orderkey + l_partkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupByWithSameExprs1() throws Exception {
+    // select sum(l_orderkey) + sum(l_orderkey) as total from lineitem group by l_orderkey + l_partkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupByWithSameExprs2() throws Exception {
+    // select sum(l_orderkey) as total1, sum(l_orderkey) as total2 from lineitem group by l_orderkey + l_partkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupByWithExpressionKeys1() throws Exception {
+    // select upper(lower(l_orderkey::text)) as key, count(1) as total from lineitem
+    // group by key order by upper(lower(l_orderkey::text)), total;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupByWithExpressionKeys2() throws Exception {
+    // select upper(lower(l_orderkey::text)) as key, count(1) as total from lineitem
+    // group by upper(lower(l_orderkey::text)) order by upper(l_orderkey::text), total;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testGroupByWithConstantKeys1() throws Exception {
+    ResultSet res = executeQuery();
+    System.out.println(resultSetToString(res));
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testDistinctAggregation1() throws Exception {
+    // select l_orderkey, max(l_orderkey) as maximum, count(distinct l_linenumber) as unique_key from lineitem
+    // group by l_orderkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  /**
+   * This is an unit test for a combination of aggregation and distinct aggregation functions.
+   */
+  public final void testDistinctAggregation2() throws Exception {
+    // select l_orderkey, count(*) as cnt, count(distinct l_linenumber) as unique_key from lineitem group by l_orderkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testDistinctAggregation3() throws Exception {
+    // select count(*), count(distinct l_orderkey), sum(distinct l_orderkey) from lineitem;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testDistinctAggregation4() throws Exception {
+    // select l_linenumber, count(*), count(distinct l_orderkey), sum(distinct l_orderkey)
+    // from lineitem group by l_linenumber;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testDistinctAggregation5() throws Exception {
+    // select sum(distinct l_orderkey), l_linenumber, count(distinct l_orderkey), count(*) as total
+    // from lineitem group by l_linenumber;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testDistinctAggregation6() throws Exception {
+    // select count(distinct l_orderkey), sum(l_orderkey), sum(l_linenumber), count(*) as v4 from lineitem
+    // group by l_orderkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testDistinctAggregationWithHaving1() throws Exception {
+    // select l_linenumber, count(*), count(distinct l_orderkey), sum(distinct l_orderkey) from lineitem
+    // group by l_linenumber having sum(distinct l_orderkey) >= 6;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testDistinctAggregationWithUnion1() throws Exception {
+    // select sum(distinct l_orderkey), l_linenumber, count(distinct l_orderkey), count(*) as total
+    // from (select * from lineitem union select * from lineitem) group by l_linenumber;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testComplexParameter() throws Exception {
+    // select sum(l_extendedprice*l_discount) as revenue from lineitem;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testComplexParameterWithSubQuery() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testComplexParameter2() throws Exception {
+    // select count(*) + max(l_orderkey) as merged from lineitem;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testHavingWithNamedTarget() throws Exception {
+    // select l_orderkey, avg(l_partkey) total, sum(l_linenumber) as num from lineitem group by l_orderkey
+    // having total >= 2 or num = 3;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testHavingWithAggFunction() throws Exception {
+    // select l_orderkey, avg(l_partkey) total, sum(l_linenumber) as num from lineitem group by l_orderkey
+    // having avg(l_partkey) = 2.5 or num = 1;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
new file mode 100644
index 0000000..e058943
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
@@ -0,0 +1,262 @@
+/**
+ * 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.query;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.DeflateCodec;
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.TableDesc;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.sql.ResultSet;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.junit.Assert.*;
+
+@Category(IntegrationTest.class)
+public class TestInsertQuery {
+  private static TpchTestBase tpch;
+  public TestInsertQuery() throws IOException {
+    super();
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    tpch = TpchTestBase.getInstance();
+  }
+
+  @Test
+  public final void testInsertOverwrite() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("InsertOverwrite");
+    ResultSet res = tpch.execute("create table " + tableName +" (col1 int4, col2 int4, col3 float8)");
+    res.close();
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = tpch.execute("insert overwrite into " + tableName
+        + " select l_orderkey, l_partkey, l_quantity from lineitem");
+    res.close();
+
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(5, desc.getStats().getNumRows().intValue());
+    }
+  }
+
+  @Test
+  public final void testInsertOverwriteSmallerColumns() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("insertoverwritesmallercolumns");
+    ResultSet res = tpch.execute("create table " + tableName + " (col1 int4, col2 int4, col3 float8)");
+    res.close();
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+    TableDesc originalDesc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+
+    res = tpch.execute("insert overwrite into " + tableName + " select l_orderkey from lineitem");
+    res.close();
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(5, desc.getStats().getNumRows().intValue());
+    }
+    assertEquals(originalDesc.getSchema(), desc.getSchema());
+  }
+
+  @Test
+  public final void testInsertOverwriteWithTargetColumns() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("InsertOverwriteWithTargetColumns");
+    ResultSet res = tpch.execute("create table " + tableName + " (col1 int4, col2 int4, col3 float8)");
+    res.close();
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+    TableDesc originalDesc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+
+    res = tpch.execute(
+        "insert overwrite into " + tableName + " (col1, col3) select l_orderkey, l_quantity from lineitem");
+    res.close();
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(5, desc.getStats().getNumRows().intValue());
+    }
+
+    res = tpch.execute("select * from " + tableName);
+
+    assertTrue(res.next());
+    assertEquals(1, res.getLong(1));
+    assertTrue(0f == res.getFloat(2));
+    assertTrue(res.wasNull());
+    assertTrue(17.0 == res.getFloat(3));
+
+    assertTrue(res.next());
+    assertEquals(1, res.getLong(1));
+    assertTrue(0f == res.getFloat(2));
+    assertTrue(res.wasNull());
+    assertTrue(36.0 == res.getFloat(3));
+
+    assertTrue(res.next());
+    assertEquals(2, res.getLong(1));
+    assertTrue(0f == res.getFloat(2));
+    assertTrue(res.wasNull());
+    assertTrue(38.0 == res.getFloat(3));
+
+    assertTrue(res.next());
+    assertTrue(0f == res.getFloat(2));
+    assertTrue(res.wasNull());
+    assertTrue(45.0 == res.getFloat(3));
+
+    assertTrue(res.next());
+    assertEquals(3, res.getLong(1));
+    assertTrue(0f == res.getFloat(2));
+    assertTrue(res.wasNull());
+    assertTrue(49.0 == res.getFloat(3));
+
+    assertFalse(res.next());
+    res.close();
+
+    assertEquals(originalDesc.getSchema(), desc.getSchema());
+  }
+
+  @Test
+  public final void testInsertOverwriteWithAsterisk() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testinsertoverwritewithasterisk");
+    ResultSet res = tpch.execute("create table " + tableName + " as select * from lineitem");
+    res.close();
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = tpch.execute("insert overwrite into " + tableName + " select * from lineitem where l_orderkey = 3");
+    res.close();
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(2, desc.getStats().getNumRows().intValue());
+    }
+  }
+
+  @Test
+  public final void testInsertOverwriteIntoSelect() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("insertoverwriteintoselect");
+    ResultSet res = tpch.execute(
+        "create table " + tableName + " as select l_orderkey from lineitem");
+    assertFalse(res.next());
+    res.close();
+
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+    TableDesc orderKeys = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(5, orderKeys.getStats().getNumRows().intValue());
+    }
+
+    // this query will result in the two rows.
+    res = tpch.execute(
+        "insert overwrite into " + tableName + " select l_orderkey from lineitem where l_orderkey = 3");
+    assertFalse(res.next());
+    res.close();
+
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+    orderKeys = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(2, orderKeys.getStats().getNumRows().intValue());
+    }
+  }
+
+  @Test
+  public final void testInsertOverwriteCapitalTableName() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testInsertOverwriteCapitalTableName");
+    ResultSet res = tpch.execute("create table " + tableName + " as select * from lineitem");
+    res.close();
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = tpch.execute("insert overwrite into " + tableName + " select * from lineitem where l_orderkey = 3");
+    res.close();
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(2, desc.getStats().getNumRows().intValue());
+    }
+  }
+
+  @Test
+  public final void testInsertOverwriteLocation() throws Exception {
+    ResultSet res =
+        tpch.execute("insert overwrite into location '/tajo-data/testInsertOverwriteCapitalTableName' select * from lineitem where l_orderkey = 3");
+    res.close();
+    FileSystem fs = FileSystem.get(tpch.getTestingCluster().getConfiguration());
+    assertTrue(fs.exists(new Path("/tajo-data/testInsertOverwriteCapitalTableName")));
+    assertEquals(1, fs.listStatus(new Path("/tajo-data/testInsertOverwriteCapitalTableName")).length);
+  }
+
+  @Test
+  public final void testInsertOverwriteWithCompression() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testInsertOverwriteWithCompression");
+    ResultSet res = tpch.execute("create table " + tableName + " (col1 int4, col2 int4, col3 float8) USING csv WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec')");
+    res.close();
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = tpch.execute("insert overwrite into " + tableName + " select  l_orderkey, l_partkey, l_quantity from lineitem where l_orderkey = 3");
+    res.close();
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    if (!cluster.isHCatalogStoreRunning()) {
+      assertEquals(2, desc.getStats().getNumRows().intValue());
+    }
+
+    FileSystem fs = FileSystem.get(tpch.getTestingCluster().getConfiguration());
+    assertTrue(fs.exists(desc.getPath()));
+    CompressionCodecFactory factory = new CompressionCodecFactory(tpch.getTestingCluster().getConfiguration());
+
+    for (FileStatus file : fs.listStatus(desc.getPath())) {
+      CompressionCodec codec = factory.getCodec(file.getPath());
+      assertTrue(codec instanceof DeflateCodec);
+    }
+  }
+
+  @Test
+  public final void testInsertOverwriteLocationWithCompression() throws Exception {
+    ResultSet res = tpch.execute("insert overwrite into location '/tajo-data/testInsertOverwriteLocationWithCompression' USING csv WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') select * from lineitem where l_orderkey = 3");
+    res.close();
+    FileSystem fs = FileSystem.get(tpch.getTestingCluster().getConfiguration());
+    Path path = new Path("/tajo-data/testInsertOverwriteLocationWithCompression");
+    assertTrue(fs.exists(path));
+    assertEquals(1, fs.listStatus(path).length);
+
+    CompressionCodecFactory factory = new CompressionCodecFactory(tpch.getTestingCluster().getConfiguration());
+    for (FileStatus file : fs.listStatus(path)){
+      CompressionCodec codec = factory.getCodec(file.getPath());
+      assertTrue(codec instanceof DeflateCodec);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
new file mode 100644
index 0000000..89519ef
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
@@ -0,0 +1,377 @@
+/**
+ * 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.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.jdbc.TajoResultSet;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.worker.TajoWorker;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
+import static junit.framework.TestCase.fail;
+import static org.junit.Assert.assertNotNull;
+
+@Category(IntegrationTest.class)
+public class TestJoinBroadcast extends QueryTestCaseBase {
+  public TestJoinBroadcast() throws Exception {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+    testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname, "true");
+    testingCluster.setAllTajoDaemonConfValue(
+        TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname, "" + (5 * 1024));
+
+    executeDDL("create_lineitem_large_ddl.sql", "lineitem_large");
+    executeDDL("create_customer_large_ddl.sql", "customer_large");
+  }
+
+  @Test
+  public final void testCrossJoin() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin2() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin3() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin4() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin5() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testWhereClauseJoin6() throws Exception {
+    ResultSet res = executeQuery();
+    System.out.println(resultSetToString(res));
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testTPCHQ2Join() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithConstantExpr1() throws Exception {
+    // outer join with constant projections
+    //
+    // select c_custkey, orders.o_orderkey, 'val' as val from customer
+    // left outer join orders on c_custkey = o_orderkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithConstantExpr2() throws Exception {
+    // outer join with constant projections
+    //
+    // select c_custkey, o.o_orderkey, 'val' as val from customer left outer join
+    // (select * from orders) o on c_custkey = o.o_orderkey
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithConstantExpr3() throws Exception {
+    // outer join with constant projections
+    //
+    // select a.c_custkey, 123::INT8 as const_val, b.min_name from customer a
+    // left outer join ( select c_custkey, min(c_name) as min_name from customer group by c_custkey) b
+    // on a.c_custkey = b.c_custkey;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testRightOuterJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testFullOuterJoin1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testJoinCoReferredEvals1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testJoinCoReferredEvalsWithSameExprs1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testJoinCoReferredEvalsWithSameExprs2() throws Exception {
+    // including grouping operator
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinAndCaseWhen() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk1() throws Exception {
+    // select region.*, customer.* from region, customer;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk2() throws Exception {
+    // select region.*, customer.* from customer, region;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk3() throws Exception {
+    // select * from customer, region
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk4() throws Exception {
+    // select length(r_regionkey), *, c_custkey*10 from customer, region
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testInnerJoinWithEmptyTable() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable2() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable3() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable4() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testRightOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testFullOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testCrossJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testJoinOnMultipleDatabases() throws Exception {
+    executeString("CREATE DATABASE JOINS");
+    assertDatabaseExists("joins");
+    executeString("CREATE TABLE JOINS.part_ as SELECT * FROM part");
+    assertTableExists("joins.part_");
+    executeString("CREATE TABLE JOINS.supplier_ as SELECT * FROM supplier");
+    assertTableExists("joins.supplier_");
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  private MasterPlan getQueryPlan(QueryId queryId) {
+    for (TajoWorker eachWorker: testingCluster.getTajoWorkers()) {
+      QueryMasterTask queryMasterTask = eachWorker.getWorkerContext().getQueryMaster().getQueryMasterTask(queryId, true);
+      if (queryMasterTask != null) {
+        return queryMasterTask.getQuery().getPlan();
+      }
+    }
+
+    fail("Can't find query from workers" + queryId);
+    return null;
+  }
+
+  @Test
+  public final void testBroadcastBasicJoin() throws Exception {
+    ResultSet res = executeQuery();
+    TajoResultSet ts = (TajoResultSet)res;
+    assertResultSet(res);
+    cleanupQuery(res);
+
+    MasterPlan plan = getQueryPlan(ts.getQueryId());
+    ExecutionBlock rootEB = plan.getRoot();
+
+    /*
+    |-eb_1395998037360_0001_000006
+       |-eb_1395998037360_0001_000005
+     */
+    assertEquals(1, plan.getChildCount(rootEB.getId()));
+
+    ExecutionBlock firstEB = plan.getChild(rootEB.getId(), 0);
+
+    assertNotNull(firstEB);
+    assertEquals(2, firstEB.getBroadcastTables().size());
+    assertTrue(firstEB.getBroadcastTables().contains("default.supplier"));
+    assertTrue(firstEB.getBroadcastTables().contains("default.part"));
+  }
+
+  @Test
+  public final void testBroadcastTwoPartJoin() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+
+    MasterPlan plan = getQueryPlan(((TajoResultSet)res).getQueryId());
+    ExecutionBlock rootEB = plan.getRoot();
+
+    /*
+    |-eb_1395996354406_0001_000010
+       |-eb_1395996354406_0001_000009
+          |-eb_1395996354406_0001_000008
+          |-eb_1395996354406_0001_000005
+     */
+    assertEquals(1, plan.getChildCount(rootEB.getId()));
+
+    ExecutionBlock firstJoinEB = plan.getChild(rootEB.getId(), 0);
+    assertNotNull(firstJoinEB);
+    assertEquals(NodeType.JOIN, firstJoinEB.getPlan().getType());
+    assertEquals(0, firstJoinEB.getBroadcastTables().size());
+
+    ExecutionBlock leafEB1 = plan.getChild(firstJoinEB.getId(), 0);
+    assertTrue(leafEB1.getBroadcastTables().contains("default.orders"));
+    assertTrue(leafEB1.getBroadcastTables().contains("default.part"));
+
+    ExecutionBlock leafEB2 = plan.getChild(firstJoinEB.getId(), 1);
+    assertTrue(leafEB2.getBroadcastTables().contains("default.nation"));
+  }
+
+  @Test
+  public final void testBroadcastSubquery() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  // It doesn't run as expected because of TAJO-747 bug.
+  // Thus, we need to block this method until resolving this bug.
+//  @Test
+//  public final void testBroadcastSubquery2() throws Exception {
+//    ResultSet res = executeQuery();
+//    assertResultSet(res);
+//    cleanupQuery(res);
+//  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java
new file mode 100644
index 0000000..3e28f9e
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.query;
+
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.junit.Test;
+
+import java.sql.ResultSet;
+
+public class TestJoinOnPartitionedTables extends QueryTestCaseBase {
+
+  public TestJoinOnPartitionedTables() {
+    super(TajoConstants.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test
+  public void testPartitionTableJoinSmallTable() throws Exception {
+
+    executeDDL("customer_ddl.sql", null);
+    ResultSet res = executeFile("insert_into_customer.sql");
+    res.close();
+
+    res = executeQuery();
+    assertResultSet(res);
+    res.close();
+
+    res = executeFile("selfJoinOfPartitionedTable.sql");
+    assertResultSet(res, "selfJoinOfPartitionedTable.result");
+    res.close();
+
+    res = executeFile("testNoProjectionJoinQual.sql");
+    assertResultSet(res, "testNoProjectionJoinQual.result");
+    res.close();
+
+    res = executeFile("testPartialFilterPushDown.sql");
+    assertResultSet(res, "testPartialFilterPushDown.result");
+    res.close();
+  }
+}


[28/51] [partial] TAJO-752: Escalate sub modules in tajo-core into the top-level modules. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
new file mode 100644
index 0000000..409a1b1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
@@ -0,0 +1,896 @@
+/*
+ * 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.master;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.engine.query.QueryUnitRequest;
+import org.apache.tajo.engine.query.QueryUnitRequestImpl;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext;
+import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryUnit;
+import org.apache.tajo.master.querymaster.QueryUnitAttempt;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.storage.DataLocation;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.URI;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public class DefaultTaskScheduler extends AbstractTaskScheduler {
+  private static final Log LOG = LogFactory.getLog(DefaultTaskScheduler.class);
+
+  private final TaskSchedulerContext context;
+  private SubQuery subQuery;
+
+  private Thread schedulingThread;
+  private AtomicBoolean stopEventHandling = new AtomicBoolean(false);
+
+  private ScheduledRequests scheduledRequests;
+  private TaskRequests taskRequests;
+
+  private int nextTaskId = 0;
+  private int scheduledObjectNum = 0;
+
+  public DefaultTaskScheduler(TaskSchedulerContext context, SubQuery subQuery) {
+    super(DefaultTaskScheduler.class.getName());
+    this.context = context;
+    this.subQuery = subQuery;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+
+    scheduledRequests = new ScheduledRequests();
+    taskRequests  = new TaskRequests();
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Start TaskScheduler");
+
+    this.schedulingThread = new Thread() {
+      public void run() {
+
+        while(!stopEventHandling.get() && !Thread.currentThread().isInterrupted()) {
+          try {
+            synchronized (schedulingThread){
+              schedulingThread.wait(100);
+            }
+          } catch (InterruptedException e) {
+            break;
+          }
+          schedule();
+        }
+        LOG.info("TaskScheduler schedulingThread stopped");
+      }
+    };
+
+    this.schedulingThread.start();
+    super.start();
+  }
+
+  private static final QueryUnitAttemptId NULL_ATTEMPT_ID;
+  public static final TajoWorkerProtocol.QueryUnitRequestProto stopTaskRunnerReq;
+  static {
+    ExecutionBlockId nullSubQuery = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
+    NULL_ATTEMPT_ID = QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(nullSubQuery, 0), 0);
+
+    TajoWorkerProtocol.QueryUnitRequestProto.Builder builder =
+        TajoWorkerProtocol.QueryUnitRequestProto.newBuilder();
+    builder.setId(NULL_ATTEMPT_ID.getProto());
+    builder.setShouldDie(true);
+    builder.setOutputTable("");
+    builder.setSerializedData("");
+    builder.setClusteredOutput(false);
+    stopTaskRunnerReq = builder.build();
+  }
+
+  @Override
+  public void stop() {
+    if(stopEventHandling.getAndSet(true)){
+      return;
+    }
+
+    if (schedulingThread != null) {
+      synchronized (schedulingThread) {
+        schedulingThread.notifyAll();
+      }
+    }
+
+    // Return all of request callbacks instantly.
+    for (TaskRequestEvent req : taskRequests.taskRequestQueue) {
+      req.getCallback().run(stopTaskRunnerReq);
+    }
+
+    LOG.info("Task Scheduler stopped");
+    super.stop();
+  }
+
+  private FileFragment[] fragmentsForNonLeafTask;
+
+  LinkedList<TaskRequestEvent> taskRequestEvents = new LinkedList<TaskRequestEvent>();
+  public void schedule() {
+
+    if (taskRequests.size() > 0) {
+      if (scheduledRequests.leafTaskNum() > 0) {
+        LOG.debug("Try to schedule tasks with taskRequestEvents: " +
+            taskRequests.size() + ", LeafTask Schedule Request: " +
+            scheduledRequests.leafTaskNum());
+        taskRequests.getTaskRequests(taskRequestEvents,
+            scheduledRequests.leafTaskNum());
+        LOG.debug("Get " + taskRequestEvents.size() + " taskRequestEvents ");
+        if (taskRequestEvents.size() > 0) {
+          scheduledRequests.assignToLeafTasks(taskRequestEvents);
+          taskRequestEvents.clear();
+        }
+      }
+    }
+
+    if (taskRequests.size() > 0) {
+      if (scheduledRequests.nonLeafTaskNum() > 0) {
+        LOG.debug("Try to schedule tasks with taskRequestEvents: " +
+            taskRequests.size() + ", NonLeafTask Schedule Request: " +
+            scheduledRequests.nonLeafTaskNum());
+        taskRequests.getTaskRequests(taskRequestEvents,
+            scheduledRequests.nonLeafTaskNum());
+        scheduledRequests.assignToNonLeafTasks(taskRequestEvents);
+        taskRequestEvents.clear();
+      }
+    }
+  }
+
+  @Override
+  public void handle(TaskSchedulerEvent event) {
+    if (event.getType() == EventType.T_SCHEDULE) {
+      if (event instanceof FragmentScheduleEvent) {
+        FragmentScheduleEvent castEvent = (FragmentScheduleEvent) event;
+        if (context.isLeafQuery()) {
+          QueryUnitAttemptScheduleContext queryUnitContext = new QueryUnitAttemptScheduleContext();
+          QueryUnit task = SubQuery.newEmptyQueryUnit(context, queryUnitContext, subQuery, nextTaskId++);
+          task.addFragment(castEvent.getLeftFragment(), true);
+          scheduledObjectNum++;
+          if (castEvent.hasRightFragments()) {
+            task.addFragments(castEvent.getRightFragments());
+            //scheduledObjectNum += castEvent.getRightFragments().size();
+          }
+          subQuery.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+        } else {
+          fragmentsForNonLeafTask = new FileFragment[2];
+          fragmentsForNonLeafTask[0] = castEvent.getLeftFragment();
+          if (castEvent.hasRightFragments()) {
+            fragmentsForNonLeafTask[1] = castEvent.getRightFragments().toArray(new FileFragment[]{})[0];
+          }
+        }
+      } else if (event instanceof FetchScheduleEvent) {
+        FetchScheduleEvent castEvent = (FetchScheduleEvent) event;
+        Map<String, List<URI>> fetches = castEvent.getFetches();
+        QueryUnitAttemptScheduleContext queryUnitContext = new QueryUnitAttemptScheduleContext();
+        QueryUnit task = SubQuery.newEmptyQueryUnit(context, queryUnitContext, subQuery, nextTaskId++);
+        scheduledObjectNum++;
+        for (Entry<String, List<URI>> eachFetch : fetches.entrySet()) {
+          task.addFetches(eachFetch.getKey(), eachFetch.getValue());
+          task.addFragment(fragmentsForNonLeafTask[0], true);
+          if (fragmentsForNonLeafTask[1] != null) {
+            task.addFragment(fragmentsForNonLeafTask[1], true);
+          }
+        }
+        subQuery.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+      } else if (event instanceof QueryUnitAttemptScheduleEvent) {
+        QueryUnitAttemptScheduleEvent castEvent = (QueryUnitAttemptScheduleEvent) event;
+        if (context.isLeafQuery()) {
+          scheduledRequests.addLeafTask(castEvent);
+        } else {
+          scheduledRequests.addNonLeafTask(castEvent);
+        }
+      }
+    } else if (event.getType() == EventType.T_SCHEDULE_CANCEL) {
+      // when a subquery is killed, unassigned query unit attmpts are canceled from the scheduler.
+      // This event is triggered by QueryUnitAttempt.
+      QueryUnitAttemptScheduleEvent castedEvent = (QueryUnitAttemptScheduleEvent) event;
+      scheduledRequests.leafTasks.remove(castedEvent.getQueryUnitAttempt().getId());
+      LOG.info(castedEvent.getQueryUnitAttempt().getId() + " is canceled from " + this.getClass().getSimpleName());
+      ((QueryUnitAttemptScheduleEvent) event).getQueryUnitAttempt().handle(
+          new TaskAttemptEvent(castedEvent.getQueryUnitAttempt().getId(), TaskAttemptEventType.TA_SCHEDULE_CANCELED));
+    }
+  }
+
+  @Override
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+
+    taskRequests.handle(event);
+    int hosts = scheduledRequests.leafTaskHostMapping.size();
+
+    // if available cluster resource are large then tasks, the scheduler thread are working immediately.
+    if(remainingScheduledObjectNum() > 0 &&
+        (remainingScheduledObjectNum() <= hosts || hosts <= taskRequests.size())){
+      synchronized (schedulingThread){
+        schedulingThread.notifyAll();
+      }
+    }
+  }
+
+  @Override
+  public int remainingScheduledObjectNum() {
+    return scheduledObjectNum;
+  }
+
+  private class TaskRequests implements EventHandler<TaskRequestEvent> {
+    private final LinkedBlockingQueue<TaskRequestEvent> taskRequestQueue =
+        new LinkedBlockingQueue<TaskRequestEvent>();
+
+    @Override
+    public void handle(TaskRequestEvent event) {
+      if(LOG.isDebugEnabled()){
+        LOG.debug("TaskRequest: " + event.getContainerId() + "," + event.getExecutionBlockId());
+      }
+
+      if(stopEventHandling.get()) {
+        event.getCallback().run(stopTaskRunnerReq);
+        return;
+      }
+      int qSize = taskRequestQueue.size();
+      if (qSize != 0 && qSize % 1000 == 0) {
+        LOG.info("Size of event-queue in YarnRMContainerAllocator is " + qSize);
+      }
+      int remCapacity = taskRequestQueue.remainingCapacity();
+      if (remCapacity < 1000) {
+        LOG.warn("Very low remaining capacity in the event-queue "
+            + "of YarnRMContainerAllocator: " + remCapacity);
+      }
+
+      taskRequestQueue.add(event);
+    }
+
+    public void getTaskRequests(final Collection<TaskRequestEvent> taskRequests,
+                                int num) {
+      taskRequestQueue.drainTo(taskRequests, num);
+    }
+
+    public int size() {
+      return taskRequestQueue.size();
+    }
+  }
+
+  /**
+   * One worker can have multiple running task runners. <code>HostVolumeMapping</code>
+   * describes various information for one worker, including :
+   * <ul>
+   *  <li>host name</li>
+   *  <li>rack name</li>
+   *  <li>unassigned tasks for each disk volume</li>
+   *  <li>last assigned volume id - it can be used for assigning task in a round-robin manner</li>
+   *  <li>the number of running tasks for each volume</li>
+   * </ul>, each task runner and the concurrency number of running tasks for volumes.
+   *
+   * Here, we identifier a task runner by {@link ContainerId}, and we use volume ids to identify
+   * all disks in this node. Actually, each volume is only used to distinguish disks, and we don't
+   * know a certain volume id indicates a certain disk. If you want to know volume id, please read the below section.
+   *
+   * <h3>Volume id</h3>
+   * Volume id is an integer. Each volume id identifies each disk volume.
+   *
+   * This volume id can be obtained from org.apache.hadoop.fs.BlockStorageLocation#getVolumeIds()}.   *
+   * HDFS cannot give any volume id due to unknown reason and disabled config 'dfs.client.file-block-locations.enabled'.
+   * In this case, the volume id will be -1 or other native integer.
+   *
+   * <h3>See Also</h3>
+   * <ul>
+   *   <li>HDFS-3672 (https://issues.apache.org/jira/browse/HDFS-3672).</li>
+   * </ul>
+   */
+  public class HostVolumeMapping {
+    private final String host;
+    private final String rack;
+    /** A key is disk volume, and a value is a list of tasks to be scheduled. */
+    private Map<Integer, LinkedHashSet<QueryUnitAttempt>> unassignedTaskForEachVolume =
+        Collections.synchronizedMap(new HashMap<Integer, LinkedHashSet<QueryUnitAttempt>>());
+    /** A value is last assigned volume id for each task runner */
+    private HashMap<ContainerId, Integer> lastAssignedVolumeId = new HashMap<ContainerId, Integer>();
+    /**
+     * A key is disk volume id, and a value is the load of this volume.
+     * This load is measured by counting how many number of tasks are running.
+     *
+     * These disk volumes are kept in an order of ascending order of the volume id.
+     * In other words, the head volume ids are likely to -1, meaning no given volume id.
+     */
+    private SortedMap<Integer, Integer> diskVolumeLoads = new TreeMap<Integer, Integer>();
+    /** The total number of remain tasks in this host */
+    private AtomicInteger remainTasksNum = new AtomicInteger(0);
+    public static final int REMOTE = -2;
+
+
+    public HostVolumeMapping(String host, String rack){
+      this.host = host;
+      this.rack = rack;
+    }
+
+    public synchronized void addQueryUnitAttempt(int volumeId, QueryUnitAttempt attemptId){
+      synchronized (unassignedTaskForEachVolume){
+        LinkedHashSet<QueryUnitAttempt> list = unassignedTaskForEachVolume.get(volumeId);
+        if (list == null) {
+          list = new LinkedHashSet<QueryUnitAttempt>();
+          unassignedTaskForEachVolume.put(volumeId, list);
+        }
+        list.add(attemptId);
+      }
+
+      remainTasksNum.incrementAndGet();
+
+      if(!diskVolumeLoads.containsKey(volumeId)) diskVolumeLoads.put(volumeId, 0);
+    }
+
+    /**
+     *  Priorities
+     *  1. a task list in a volume of host
+     *  2. unknown block or Non-splittable task in host
+     *  3. remote tasks. unassignedTaskForEachVolume is only contained local task. so it will be null
+     */
+    public synchronized QueryUnitAttemptId getLocalTask(ContainerId containerId) {
+      int volumeId;
+      QueryUnitAttemptId queryUnitAttemptId = null;
+
+      if (!lastAssignedVolumeId.containsKey(containerId)) {
+        volumeId = getLowestVolumeId();
+        increaseConcurrency(containerId, volumeId);
+      } else {
+        volumeId = lastAssignedVolumeId.get(containerId);
+      }
+
+      if (unassignedTaskForEachVolume.size() >  0) {
+        int retry = unassignedTaskForEachVolume.size();
+        do {
+          //clean and get a remaining local task
+          queryUnitAttemptId = getAndRemove(volumeId);
+          if(!unassignedTaskForEachVolume.containsKey(volumeId)) {
+            decreaseConcurrency(containerId);
+            if (volumeId > REMOTE) {
+              diskVolumeLoads.remove(volumeId);
+            }
+          }
+
+          if (queryUnitAttemptId == null) {
+            //reassign next volume
+            volumeId = getLowestVolumeId();
+            increaseConcurrency(containerId, volumeId);
+            retry--;
+          } else {
+            break;
+          }
+        } while (retry > 0);
+      } else {
+        this.remainTasksNum.set(0);
+      }
+      return queryUnitAttemptId;
+    }
+
+    public synchronized QueryUnitAttemptId getQueryUnitAttemptIdByRack(String rack) {
+      QueryUnitAttemptId queryUnitAttemptId = null;
+
+      if (unassignedTaskForEachVolume.size() > 0 && this.rack.equals(rack)) {
+        int retry = unassignedTaskForEachVolume.size();
+        do {
+          //clean and get a remaining task
+          int volumeId = getLowestVolumeId();
+          queryUnitAttemptId = getAndRemove(volumeId);
+          if (queryUnitAttemptId == null) {
+            if (volumeId > REMOTE) {
+              diskVolumeLoads.remove(volumeId);
+            }
+            retry--;
+          } else {
+            break;
+          }
+        } while (retry > 0);
+      }
+      return queryUnitAttemptId;
+    }
+
+    private synchronized QueryUnitAttemptId getAndRemove(int volumeId){
+      QueryUnitAttemptId queryUnitAttemptId = null;
+      if(!unassignedTaskForEachVolume.containsKey(volumeId)) return queryUnitAttemptId;
+
+      LinkedHashSet<QueryUnitAttempt> list = unassignedTaskForEachVolume.get(volumeId);
+      if(list != null && list.size() > 0){
+        QueryUnitAttempt queryUnitAttempt;
+        synchronized (unassignedTaskForEachVolume) {
+          Iterator<QueryUnitAttempt> iterator = list.iterator();
+          queryUnitAttempt = iterator.next();
+          iterator.remove();
+        }
+
+        this.remainTasksNum.getAndDecrement();
+        queryUnitAttemptId = queryUnitAttempt.getId();
+        for (DataLocation location : queryUnitAttempt.getQueryUnit().getDataLocations()) {
+          if (!this.getHost().equals(location.getHost())) {
+            HostVolumeMapping volumeMapping = scheduledRequests.leafTaskHostMapping.get(location.getHost());
+            volumeMapping.removeQueryUnitAttempt(location.getVolumeId(), queryUnitAttempt);
+          }
+        }
+      }
+
+      if(list == null || list.isEmpty()) {
+        unassignedTaskForEachVolume.remove(volumeId);
+      }
+      return queryUnitAttemptId;
+    }
+
+    private synchronized void removeQueryUnitAttempt(int volumeId, QueryUnitAttempt queryUnitAttempt){
+      if(!unassignedTaskForEachVolume.containsKey(volumeId)) return;
+
+      LinkedHashSet<QueryUnitAttempt> tasks  = unassignedTaskForEachVolume.get(volumeId);
+
+      if(tasks != null && tasks.size() > 0){
+        tasks.remove(queryUnitAttempt);
+        remainTasksNum.getAndDecrement();
+      } else {
+        unassignedTaskForEachVolume.remove(volumeId);
+      }
+    }
+
+    /**
+     * Increase the count of running tasks and disk loads for a certain task runner.
+     *
+     * @param containerId The task runner identifier
+     * @param volumeId Volume identifier
+     * @return the volume load (i.e., how many running tasks use this volume)
+     */
+    private synchronized int increaseConcurrency(ContainerId containerId, int volumeId) {
+
+      int concurrency = 1;
+      if (diskVolumeLoads.containsKey(volumeId)) {
+        concurrency = diskVolumeLoads.get(volumeId) + 1;
+      }
+
+      if (volumeId > -1) {
+        LOG.info("Assigned host : " + host + ", Volume : " + volumeId + ", Concurrency : " + concurrency);
+      } else if (volumeId == -1) {
+        // this case is disabled namenode block meta or compressed text file or amazon s3
+        LOG.info("Assigned host : " + host + ", Unknown Volume : " + volumeId + ", Concurrency : " + concurrency);
+      } else if (volumeId == REMOTE) {
+        // this case has processed all block on host and it will be assigned to remote
+        LOG.info("Assigned host : " + host + ", Remaining local tasks : " + getRemainingLocalTaskSize()
+            + ", Remote Concurrency : " + concurrency);
+      }
+      diskVolumeLoads.put(volumeId, concurrency);
+      lastAssignedVolumeId.put(containerId, volumeId);
+      return concurrency;
+    }
+
+    /**
+     * Decrease the count of running tasks of a certain task runner
+     */
+    private synchronized void decreaseConcurrency(ContainerId containerId){
+      Integer volumeId = lastAssignedVolumeId.get(containerId);
+      if(volumeId != null && diskVolumeLoads.containsKey(volumeId)){
+        Integer concurrency = diskVolumeLoads.get(volumeId);
+        if(concurrency > 0){
+          diskVolumeLoads.put(volumeId, concurrency - 1);
+        } else {
+          if (volumeId > REMOTE) {
+            diskVolumeLoads.remove(volumeId);
+          }
+        }
+      }
+      lastAssignedVolumeId.remove(containerId);
+    }
+
+    /**
+     *  volume of a host : 0 ~ n
+     *  compressed task, amazon s3, unKnown volume : -1
+     *  remote task : -2
+     */
+    public int getLowestVolumeId(){
+      Map.Entry<Integer, Integer> volumeEntry = null;
+
+      for (Map.Entry<Integer, Integer> entry : diskVolumeLoads.entrySet()) {
+        if(volumeEntry == null) volumeEntry = entry;
+
+        if (volumeEntry.getValue() >= entry.getValue()) {
+          volumeEntry = entry;
+        }
+      }
+
+      if(volumeEntry != null){
+        return volumeEntry.getKey();
+      } else {
+        return REMOTE;
+      }
+    }
+
+    public boolean isAssigned(ContainerId containerId){
+      return lastAssignedVolumeId.containsKey(containerId);
+    }
+
+    public boolean isRemote(ContainerId containerId){
+      Integer volumeId = lastAssignedVolumeId.get(containerId);
+      if(volumeId == null || volumeId > REMOTE){
+        return false;
+      } else {
+        return true;
+      }
+    }
+
+    public int getRemoteConcurrency(){
+      return getVolumeConcurrency(REMOTE);
+    }
+
+    public int getVolumeConcurrency(int volumeId){
+      Integer size = diskVolumeLoads.get(volumeId);
+      if(size == null) return 0;
+      else return size;
+    }
+
+    public int getRemainingLocalTaskSize(){
+      return remainTasksNum.get();
+    }
+
+    public String getHost() {
+
+      return host;
+    }
+
+    public String getRack() {
+      return rack;
+    }
+  }
+
+  private class ScheduledRequests {
+    // two list leafTasks and nonLeafTasks keep all tasks to be scheduled. Even though some task is included in
+    // leafTaskHostMapping or leafTasksRackMapping, some task T will not be sent to a task runner
+    // if the task is not included in leafTasks and nonLeafTasks.
+    private final Set<QueryUnitAttemptId> leafTasks = Collections.synchronizedSet(new HashSet<QueryUnitAttemptId>());
+    private final Set<QueryUnitAttemptId> nonLeafTasks = Collections.synchronizedSet(new HashSet<QueryUnitAttemptId>());
+    private Map<String, HostVolumeMapping> leafTaskHostMapping = new HashMap<String, HostVolumeMapping>();
+    private final Map<String, HashSet<QueryUnitAttemptId>> leafTasksRackMapping =
+        new HashMap<String, HashSet<QueryUnitAttemptId>>();
+
+    private void addLeafTask(QueryUnitAttemptScheduleEvent event) {
+      QueryUnitAttempt queryUnitAttempt = event.getQueryUnitAttempt();
+      List<DataLocation> locations = queryUnitAttempt.getQueryUnit().getDataLocations();
+
+      for (DataLocation location : locations) {
+        String host = location.getHost();
+
+        HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
+        if (hostVolumeMapping == null) {
+          String rack = RackResolver.resolve(host).getNetworkLocation();
+          hostVolumeMapping = new HostVolumeMapping(host, rack);
+          leafTaskHostMapping.put(host, hostVolumeMapping);
+        }
+        hostVolumeMapping.addQueryUnitAttempt(location.getVolumeId(), queryUnitAttempt);
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Added attempt req to host " + host);
+        }
+
+        HashSet<QueryUnitAttemptId> list = leafTasksRackMapping.get(hostVolumeMapping.getRack());
+        if (list == null) {
+          list = new HashSet<QueryUnitAttemptId>();
+          leafTasksRackMapping.put(hostVolumeMapping.getRack(), list);
+        }
+
+        list.add(queryUnitAttempt.getId());
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Added attempt req to rack " + hostVolumeMapping.getRack());
+        }
+      }
+
+      leafTasks.add(queryUnitAttempt.getId());
+    }
+
+    private void addNonLeafTask(QueryUnitAttemptScheduleEvent event) {
+      nonLeafTasks.add(event.getQueryUnitAttempt().getId());
+    }
+
+    public int leafTaskNum() {
+      return leafTasks.size();
+    }
+
+    public int nonLeafTaskNum() {
+      return nonLeafTasks.size();
+    }
+
+    public Set<QueryUnitAttemptId> assignedRequest = new HashSet<QueryUnitAttemptId>();
+
+    private QueryUnitAttemptId allocateLocalTask(String host, ContainerId containerId){
+      HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
+
+      if (hostVolumeMapping != null) { //tajo host is located in hadoop datanode
+        while (hostVolumeMapping.getRemainingLocalTaskSize() > 0) {
+          QueryUnitAttemptId attemptId = hostVolumeMapping.getLocalTask(containerId);
+          //find remaining local task
+          if (leafTasks.contains(attemptId)) {
+            leafTasks.remove(attemptId);
+            //LOG.info(attemptId + " Assigned based on host match " + hostName);
+            hostLocalAssigned++;
+            totalAssigned++;
+            return attemptId;
+          }
+        }
+      }
+      return null;
+    }
+
+    private QueryUnitAttemptId allocateRackTask(String host) {
+
+      List<HostVolumeMapping> remainingTasks = new ArrayList<HostVolumeMapping>(leafTaskHostMapping.values());
+      String rack = RackResolver.resolve(host).getNetworkLocation();
+      QueryUnitAttemptId attemptId = null;
+
+      if (remainingTasks.size() > 0) {
+        //find largest remaining task of other host in rack
+        Collections.sort(remainingTasks, new Comparator<HostVolumeMapping>() {
+          @Override
+          public int compare(HostVolumeMapping v1, HostVolumeMapping v2) {
+            // descending remaining tasks
+            return Integer.valueOf(v2.remainTasksNum.get()).compareTo(Integer.valueOf(v1.remainTasksNum.get()));
+          }
+        });
+
+        for (HostVolumeMapping tasks : remainingTasks) {
+          while (tasks.getRemainingLocalTaskSize() > 0){
+            QueryUnitAttemptId tId = tasks.getQueryUnitAttemptIdByRack(rack);
+
+            if (tId == null) break;
+
+            if (leafTasks.contains(tId)) {
+              leafTasks.remove(tId);
+              attemptId = tId;
+              break;
+            }
+          }
+          if(attemptId != null) break;
+        }
+      }
+
+      //find task in rack
+      if (attemptId == null) {
+        HashSet<QueryUnitAttemptId> list = leafTasksRackMapping.get(rack);
+        if (list != null) {
+          synchronized (list) {
+            Iterator<QueryUnitAttemptId> iterator = list.iterator();
+            while (iterator.hasNext()) {
+              QueryUnitAttemptId tId = iterator.next();
+              iterator.remove();
+              if (leafTasks.contains(tId)) {
+                leafTasks.remove(tId);
+                attemptId = tId;
+                break;
+              }
+            }
+          }
+        }
+      }
+
+      if (attemptId != null) {
+        rackLocalAssigned++;
+        totalAssigned++;
+
+        LOG.info(String.format("Assigned Local/Rack/Total: (%d/%d/%d), Locality: %.2f%%, Rack host: %s",
+            hostLocalAssigned, rackLocalAssigned, totalAssigned,
+            ((double) hostLocalAssigned / (double) totalAssigned) * 100, host));
+
+      }
+      return attemptId;
+    }
+
+    public void assignToLeafTasks(LinkedList<TaskRequestEvent> taskRequests) {
+      Collections.shuffle(taskRequests);
+      LinkedList<TaskRequestEvent> remoteTaskRequests = new LinkedList<TaskRequestEvent>();
+
+      TaskRequestEvent taskRequest;
+      while (leafTasks.size() > 0 && (!taskRequests.isEmpty() || !remoteTaskRequests.isEmpty())) {
+        taskRequest = taskRequests.pollFirst();
+        if(taskRequest == null) { // if there are only remote task requests
+          taskRequest = remoteTaskRequests.pollFirst();
+        }
+
+        // checking if this container is still alive.
+        // If not, ignore the task request and stop the task runner
+        ContainerProxy container = context.getMasterContext().getResourceAllocator()
+            .getContainer(taskRequest.getContainerId());
+        if(container == null) {
+          taskRequest.getCallback().run(stopTaskRunnerReq);
+          continue;
+        }
+
+        // getting the hostname of requested node
+        String host = container.getTaskHostName();
+
+        // if there are no worker matched to the hostname a task request
+        if(!leafTaskHostMapping.containsKey(host)){
+          host = NetUtils.normalizeHost(host);
+
+          if(!leafTaskHostMapping.containsKey(host) && !taskRequests.isEmpty()){
+            // this case means one of either cases:
+            // * there are no blocks which reside in this node.
+            // * all blocks which reside in this node are consumed, and this task runner requests a remote task.
+            // In this case, we transfer the task request to the remote task request list, and skip the followings.
+            remoteTaskRequests.add(taskRequest);
+            continue;
+          }
+        }
+
+        ContainerId containerId = taskRequest.getContainerId();
+        LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," +
+            "containerId=" + containerId);
+
+        //////////////////////////////////////////////////////////////////////
+        // disk or host-local allocation
+        //////////////////////////////////////////////////////////////////////
+        QueryUnitAttemptId attemptId = allocateLocalTask(host, containerId);
+
+        if (attemptId == null) { // if a local task cannot be found
+          HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
+
+          if(hostVolumeMapping != null) {
+            if(!hostVolumeMapping.isRemote(containerId)){
+              // assign to remote volume
+              hostVolumeMapping.decreaseConcurrency(containerId);
+              hostVolumeMapping.increaseConcurrency(containerId, HostVolumeMapping.REMOTE);
+            }
+            // this part is remote concurrency management of a tail tasks
+            int tailLimit = Math.max(remainingScheduledObjectNum() / (leafTaskHostMapping.size() * 2), 1);
+
+            if(hostVolumeMapping.getRemoteConcurrency() > tailLimit){
+              //release container
+              hostVolumeMapping.decreaseConcurrency(containerId);
+              taskRequest.getCallback().run(stopTaskRunnerReq);
+              subQuery.releaseContainer(containerId);
+              continue;
+            }
+          }
+
+          //////////////////////////////////////////////////////////////////////
+          // rack-local allocation
+          //////////////////////////////////////////////////////////////////////
+          attemptId = allocateRackTask(host);
+
+          //////////////////////////////////////////////////////////////////////
+          // random node allocation
+          //////////////////////////////////////////////////////////////////////
+          if (attemptId == null && leafTaskNum() > 0) {
+            synchronized (leafTasks){
+              attemptId = leafTasks.iterator().next();
+              leafTasks.remove(attemptId);
+              rackLocalAssigned++;
+              totalAssigned++;
+              LOG.info(String.format("Assigned Local/Remote/Total: (%d/%d/%d), Locality: %.2f%%,",
+                  hostLocalAssigned, rackLocalAssigned, totalAssigned,
+                  ((double) hostLocalAssigned / (double) totalAssigned) * 100));
+            }
+          }
+        }
+
+        if (attemptId != null) {
+          QueryUnit task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
+          QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
+              attemptId,
+              new ArrayList<FragmentProto>(task.getAllFragments()),
+              "",
+              false,
+              task.getLogicalPlan().toJson(),
+              context.getMasterContext().getQueryContext(),
+              subQuery.getDataChannel(), subQuery.getBlock().getEnforcer());
+          if (checkIfInterQuery(subQuery.getMasterPlan(), subQuery.getBlock())) {
+            taskAssign.setInterQuery();
+          }
+
+          context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
+              taskRequest.getContainerId(),
+              host, container.getTaskPort()));
+          assignedRequest.add(attemptId);
+
+          scheduledObjectNum -= task.getAllFragments().size();
+          taskRequest.getCallback().run(taskAssign.getProto());
+        } else {
+          throw new RuntimeException("Illegal State!!!!!!!!!!!!!!!!!!!!!");
+        }
+      }
+    }
+
+    private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) {
+      if (masterPlan.isRoot(block)) {
+        return false;
+      }
+
+      ExecutionBlock parent = masterPlan.getParent(block);
+      if (masterPlan.isRoot(parent) && parent.hasUnion()) {
+        return false;
+      }
+
+      return true;
+    }
+
+    public void assignToNonLeafTasks(LinkedList<TaskRequestEvent> taskRequests) {
+      Collections.shuffle(taskRequests);
+
+      TaskRequestEvent taskRequest;
+      while (!taskRequests.isEmpty()) {
+        taskRequest = taskRequests.pollFirst();
+        LOG.debug("assignToNonLeafTasks: " + taskRequest.getExecutionBlockId());
+
+        QueryUnitAttemptId attemptId;
+        // random allocation
+        if (nonLeafTasks.size() > 0) {
+          synchronized (nonLeafTasks){
+            attemptId = nonLeafTasks.iterator().next();
+            nonLeafTasks.remove(attemptId);
+          }
+          LOG.debug("Assigned based on * match");
+
+          QueryUnit task;
+          task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
+          QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
+              attemptId,
+              Lists.newArrayList(task.getAllFragments()),
+              "",
+              false,
+              task.getLogicalPlan().toJson(),
+              context.getMasterContext().getQueryContext(),
+              subQuery.getDataChannel(),
+              subQuery.getBlock().getEnforcer());
+          if (checkIfInterQuery(subQuery.getMasterPlan(), subQuery.getBlock())) {
+            taskAssign.setInterQuery();
+          }
+          for (ScanNode scan : task.getScanNodes()) {
+            Collection<URI> fetches = task.getFetch(scan);
+            if (fetches != null) {
+              for (URI fetch : fetches) {
+                taskAssign.addFetch(scan.getTableName(), fetch);
+              }
+            }
+          }
+
+          ContainerProxy container = context.getMasterContext().getResourceAllocator().getContainer(
+              taskRequest.getContainerId());
+          context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
+              taskRequest.getContainerId(), container.getTaskHostName(), container.getTaskPort()));
+          taskRequest.getCallback().run(taskAssign.getProto());
+          totalAssigned++;
+          scheduledObjectNum--;
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java
new file mode 100644
index 0000000..561f980
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.event.TaskSchedulerEvent;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+
+public class FetchScheduleEvent extends TaskSchedulerEvent {
+  private final Map<String, List<URI>> fetches;
+
+  public FetchScheduleEvent(final EventType eventType, final ExecutionBlockId blockId,
+                            final Map<String, List<URI>> fetches) {
+    super(eventType, blockId);
+    this.fetches = fetches;
+  }
+
+  public Map<String, List<URI>> getFetches() {
+    return fetches;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java b/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
new file mode 100644
index 0000000..598b1c5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
@@ -0,0 +1,73 @@
+/*
+ * 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.master;
+
+import com.google.common.base.Objects;
+import org.apache.tajo.storage.fragment.FileFragment;
+
+/**
+ * FragmentPair consists of two fragments, a left fragment and a right fragment.
+ * According to queries, it can have the different values.
+ * For join queries, it is assumed to have both fragments.
+ * Also, the left fragment is assumed to be a fragment of the larger table.
+ * For other queries, it is assumed to have only a left fragment.
+ */
+public class FragmentPair {
+  private FileFragment leftFragment;
+  private FileFragment rightFragment;
+
+  public FragmentPair(FileFragment left) {
+    this.leftFragment = left;
+  }
+
+  public FragmentPair(FileFragment left, FileFragment right) {
+    this.leftFragment = left;
+    this.rightFragment = right;
+  }
+
+  public FileFragment getLeftFragment() {
+    return leftFragment;
+  }
+
+  public FileFragment getRightFragment() {
+    return rightFragment;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof FragmentPair) {
+      FragmentPair other = (FragmentPair) o;
+      boolean eq = this.leftFragment.equals(other.leftFragment);
+      if (this.rightFragment != null && other.rightFragment != null) {
+        eq &= this.rightFragment.equals(other.rightFragment);
+      } else if (this.rightFragment == null && other.rightFragment == null) {
+        eq &= true;
+      } else {
+        return false;
+      }
+      return eq;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(leftFragment, rightFragment);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithm.java
new file mode 100644
index 0000000..10d993d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithm.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+/**
+ * FragmentScheduleAlgorithm is used by LazyTaskScheduler.
+ * FragmentScheduleAlgorithm selects a fragment for the given argument.
+ *
+ * There are two implementations of DefaultFragmentScheduleAlgorithm and GreedyFragmentScheduleAlgorithm.
+ */
+public interface FragmentScheduleAlgorithm {
+  void addFragment(FragmentPair fragmentPair);
+  void removeFragment(FragmentPair fragmentPair);
+
+  FragmentPair getHostLocalFragment(String host);
+  FragmentPair getHostLocalFragment(String host, Integer diskId);
+  FragmentPair getRackLocalFragment(String host);
+  FragmentPair getRandomFragment();
+  FragmentPair[] getAllFragments();
+
+  int size();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithmFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithmFactory.java b/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithmFactory.java
new file mode 100644
index 0000000..820a0fb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/FragmentScheduleAlgorithmFactory.java
@@ -0,0 +1,68 @@
+/*
+ * 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.master;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.Map;
+
+public class FragmentScheduleAlgorithmFactory {
+
+  private static Class<? extends FragmentScheduleAlgorithm> CACHED_ALGORITHM_CLASS;
+  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
+  private static final Class<?>[] DEFAULT_PARAMS = {};
+
+  public static Class<? extends FragmentScheduleAlgorithm> getScheduleAlgorithmClass(Configuration conf)
+      throws IOException {
+    if (CACHED_ALGORITHM_CLASS != null) {
+      return CACHED_ALGORITHM_CLASS;
+    } else {
+      CACHED_ALGORITHM_CLASS = conf.getClass("tajo.querymaster.lazy-task-scheduler.algorithm", null,
+          FragmentScheduleAlgorithm.class);
+    }
+
+    if (CACHED_ALGORITHM_CLASS == null) {
+      throw new IOException("Scheduler algorithm is null");
+    }
+    return CACHED_ALGORITHM_CLASS;
+  }
+
+  public static <T extends FragmentScheduleAlgorithm> T get(Class<T> clazz) {
+    T result;
+    try {
+      Constructor<T> constructor = (Constructor<T>) CONSTRUCTOR_CACHE.get(clazz);
+      if (constructor == null) {
+        constructor = clazz.getDeclaredConstructor(DEFAULT_PARAMS);
+        constructor.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(clazz, constructor);
+      }
+      result = constructor.newInstance(new Object[]{});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return result;
+  }
+
+  public static FragmentScheduleAlgorithm get(Configuration conf) throws IOException {
+    return get(getScheduleAlgorithmClass(conf));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/6594ac1c/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
new file mode 100644
index 0000000..8acf2b2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -0,0 +1,738 @@
+/**
+ * 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.master;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.algebra.AlterTablespaceSetType;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.exception.*;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.exception.IllegalQueryStatusException;
+import org.apache.tajo.engine.exception.VerifyException;
+import org.apache.tajo.engine.parser.HiveQLAnalyzer;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.master.TajoMaster.MasterContext;
+import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.storage.*;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto;
+import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto.AlterTablespaceCommand;
+import static org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
+import static org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse.SerializedResultSet;
+
+public class GlobalEngine extends AbstractService {
+  /** Class Logger */
+  private final static Log LOG = LogFactory.getLog(GlobalEngine.class);
+
+  private final MasterContext context;
+  private final AbstractStorageManager sm;
+
+  private SQLAnalyzer analyzer;
+  private HiveQLAnalyzer converter;
+  private CatalogService catalog;
+  private PreLogicalPlanVerifier preVerifier;
+  private LogicalPlanner planner;
+  private LogicalOptimizer optimizer;
+  private LogicalPlanVerifier annotatedPlanVerifier;
+  private DistributedQueryHookManager hookManager;
+
+  public GlobalEngine(final MasterContext context) {
+    super(GlobalEngine.class.getName());
+    this.context = context;
+    this.catalog = context.getCatalog();
+    this.sm = context.getStorageManager();
+  }
+
+  public void start() {
+    try  {
+      analyzer = new SQLAnalyzer();
+      converter = new HiveQLAnalyzer();
+      preVerifier = new PreLogicalPlanVerifier(context.getCatalog());
+      planner = new LogicalPlanner(context.getCatalog());
+      optimizer = new LogicalOptimizer(context.getConf());
+      annotatedPlanVerifier = new LogicalPlanVerifier(context.getConf(), context.getCatalog());
+
+      hookManager = new DistributedQueryHookManager();
+      hookManager.addHook(new CreateTableHook());
+      hookManager.addHook(new InsertHook());
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+    }
+    super.start();
+  }
+
+  public void stop() {
+    super.stop();
+  }
+
+  public SubmitQueryResponse executeQuery(Session session, String sql)
+      throws InterruptedException, IOException, IllegalQueryStatusException {
+
+    LOG.info("SQL: " + sql);
+    QueryContext queryContext = new QueryContext();
+
+    try {
+      // setting environment variables
+      String [] cmds = sql.split(" ");
+      if(cmds != null) {
+          if(cmds[0].equalsIgnoreCase("set")) {
+            String[] params = cmds[1].split("=");
+            context.getConf().set(params[0], params[1]);
+            SubmitQueryResponse.Builder responseBuilder = SubmitQueryResponse.newBuilder();
+            responseBuilder.setUserName(context.getConf().getVar(TajoConf.ConfVars.USERNAME));
+            responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+            responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+            return responseBuilder.build();
+          }
+      }
+
+      final boolean hiveQueryMode = context.getConf().getBoolVar(TajoConf.ConfVars.HIVE_QUERY_MODE);
+      LOG.info("hive.query.mode:" + hiveQueryMode);
+
+      if (hiveQueryMode) {
+        context.getSystemMetrics().counter("Query", "numHiveMode").inc();
+        queryContext.setHiveQueryMode();
+      }
+
+      context.getSystemMetrics().counter("Query", "totalQuery").inc();
+
+      Expr planningContext = hiveQueryMode ? converter.parse(sql) : analyzer.parse(sql);
+      LogicalPlan plan = createLogicalPlan(session, planningContext);
+      SubmitQueryResponse response = executeQueryInternal(queryContext, session, plan, sql);
+      return response;
+    } catch (Throwable t) {
+      context.getSystemMetrics().counter("Query", "errorQuery").inc();
+      LOG.error("\nStack Trace:\n" + StringUtils.stringifyException(t));
+      SubmitQueryResponse.Builder responseBuilder = SubmitQueryResponse.newBuilder();
+      responseBuilder.setUserName(context.getConf().getVar(TajoConf.ConfVars.USERNAME));
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      responseBuilder.setIsForwarded(true);
+      responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
+      String errorMessage = t.getMessage();
+      if (t.getMessage() == null) {
+        errorMessage = StringUtils.stringifyException(t);
+      }
+      responseBuilder.setErrorMessage(errorMessage);
+      return responseBuilder.build();
+    }
+  }
+
+  private SubmitQueryResponse executeQueryInternal(QueryContext queryContext,
+                                                      Session session,
+                                                      LogicalPlan plan,
+                                                      String sql) throws Exception {
+
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+
+    SubmitQueryResponse.Builder responseBuilder = SubmitQueryResponse.newBuilder();
+    responseBuilder.setIsForwarded(false);
+    responseBuilder.setUserName(context.getConf().getVar(TajoConf.ConfVars.USERNAME));
+
+    if (PlannerUtil.checkIfDDLPlan(rootNode)) {
+      context.getSystemMetrics().counter("Query", "numDDLQuery").inc();
+      updateQuery(session, rootNode.getChild());
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+
+    } else if (plan.isExplain()) { // explain query
+      String explainStr = PlannerUtil.buildExplainString(plan.getRootBlock().getRoot());
+      Schema schema = new Schema();
+      schema.addColumn("explain", TajoDataTypes.Type.TEXT);
+      RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(schema);
+
+      SerializedResultSet.Builder serializedResBuilder = SerializedResultSet.newBuilder();
+
+      VTuple tuple = new VTuple(1);
+      String[] lines = explainStr.split("\n");
+      int bytesNum = 0;
+      for (String line : lines) {
+        tuple.put(0, DatumFactory.createText(line));
+        byte [] encodedData = encoder.toBytes(tuple);
+        bytesNum += encodedData.length;
+        serializedResBuilder.addSerializedTuples(ByteString.copyFrom(encodedData));
+      }
+      serializedResBuilder.setSchema(schema.getProto());
+      serializedResBuilder.setBytesNum(bytesNum);
+
+      responseBuilder.setResultSet(serializedResBuilder.build());
+      responseBuilder.setMaxRowNum(lines.length);
+      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+
+      // Simple query indicates a form of 'select * from tb_name [LIMIT X];'.
+    } else if (PlannerUtil.checkIfSimpleQuery(plan)) {
+      ScanNode scanNode = plan.getRootBlock().getNode(NodeType.SCAN);
+      TableDesc desc = scanNode.getTableDesc();
+      if (plan.getRootBlock().hasNode(NodeType.LIMIT)) {
+        LimitNode limitNode = plan.getRootBlock().getNode(NodeType.LIMIT);
+        responseBuilder.setMaxRowNum((int) limitNode.getFetchFirstNum());
+      } else {
+        if (desc.getStats().getNumBytes() > 0 && desc.getStats().getNumRows() == 0) {
+          responseBuilder.setMaxRowNum(Integer.MAX_VALUE);
+        }
+      }
+      responseBuilder.setTableDesc(desc.getProto());
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+
+      // NonFromQuery indicates a form of 'select a, x+y;'
+    } else if (PlannerUtil.checkIfNonFromQuery(plan)) {
+      Target [] targets = plan.getRootBlock().getRawTargets();
+      if (targets == null) {
+        throw new PlanningException("No targets");
+      }
+      Tuple outTuple = new VTuple(targets.length);
+      for (int i = 0; i < targets.length; i++) {
+        EvalNode eval = targets[i].getEvalTree();
+        outTuple.put(i, eval.eval(null, null));
+      }
+
+      Schema schema = PlannerUtil.targetToSchema(targets);
+      RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(schema);
+      byte [] serializedBytes = encoder.toBytes(outTuple);
+      SerializedResultSet.Builder serializedResBuilder = SerializedResultSet.newBuilder();
+      serializedResBuilder.addSerializedTuples(ByteString.copyFrom(serializedBytes));
+      serializedResBuilder.setSchema(schema.getProto());
+      serializedResBuilder.setBytesNum(serializedBytes.length);
+
+      responseBuilder.setResultSet(serializedResBuilder);
+      responseBuilder.setMaxRowNum(1);
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+
+    } else { // it requires distributed execution. So, the query is forwarded to a query master.
+      context.getSystemMetrics().counter("Query", "numDMLQuery").inc();
+      hookManager.doHooks(queryContext, plan);
+
+      QueryJobManager queryJobManager = this.context.getQueryJobManager();
+      QueryInfo queryInfo;
+
+      queryInfo = queryJobManager.createNewQueryJob(session, queryContext, sql, rootNode);
+
+      if(queryInfo == null) {
+        responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+        responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
+        responseBuilder.setErrorMessage("Fail starting QueryMaster.");
+      } else {
+        responseBuilder.setIsForwarded(true);
+        responseBuilder.setQueryId(queryInfo.getQueryId().getProto());
+        responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+        if(queryInfo.getQueryMasterHost() != null) {
+          responseBuilder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+        }
+        responseBuilder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
+        LOG.info("Query is forwarded to " + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
+      }
+    }
+    SubmitQueryResponse response = responseBuilder.build();
+    return response;
+  }
+
+  public QueryId updateQuery(Session session, String sql) throws IOException, SQLException, PlanningException {
+    try {
+      LOG.info("SQL: " + sql);
+      // parse the query
+      Expr expr = analyzer.parse(sql);
+
+      LogicalPlan plan = createLogicalPlan(session, expr);
+      LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+
+      if (!PlannerUtil.checkIfDDLPlan(rootNode)) {
+        throw new SQLException("This is not update query:\n" + sql);
+      } else {
+        updateQuery(session, rootNode.getChild());
+        return QueryIdFactory.NULL_QUERY_ID;
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw new IOException(e.getMessage(), e);
+    }
+  }
+
+  private boolean updateQuery(Session session, LogicalNode root) throws IOException {
+
+    switch (root.getType()) {
+      case CREATE_DATABASE:
+        CreateDatabaseNode createDatabase = (CreateDatabaseNode) root;
+        createDatabase(session, createDatabase.getDatabaseName(), null, createDatabase.isIfNotExists());
+        return true;
+      case DROP_DATABASE:
+        DropDatabaseNode dropDatabaseNode = (DropDatabaseNode) root;
+        dropDatabase(session, dropDatabaseNode.getDatabaseName(), dropDatabaseNode.isIfExists());
+        return true;
+      case CREATE_TABLE:
+        CreateTableNode createTable = (CreateTableNode) root;
+        createTable(session, createTable, createTable.isIfNotExists());
+        return true;
+      case DROP_TABLE:
+        DropTableNode dropTable = (DropTableNode) root;
+        dropTable(session, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge());
+        return true;
+      case ALTER_TABLESPACE:
+        AlterTablespaceNode alterTablespace = (AlterTablespaceNode) root;
+        alterTablespace(session, alterTablespace);
+        return true;
+      case ALTER_TABLE:
+        AlterTableNode alterTable = (AlterTableNode) root;
+        alterTable(session,alterTable);
+        return true;
+      default:
+        throw new InternalError("updateQuery cannot handle such query: \n" + root.toJson());
+    }
+  }
+
+  private LogicalPlan createLogicalPlan(Session session, Expr expression) throws PlanningException {
+
+    VerificationState state = new VerificationState();
+    preVerifier.verify(session, state, expression);
+    if (!state.verified()) {
+      StringBuilder sb = new StringBuilder();
+      for (String error : state.getErrorMessages()) {
+        sb.append(error).append("\n");
+      }
+      throw new VerifyException(sb.toString());
+    }
+
+    LogicalPlan plan = planner.createPlan(session, expression);
+    LOG.info("=============================================");
+    LOG.info("Non Optimized Query: \n" + plan.toString());
+    LOG.info("=============================================");
+    optimizer.optimize(plan);
+    LOG.info("=============================================");
+    LOG.info("Optimized Query: \n" + plan.toString());
+    LOG.info("=============================================");
+
+    annotatedPlanVerifier.verify(session, state, plan);
+
+    if (!state.verified()) {
+      StringBuilder sb = new StringBuilder();
+      for (String error : state.getErrorMessages()) {
+        sb.append(error).append("\n");
+      }
+      throw new VerifyException(sb.toString());
+    }
+
+    return plan;
+  }
+
+  /**
+   * Alter a given table
+   */
+  public void alterTablespace(final Session session, final AlterTablespaceNode alterTablespace) {
+
+    final CatalogService catalog = context.getCatalog();
+    final String spaceName = alterTablespace.getTablespaceName();
+
+    AlterTablespaceProto.Builder builder = AlterTablespaceProto.newBuilder();
+    builder.setSpaceName(spaceName);
+    if (alterTablespace.getSetType() == AlterTablespaceSetType.LOCATION) {
+      AlterTablespaceCommand.Builder commandBuilder = AlterTablespaceCommand.newBuilder();
+      commandBuilder.setType(AlterTablespaceProto.AlterTablespaceType.LOCATION);
+      commandBuilder.setLocation(AlterTablespaceProto.SetLocation.newBuilder().setUri(alterTablespace.getLocation()));
+      commandBuilder.build();
+      builder.addCommand(commandBuilder);
+    } else {
+      throw new RuntimeException("This 'ALTER TABLESPACE' is not supported yet.");
+    }
+
+    catalog.alterTablespace(builder.build());
+  }
+
+  /**
+   * Alter a given table
+   */
+  public void alterTable(final Session session, final AlterTableNode alterTable) throws IOException {
+
+    final CatalogService catalog = context.getCatalog();
+    final String tableName = alterTable.getTableName();
+
+    String databaseName;
+    String simpleTableName;
+    if (CatalogUtil.isFQTableName(tableName)) {
+      String[] split = CatalogUtil.splitFQTableName(tableName);
+      databaseName = split[0];
+      simpleTableName = split[1];
+    } else {
+      databaseName = session.getCurrentDatabase();
+      simpleTableName = tableName;
+    }
+    final String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
+
+    if (!catalog.existsTable(databaseName, simpleTableName)) {
+      throw new NoSuchTableException(qualifiedName);
+    }
+
+    switch (alterTable.getAlterTableOpType()) {
+      case RENAME_TABLE:
+        if (!catalog.existsTable(databaseName, simpleTableName)) {
+          throw new NoSuchTableException(alterTable.getTableName());
+        }
+        if (catalog.existsTable(databaseName, alterTable.getNewTableName())) {
+          throw new AlreadyExistsTableException(alterTable.getNewTableName());
+        }
+
+        TableDesc desc = catalog.getTableDesc(databaseName, simpleTableName);
+
+        if (!desc.isExternal()) { // if the table is the managed table
+          Path oldPath = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR),
+              databaseName, simpleTableName);
+          Path newPath = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR),
+              databaseName, alterTable.getNewTableName());
+          FileSystem fs = oldPath.getFileSystem(context.getConf());
+
+          if (!fs.exists(oldPath)) {
+            throw new IOException("No such a table directory: " + oldPath);
+          }
+          if (fs.exists(newPath)) {
+            throw new IOException("Already table directory exists: " + newPath);
+          }
+
+          fs.rename(oldPath, newPath);
+        }
+        catalog.alterTable(CatalogUtil.renameTable(qualifiedName, alterTable.getNewTableName(),
+            AlterTableType.RENAME_TABLE));
+        break;
+      case RENAME_COLUMN:
+        if (existColumnName(qualifiedName, alterTable.getNewColumnName())) {
+          throw new ColumnNameAlreadyExistException(alterTable.getNewColumnName());
+        }
+        catalog.alterTable(CatalogUtil.renameColumn(qualifiedName, alterTable.getColumnName(), alterTable.getNewColumnName(), AlterTableType.RENAME_COLUMN));
+        break;
+      case ADD_COLUMN:
+        if (existColumnName(qualifiedName, alterTable.getAddNewColumn().getSimpleName())) {
+          throw new ColumnNameAlreadyExistException(alterTable.getAddNewColumn().getSimpleName());
+        }
+        catalog.alterTable(CatalogUtil.addNewColumn(qualifiedName, alterTable.getAddNewColumn(), AlterTableType.ADD_COLUMN));
+        break;
+      default:
+        //TODO
+    }
+  }
+
+  private boolean existColumnName(String tableName, String columnName) {
+    final TableDesc tableDesc = catalog.getTableDesc(tableName);
+    return tableDesc.getSchema().containsByName(columnName) ? true : false;
+  }
+
+  private TableDesc createTable(Session session, CreateTableNode createTable, boolean ifNotExists) throws IOException {
+    TableMeta meta;
+
+    if (createTable.hasOptions()) {
+      meta = CatalogUtil.newTableMeta(createTable.getStorageType(), createTable.getOptions());
+    } else {
+      meta = CatalogUtil.newTableMeta(createTable.getStorageType());
+    }
+
+    if(createTable.isExternal()){
+      Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
+    } else {
+      String databaseName;
+      String tableName;
+      if (CatalogUtil.isFQTableName(createTable.getTableName())) {
+        databaseName = CatalogUtil.extractQualifier(createTable.getTableName());
+        tableName = CatalogUtil.extractSimpleName(createTable.getTableName());
+      } else {
+        databaseName = session.getCurrentDatabase();
+        tableName = createTable.getTableName();
+      }
+
+      // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} )
+      Path tablePath = StorageUtil.concatPath(sm.getWarehouseDir(), databaseName, tableName);
+      createTable.setPath(tablePath);
+    }
+
+    return createTableOnPath(session, createTable.getTableName(), createTable.getTableSchema(),
+        meta, createTable.getPath(), createTable.isExternal(), createTable.getPartitionMethod(), ifNotExists);
+  }
+
+  public TableDesc createTableOnPath(Session session, String tableName, Schema schema, TableMeta meta,
+                                     Path path, boolean isExternal, PartitionMethodDesc partitionDesc,
+                                     boolean ifNotExists)
+      throws IOException {
+    String databaseName;
+    String simpleTableName;
+    if (CatalogUtil.isFQTableName(tableName)) {
+      String [] splitted = CatalogUtil.splitFQTableName(tableName);
+      databaseName = splitted[0];
+      simpleTableName = splitted[1];
+    } else {
+      databaseName = session.getCurrentDatabase();
+      simpleTableName = tableName;
+    }
+    String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
+
+    boolean exists = catalog.existsTable(databaseName, simpleTableName);
+
+    if (exists) {
+      if (ifNotExists) {
+        LOG.info("relation \"" + qualifiedName + "\" is already exists." );
+        return catalog.getTableDesc(databaseName, simpleTableName);
+      } else {
+        throw new AlreadyExistsTableException(CatalogUtil.buildFQName(databaseName, tableName));
+      }
+    }
+
+    FileSystem fs = path.getFileSystem(context.getConf());
+
+    if (isExternal) {
+      if(!fs.exists(path)) {
+        LOG.error("ERROR: " + path.toUri() + " does not exist");
+        throw new IOException("ERROR: " + path.toUri() + " does not exist");
+      }
+    } else {
+      fs.mkdirs(path);
+    }
+
+    long totalSize = 0;
+
+    try {
+      totalSize = sm.calculateSize(path);
+    } catch (IOException e) {
+      LOG.warn("Cannot calculate the size of the relation", e);
+    }
+
+    TableStats stats = new TableStats();
+    stats.setNumBytes(totalSize);
+    TableDesc desc = new TableDesc(CatalogUtil.buildFQName(databaseName, simpleTableName),
+        schema, meta, path, isExternal);
+    desc.setStats(stats);
+    if (partitionDesc != null) {
+      desc.setPartitionMethod(partitionDesc);
+    }
+
+    if (catalog.createTable(desc)) {
+      LOG.info("Table " + desc.getName() + " is created (" + desc.getStats().getNumBytes() + ")");
+      return desc;
+    } else {
+      LOG.info("Table creation " + tableName + " is failed.");
+      throw new CatalogException("Cannot create table \"" + tableName + "\".");
+    }
+  }
+
+  public boolean createDatabase(@Nullable Session session, String databaseName,
+                                @Nullable String tablespace,
+                                boolean ifNotExists) throws IOException {
+
+    String tablespaceName;
+    if (tablespace == null) {
+      tablespaceName = DEFAULT_TABLESPACE_NAME;
+    } else {
+      tablespaceName = tablespace;
+    }
+
+    // CREATE DATABASE IF NOT EXISTS
+    boolean exists = catalog.existDatabase(databaseName);
+    if (exists) {
+      if (ifNotExists) {
+        LOG.info("database \"" + databaseName + "\" is already exists." );
+        return true;
+      } else {
+        throw new AlreadyExistsDatabaseException(databaseName);
+      }
+    }
+
+    if (catalog.createDatabase(databaseName, tablespaceName)) {
+      String normalized = databaseName;
+      Path databaseDir = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR), normalized);
+      FileSystem fs = databaseDir.getFileSystem(context.getConf());
+      fs.mkdirs(databaseDir);
+    }
+
+    return true;
+  }
+
+  public boolean dropDatabase(Session session, String databaseName, boolean ifExists) {
+
+    boolean exists = catalog.existDatabase(databaseName);
+    if(!exists) {
+      if (ifExists) { // DROP DATABASE IF EXISTS
+        LOG.info("database \"" + databaseName + "\" does not exists." );
+        return true;
+      } else { // Otherwise, it causes an exception.
+        throw new NoSuchDatabaseException(databaseName);
+      }
+    }
+
+    if (session.getCurrentDatabase().equals(databaseName)) {
+      throw new RuntimeException("ERROR: Cannot drop the current open database");
+    }
+
+    boolean result = catalog.dropDatabase(databaseName);
+    LOG.info("database " + databaseName + " is dropped.");
+    return result;
+  }
+
+  /**
+   * Drop a given named table
+   *
+   * @param tableName to be dropped
+   * @param purge Remove all data if purge is true.
+   */
+  public boolean dropTable(Session session, String tableName, boolean ifExists, boolean purge) {
+    CatalogService catalog = context.getCatalog();
+
+    String databaseName;
+    String simpleTableName;
+    if (CatalogUtil.isFQTableName(tableName)) {
+      String [] splitted = CatalogUtil.splitFQTableName(tableName);
+      databaseName = splitted[0];
+      simpleTableName = splitted[1];
+    } else {
+      databaseName = session.getCurrentDatabase();
+      simpleTableName = tableName;
+    }
+    String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
+
+    boolean exists = catalog.existsTable(qualifiedName);
+    if(!exists) {
+      if (ifExists) { // DROP TABLE IF EXISTS
+        LOG.info("relation \"" + qualifiedName + "\" is already exists." );
+        return true;
+      } else { // Otherwise, it causes an exception.
+        throw new NoSuchTableException(qualifiedName);
+      }
+    }
+
+    Path path = catalog.getTableDesc(qualifiedName).getPath();
+    catalog.dropTable(qualifiedName);
+
+    if (purge) {
+      try {
+        FileSystem fs = path.getFileSystem(context.getConf());
+        fs.delete(path, true);
+      } catch (IOException e) {
+        throw new InternalError(e.getMessage());
+      }
+    }
+
+    LOG.info(String.format("relation \"%s\" is " + (purge ? " purged." : " dropped."), qualifiedName));
+    return true;
+  }
+
+  public interface DistributedQueryHook {
+    boolean isEligible(QueryContext queryContext, LogicalPlan plan);
+    void hook(QueryContext queryContext, LogicalPlan plan) throws Exception;
+  }
+
+  public static class DistributedQueryHookManager {
+    private List<DistributedQueryHook> hooks = new ArrayList<DistributedQueryHook>();
+    public void addHook(DistributedQueryHook hook) {
+      hooks.add(hook);
+    }
+
+    public void doHooks(QueryContext queryContext, LogicalPlan plan) {
+      for (DistributedQueryHook hook : hooks) {
+        if (hook.isEligible(queryContext, plan)) {
+          try {
+            hook.hook(queryContext, plan);
+          } catch (Throwable t) {
+            t.printStackTrace();
+          }
+        }
+      }
+    }
+  }
+
+  public class CreateTableHook implements DistributedQueryHook {
+
+    @Override
+    public boolean isEligible(QueryContext queryContext, LogicalPlan plan) {
+      LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+      return rootNode.getChild().getType() == NodeType.CREATE_TABLE;
+    }
+
+    @Override
+    public void hook(QueryContext queryContext, LogicalPlan plan) throws Exception {
+      LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+      CreateTableNode createTableNode = rootNode.getChild();
+      String [] splitted  = CatalogUtil.splitFQTableName(createTableNode.getTableName());
+      String databaseName = splitted[0];
+      String tableName = splitted[1];
+      queryContext.setOutputTable(tableName);
+      queryContext.setOutputPath(
+          StorageUtil.concatPath(TajoConf.getWarehouseDir(context.getConf()), databaseName, tableName));
+      if(createTableNode.getPartitionMethod() != null) {
+        queryContext.setPartitionMethod(createTableNode.getPartitionMethod());
+      }
+      queryContext.setCreateTable();
+    }
+  }
+
+  public static class InsertHook implements DistributedQueryHook {
+
+    @Override
+    public boolean isEligible(QueryContext queryContext, LogicalPlan plan) {
+      return plan.getRootBlock().getRootType() == NodeType.INSERT;
+    }
+
+    @Override
+  public void hook(QueryContext queryContext, LogicalPlan plan) throws Exception {
+      queryContext.setInsert();
+
+      InsertNode insertNode = plan.getRootBlock().getNode(NodeType.INSERT);
+
+      // Set QueryContext settings, such as output table name and output path.
+      // It also remove data files if overwrite is true.
+      Path outputPath;
+      if (insertNode.hasTargetTable()) { // INSERT INTO [TB_NAME]
+        queryContext.setOutputTable(insertNode.getTableName());
+        queryContext.setOutputPath(insertNode.getPath());
+      } else { // INSERT INTO LOCATION ...
+        // When INSERT INTO LOCATION, must not set output table.
+        outputPath = insertNode.getPath();
+        queryContext.setFileOutput();
+        queryContext.setOutputPath(outputPath);
+      }
+
+      if (insertNode.isOverwrite()) {
+        queryContext.setOutputOverwrite();
+      }
+    }
+  }
+}