You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2014/12/22 09:11:36 UTC

[1/9] tajo git commit: TAJO-1261: Separate query and ddl execution codes from GlobalEngine.

Repository: tajo
Updated Branches:
  refs/heads/index_support 94faf6ec8 -> 3b5c28711


http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/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
index 9ab4f0a..c96b86e 100644
--- 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
@@ -37,6 +37,8 @@ import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
+import org.apache.tajo.master.exec.prehook.InsertIntoHook;
 import org.apache.tajo.plan.LogicalOptimizer;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
@@ -390,10 +392,6 @@ public class QueryMasterTask extends CompositeService {
 
       optimizer.optimize(queryContext, 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) {


[2/9] tajo git commit: TAJO-1261: Separate query and ddl execution codes from GlobalEngine.

Posted by ji...@apache.org.
TAJO-1261: Separate query and ddl execution codes from GlobalEngine.

Closes #312


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

Branch: refs/heads/index_support
Commit: a4c3484232e139d4d21d0cfb9c31e5d784de652b
Parents: 3413107
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Dec 19 20:55:26 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 22 15:07:17 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   7 +-
 .../org/apache/tajo/master/GlobalEngine.java    | 789 +------------------
 .../java/org/apache/tajo/master/TajoMaster.java |   2 +-
 .../tajo/master/TajoMasterClientService.java    |   9 +-
 .../apache/tajo/master/exec/DDLExecutor.java    | 434 ++++++++++
 .../apache/tajo/master/exec/QueryExecutor.java  | 391 +++++++++
 .../master/exec/prehook/CreateTableHook.java    |  53 ++
 .../exec/prehook/DistributedQueryHook.java      |  27 +
 .../prehook/DistributedQueryHookManager.java    |  45 ++
 .../master/exec/prehook/InsertIntoHook.java     |  61 ++
 .../master/querymaster/QueryMasterTask.java     |   6 +-
 11 files changed, 1056 insertions(+), 768 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index e30e24c..36cff8a 100644
--- a/CHANGES
+++ b/CHANGES
@@ -24,8 +24,11 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
-    TAJO-1247: Store type 'TEXTFILE' should be TEXT while keeping enum 'TEXTFILE' in protobuf.
-    (DaeMyung Kang via hyunsik)
+    TAJO-1261: Separate query and ddl execution codes from GlobalEngine. 
+    (hyunsik)
+
+    TAJO-1247: Store type 'TEXTFILE' should be TEXT while keeping enum 
+    'TEXTFILE' in protobuf. (DaeMyung Kang via hyunsik)
 
     TAJO-1221: HA TajoClient should not connect TajoMaster at the first. 
     (jaehwa)

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/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
index 71b1f9b..d7e7670 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -19,66 +19,42 @@
 package org.apache.tajo.master;
 
 import com.google.common.annotations.VisibleForTesting;
-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.FileStatus;
-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.SessionVars;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.algebra.AlterTablespaceSetType;
 import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.algebra.JsonHelper;
-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.CatalogService;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-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.parser.SQLAnalyzer;
-import org.apache.tajo.engine.planner.physical.EvalExprExec;
-import org.apache.tajo.engine.planner.physical.StoreTableExec;
 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.Query;
-import org.apache.tajo.master.querymaster.QueryInfo;
-import org.apache.tajo.master.querymaster.QueryJobManager;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.exec.DDLExecutor;
+import org.apache.tajo.master.exec.QueryExecutor;
+import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
 import org.apache.tajo.master.session.Session;
 import org.apache.tajo.plan.*;
-import org.apache.tajo.plan.expr.EvalNode;
-import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.plan.logical.InsertNode;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.verifier.LogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.VerificationState;
 import org.apache.tajo.plan.verifier.VerifyException;
-import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.util.CommonTestingUtil;
-import org.apache.tajo.util.ProtoUtil;
-import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
 import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.TimeZone;
-
-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.UpdateTableStatsProto;
-import static org.apache.tajo.ipc.ClientProtos.SerializedResultSet;
+
 import static org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
 
 public class GlobalEngine extends AbstractService {
@@ -96,11 +72,17 @@ public class GlobalEngine extends AbstractService {
   private LogicalPlanVerifier annotatedPlanVerifier;
   private DistributedQueryHookManager hookManager;
 
+  private QueryExecutor queryExecutor;
+  private DDLExecutor ddlExecutor;
+
   public GlobalEngine(final MasterContext context) {
     super(GlobalEngine.class.getName());
     this.context = context;
     this.catalog = context.getCatalog();
     this.sm = context.getStorageManager();
+
+    this.ddlExecutor = new DDLExecutor(context);
+    this.queryExecutor = new QueryExecutor(context, ddlExecutor);
   }
 
   public void start() {
@@ -110,10 +92,6 @@ public class GlobalEngine extends AbstractService {
       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);
     }
@@ -148,6 +126,14 @@ public class GlobalEngine extends AbstractService {
     return annotatedPlanVerifier;
   }
 
+  public DDLExecutor getDDLExecutor() {
+    return ddlExecutor;
+  }
+
+  public QueryExecutor getQueryExecutor() {
+    return queryExecutor;
+  }
+
   private QueryContext createQueryContext(Session session) {
     QueryContext newQueryContext =  new QueryContext(context.getConf(), session);
 
@@ -168,12 +154,12 @@ public class GlobalEngine extends AbstractService {
       if (isJson) {
         planningContext = buildExpressionFromJson(query);
       } else {
-        planningContext = buildExpressionFromSql(queryContext, query);
+        planningContext = buildExpressionFromSql(query);
       }
 
       String jsonExpr = planningContext.toJson();
       LogicalPlan plan = createLogicalPlan(queryContext, planningContext);
-      SubmitQueryResponse response = executeQueryInternal(queryContext, session, plan, query, jsonExpr);
+      SubmitQueryResponse response = queryExecutor.execute(queryContext, session, query, jsonExpr, plan);
       return response;
     } catch (Throwable t) {
       context.getSystemMetrics().counter("Query", "errorQuery").inc();
@@ -197,288 +183,14 @@ public class GlobalEngine extends AbstractService {
     return JsonHelper.fromJson(json, Expr.class);
   }
 
-  public Expr buildExpressionFromSql(QueryContext queryContext, String sql)
-      throws InterruptedException, IOException, IllegalQueryStatusException {
+  public Expr buildExpressionFromSql(String sql) throws InterruptedException, IOException,
+      IllegalQueryStatusException {
     context.getSystemMetrics().counter("Query", "totalQuery").inc();
     return analyzer.parse(sql);
   }
 
-  private SubmitQueryResponse executeQueryInternal(QueryContext queryContext,
-                                                   Session session,
-                                                   LogicalPlan plan,
-                                                   String sql,
-                                                   String jsonExpr) throws Exception {
-
-    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
-
-    SubmitQueryResponse.Builder responseBuilder = SubmitQueryResponse.newBuilder();
-    responseBuilder.setIsForwarded(false);
-    responseBuilder.setUserName(queryContext.get(SessionVars.USERNAME));
-
-    if (PlannerUtil.checkIfSetSession(rootNode)) {
-
-      SetSessionNode setSessionNode = rootNode.getChild();
-
-      final String varName = setSessionNode.getName();
-
-      // SET CATALOG 'XXX'
-      if (varName.equals(SessionVars.CURRENT_DATABASE.name())) {
-        String databaseName = setSessionNode.getValue();
-
-        if (catalog.existDatabase(databaseName)) {
-          session.selectDatabase(setSessionNode.getValue());
-        } else {
-          responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
-          responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
-          responseBuilder.setErrorMessage("database \"" + databaseName + "\" does not exists.");
-          return responseBuilder.build();
-        }
-
-        // others
-      } else {
-        if (setSessionNode.isDefaultValue()) {
-          session.removeVariable(varName);
-        } else {
-          session.setVariable(varName, setSessionNode.getValue());
-        }
-      }
-
-      context.getSystemMetrics().counter("Query", "numDDLQuery").inc();
-      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
-      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
-
-    } else if (PlannerUtil.checkIfDDLPlan(rootNode)) {
-      context.getSystemMetrics().counter("Query", "numDDLQuery").inc();
-      updateQuery(queryContext, 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);
-      if (scanNode == null) {
-        scanNode = plan.getRootBlock().getNode(NodeType.PARTITIONS_SCAN);
-      }
-      TableDesc desc = scanNode.getTableDesc();
-      int maxRow = Integer.MAX_VALUE;
-      if (plan.getRootBlock().hasNode(NodeType.LIMIT)) {
-        LimitNode limitNode = plan.getRootBlock().getNode(NodeType.LIMIT);
-        maxRow = (int) limitNode.getFetchFirstNum();
-      }
-      if (desc.getStats().getNumRows() == 0) {
-        desc.getStats().setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
-      }
-
-      QueryId queryId = QueryIdFactory.newQueryId(context.getResourceManager().getSeedQueryId());
-
-      NonForwardQueryResultScanner queryResultScanner =
-          new NonForwardQueryResultScanner(context.getConf(), session.getSessionId(), queryId, scanNode, desc, maxRow);
-
-      queryResultScanner.init();
-      session.addNonForwardQueryResultScanner(queryResultScanner);
-
-      responseBuilder.setQueryId(queryId.getProto());
-      responseBuilder.setMaxRowNum(maxRow);
-      responseBuilder.setTableDesc(desc.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");
-      }
-      final 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));
-      }
-      boolean isInsert = rootNode.getChild() != null && rootNode.getChild().getType() == NodeType.INSERT;
-      if (isInsert) {
-        InsertNode insertNode = rootNode.getChild();
-        insertNonFromQuery(queryContext, insertNode, responseBuilder);
-      } else {
-        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.
-      StoreType storeType = PlannerUtil.getStoreType(plan);
-      if (storeType != null) {
-        StorageManager sm = StorageManager.getStorageManager(context.getConf(), storeType);
-        StorageProperty storageProperty = sm.getStorageProperty();
-        if (!storageProperty.isSupportsInsertInto()) {
-          throw new VerifyException("Inserting into non-file storage is not supported.");
-        }
-        sm.beforeInsertOrCATS(rootNode.getChild());
-      }
-      context.getSystemMetrics().counter("Query", "numDMLQuery").inc();
-      hookManager.doHooks(queryContext, plan);
-
-      QueryJobManager queryJobManager = this.context.getQueryJobManager();
-      QueryInfo queryInfo;
-
-      queryInfo = queryJobManager.createNewQueryJob(session, queryContext, sql, jsonExpr, rootNode);
-
-      if(queryInfo == null) {
-        responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
-        responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
-        responseBuilder.setErrorMessage("Fail starting QueryMaster.");
-        LOG.error("Fail starting QueryMaster: " + sql);
-      } 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 " + queryInfo.getQueryId().toString() + "," + queryInfo.getSql() + "," +
-            " is forwarded to " + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
-      }
-    }
-
-    responseBuilder.setSessionVars(ProtoUtil.convertFromMap(session.getAllVariables()));
-    SubmitQueryResponse response = responseBuilder.build();
-    return response;
-  }
-
-  private void insertNonFromQuery(QueryContext queryContext, InsertNode insertNode, SubmitQueryResponse.Builder responseBuilder)
-      throws Exception {
-    String nodeUniqName = insertNode.getTableName() == null ? insertNode.getPath().getName() : insertNode.getTableName();
-    String queryId = nodeUniqName + "_" + System.currentTimeMillis();
-
-    FileSystem fs = TajoConf.getWarehouseDir(context.getConf()).getFileSystem(context.getConf());
-    Path stagingDir = QueryMasterTask.initStagingDir(context.getConf(), queryId.toString(), queryContext);
-    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-
-    TableDesc tableDesc = null;
-    Path finalOutputDir = null;
-    if (insertNode.getTableName() != null) {
-      tableDesc = this.catalog.getTableDesc(insertNode.getTableName());
-      finalOutputDir = new Path(tableDesc.getPath());
-    } else {
-      finalOutputDir = insertNode.getPath();
-    }
-
-    TaskAttemptContext taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
-    taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
-
-    EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
-    StoreTableExec exec = new StoreTableExec(taskAttemptContext, insertNode, evalExprExec);
-    try {
-      exec.init();
-      exec.next();
-    } finally {
-      exec.close();
-    }
-
-    if (insertNode.isOverwrite()) { // 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(stagingDir, 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 {
-      FileStatus[] files = fs.listStatus(stagingResultDir);
-      for (FileStatus eachFile: files) {
-        Path targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName());
-        if (fs.exists(targetFilePath)) {
-          targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName() + "_" + System.currentTimeMillis());
-        }
-        fs.rename(eachFile.getPath(), targetFilePath);
-      }
-    }
-
-    if (insertNode.hasTargetTable()) {
-      TableStats stats = tableDesc.getStats();
-      long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-      stats.setNumBytes(volume);
-      stats.setNumRows(1);
-
-      UpdateTableStatsProto.Builder builder = UpdateTableStatsProto.newBuilder();
-      builder.setTableName(tableDesc.getName());
-      builder.setStats(stats.getProto());
-
-      catalog.updateTableStats(builder.build());
-
-      responseBuilder.setTableDesc(tableDesc.getProto());
-    } else {
-      TableStats stats = new TableStats();
-      long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-      stats.setNumBytes(volume);
-      stats.setNumRows(1);
-
-      // Empty TableDesc
-      List<CatalogProtos.ColumnProto> columns = new ArrayList<CatalogProtos.ColumnProto>();
-      CatalogProtos.TableDescProto tableDescProto = CatalogProtos.TableDescProto.newBuilder()
-          .setTableName(nodeUniqName)
-          .setMeta(CatalogProtos.TableProto.newBuilder().setStoreType(CatalogProtos.StoreType.CSV).build())
-          .setSchema(CatalogProtos.SchemaProto.newBuilder().addAllFields(columns).build())
-          .setStats(stats.getProto())
-          .build();
-
-      responseBuilder.setTableDesc(tableDescProto);
-    }
-
-    // If queryId == NULL_QUERY_ID and MaxRowNum == -1, TajoCli prints only number of inserted rows.
-    responseBuilder.setMaxRowNum(-1);
-    responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
-    responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
-  }
-
-
-  public QueryId updateQuery(QueryContext queryContext, String sql, boolean isJson) throws IOException, SQLException, PlanningException {
+  public QueryId updateQuery(QueryContext queryContext, String sql, boolean isJson) throws IOException,
+      SQLException, PlanningException {
     try {
       LOG.info("SQL: " + sql);
 
@@ -496,7 +208,7 @@ public class GlobalEngine extends AbstractService {
       if (!PlannerUtil.checkIfDDLPlan(rootNode)) {
         throw new SQLException("This is not update query:\n" + sql);
       } else {
-        updateQuery(queryContext, rootNode.getChild());
+        ddlExecutor.execute(queryContext, plan);
         return QueryIdFactory.NULL_QUERY_ID;
       }
     } catch (Exception e) {
@@ -505,44 +217,6 @@ public class GlobalEngine extends AbstractService {
     }
   }
 
-  private boolean updateQuery(QueryContext queryContext, LogicalNode root) throws IOException {
-
-    switch (root.getType()) {
-      case SET_SESSION:
-
-      case CREATE_DATABASE:
-        CreateDatabaseNode createDatabase = (CreateDatabaseNode) root;
-        createDatabase(queryContext, createDatabase.getDatabaseName(), null, createDatabase.isIfNotExists());
-        return true;
-      case DROP_DATABASE:
-        DropDatabaseNode dropDatabaseNode = (DropDatabaseNode) root;
-        dropDatabase(queryContext, dropDatabaseNode.getDatabaseName(), dropDatabaseNode.isIfExists());
-        return true;
-      case CREATE_TABLE:
-        CreateTableNode createTable = (CreateTableNode) root;
-        createTable(queryContext, createTable, createTable.isIfNotExists());
-        return true;
-      case DROP_TABLE:
-        DropTableNode dropTable = (DropTableNode) root;
-        dropTable(queryContext, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge());
-        return true;
-      case ALTER_TABLESPACE:
-        AlterTablespaceNode alterTablespace = (AlterTablespaceNode) root;
-        alterTablespace(queryContext, alterTablespace);
-        return true;
-      case ALTER_TABLE:
-        AlterTableNode alterTable = (AlterTableNode) root;
-        alterTable(queryContext,alterTable);
-        return true;
-      case TRUNCATE_TABLE:
-        TruncateTableNode truncateTable = (TruncateTableNode) root;
-        truncateTable(queryContext, truncateTable);
-        return true;
-      default:
-        throw new InternalError("updateQuery cannot handle such query: \n" + root.toJson());
-    }
-  }
-
   private LogicalPlan createLogicalPlan(QueryContext queryContext, Expr expression) throws PlanningException {
 
     VerificationState state = new VerificationState();
@@ -599,403 +273,4 @@ public class GlobalEngine extends AbstractService {
       }
     }
   }
-
-  /**
-   * Alter a given table
-   */
-  public void alterTablespace(final QueryContext queryContext, 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) {
-      AlterTablespaceProto.AlterTablespaceCommand.Builder commandBuilder =
-          AlterTablespaceProto.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 QueryContext queryContext, 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 = queryContext.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
-    }
-  }
-
-  /**
-   * Truncate table a given table
-   */
-  public void truncateTable(final QueryContext queryContext, final TruncateTableNode truncateTableNode)
-      throws IOException {
-    List<String> tableNames = truncateTableNode.getTableNames();
-    final CatalogService catalog = context.getCatalog();
-
-    String databaseName;
-    String simpleTableName;
-
-    List<TableDesc> tableDescList = new ArrayList<TableDesc>();
-    for (String eachTableName: tableNames) {
-      if (CatalogUtil.isFQTableName(eachTableName)) {
-        String[] split = CatalogUtil.splitFQTableName(eachTableName);
-        databaseName = split[0];
-        simpleTableName = split[1];
-      } else {
-        databaseName = queryContext.getCurrentDatabase();
-        simpleTableName = eachTableName;
-      }
-      final String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
-
-      if (!catalog.existsTable(databaseName, simpleTableName)) {
-        throw new NoSuchTableException(qualifiedName);
-      }
-
-      Path warehousePath = new Path(TajoConf.getWarehouseDir(context.getConf()), databaseName);
-      TableDesc tableDesc = catalog.getTableDesc(databaseName, simpleTableName);
-      Path tablePath = new Path(tableDesc.getPath());
-      if (tablePath.getParent() == null ||
-          !tablePath.getParent().toUri().getPath().equals(warehousePath.toUri().getPath())) {
-        throw new IOException("Can't truncate external table:" + eachTableName + ", data dir=" + tablePath +
-            ", warehouse dir=" + warehousePath);
-      }
-      tableDescList.add(tableDesc);
-    }
-
-    for (TableDesc eachTable: tableDescList) {
-      Path path = new Path(eachTable.getPath());
-      LOG.info("Truncate table: " + eachTable.getName() + ", delete all data files in " + path);
-      FileSystem fs = path.getFileSystem(context.getConf());
-
-      FileStatus[] files = fs.listStatus(path);
-      if (files != null) {
-        for (FileStatus eachFile: files) {
-          fs.delete(eachFile.getPath(), true);
-        }
-      }
-    }
-  }
-
-  private boolean existColumnName(String tableName, String columnName) {
-    final TableDesc tableDesc = catalog.getTableDesc(tableName);
-    return tableDesc.getSchema().containsByName(columnName) ? true : false;
-  }
-
-  private TableDesc createTable(QueryContext queryContext, 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(PlannerUtil.isFileStorageType(createTable.getStorageType()) && createTable.isExternal()){
-      Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
-    }
-
-    return createTable(queryContext, createTable.getTableName(), createTable.getStorageType(),
-        createTable.getTableSchema(), meta, createTable.getPath(), createTable.isExternal(),
-        createTable.getPartitionMethod(), ifNotExists);
-  }
-
-  public TableDesc createTable(QueryContext queryContext, String tableName, StoreType storeType,
-                               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 = queryContext.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));
-      }
-    }
-
-    TableDesc desc = new TableDesc(CatalogUtil.buildFQName(databaseName, simpleTableName),
-        schema, meta, (path != null ? path.toUri(): null), isExternal);
-    
-    if (partitionDesc != null) {
-      desc.setPartitionMethod(partitionDesc);
-    }
-
-    StorageManager.getStorageManager(queryContext.getConf(), storeType).createTable(desc, ifNotExists);
-
-    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 QueryContext queryContext, 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(QueryContext queryContext, 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 (queryContext.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(QueryContext queryContext, 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 = queryContext.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);
-      }
-    }
-
-    TableDesc tableDesc = catalog.getTableDesc(qualifiedName);
-    catalog.dropTable(qualifiedName);
-
-    if (purge) {
-      try {
-        StorageManager.getStorageManager(queryContext.getConf(),
-            tableDesc.getMeta().getStoreType()).purgeTable(tableDesc);
-      } 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());
-        if (insertNode.hasPartition()) {
-          queryContext.setPartitionMethod(insertNode.getPartitionMethod());
-        }
-      } 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();
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/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
index f307127..d021e43 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -371,7 +371,7 @@ public class TajoMaster extends CompositeService {
     }
 
     if (!catalog.existDatabase(DEFAULT_DATABASE_NAME)) {
-      globalEngine.createDatabase(null, DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME, false);
+      globalEngine.getDDLExecutor().createDatabase(null, DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME, false);
     } else {
       LOG.info(String.format("Default database (%s) is already prepared.", DEFAULT_DATABASE_NAME));
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/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
index 7014034..ee99353 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -658,7 +658,7 @@ public class TajoMasterClientService extends AbstractService {
         Session session = context.getSessionManager().getSession(request.getSessionId().getId());
         QueryContext queryContext = new QueryContext(conf, session);
 
-        if (context.getGlobalEngine().createDatabase(queryContext, request.getValue(), null, false)) {
+        if (context.getGlobalEngine().getDDLExecutor().createDatabase(queryContext, request.getValue(), null, false)) {
           return BOOL_TRUE;
         } else {
           return BOOL_FALSE;
@@ -688,7 +688,7 @@ public class TajoMasterClientService extends AbstractService {
         Session session = context.getSessionManager().getSession(request.getSessionId().getId());
         QueryContext queryContext = new QueryContext(conf, session);
 
-        if (context.getGlobalEngine().dropDatabase(queryContext, request.getValue(), false)) {
+        if (context.getGlobalEngine().getDDLExecutor().dropDatabase(queryContext, request.getValue(), false)) {
           return BOOL_TRUE;
         } else {
           return BOOL_FALSE;
@@ -814,7 +814,7 @@ public class TajoMasterClientService extends AbstractService {
 
         TableDesc desc;
         try {
-          desc = context.getGlobalEngine().createTable(queryContext, request.getName(),
+          desc = context.getGlobalEngine().getDDLExecutor().createTable(queryContext, request.getName(),
               meta.getStoreType(), schema,
               meta, path, true, partitionDesc, false);
         } catch (Exception e) {
@@ -843,7 +843,8 @@ public class TajoMasterClientService extends AbstractService {
         Session session = context.getSessionManager().getSession(dropTable.getSessionId().getId());
         QueryContext queryContext = new QueryContext(conf, session);
 
-        context.getGlobalEngine().dropTable(queryContext, dropTable.getName(), false, dropTable.getPurge());
+        context.getGlobalEngine().getDDLExecutor().dropTable(queryContext, dropTable.getName(), false,
+            dropTable.getPurge());
         return BOOL_TRUE;
       } catch (Throwable t) {
         throw new ServiceException(t);

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
new file mode 100644
index 0000000..acbaa01
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
@@ -0,0 +1,434 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.exec;
+
+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.tajo.algebra.AlterTablespaceSetType;
+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.proto.CatalogProtos.AlterTablespaceProto;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageUtil;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
+
+/**
+ * Executor for DDL statements. They are executed on only TajoMaster.
+ */
+public class DDLExecutor {
+  private static final Log LOG = LogFactory.getLog(DDLExecutor.class);
+
+  private final TajoMaster.MasterContext context;
+  private final CatalogService catalog;
+  private final StorageManager storageManager;
+
+  public DDLExecutor(TajoMaster.MasterContext context) {
+    this.context = context;
+    this.catalog = context.getCatalog();
+    this.storageManager = context.getStorageManager();
+  }
+
+  public boolean execute(QueryContext queryContext, LogicalPlan plan) throws IOException {
+    LogicalNode root = ((LogicalRootNode) plan.getRootBlock().getRoot()).getChild();
+
+    switch (root.getType()) {
+
+    case ALTER_TABLESPACE:
+      AlterTablespaceNode alterTablespace = (AlterTablespaceNode) root;
+      alterTablespace(context, queryContext, alterTablespace);
+      return true;
+
+
+    case CREATE_DATABASE:
+      CreateDatabaseNode createDatabase = (CreateDatabaseNode) root;
+      createDatabase(queryContext, createDatabase.getDatabaseName(), null, createDatabase.isIfNotExists());
+      return true;
+    case DROP_DATABASE:
+      DropDatabaseNode dropDatabaseNode = (DropDatabaseNode) root;
+      dropDatabase(queryContext, dropDatabaseNode.getDatabaseName(), dropDatabaseNode.isIfExists());
+      return true;
+
+
+    case CREATE_TABLE:
+      CreateTableNode createTable = (CreateTableNode) root;
+      createTable(queryContext, createTable, createTable.isIfNotExists());
+      return true;
+    case DROP_TABLE:
+      DropTableNode dropTable = (DropTableNode) root;
+      dropTable(queryContext, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge());
+      return true;
+    case TRUNCATE_TABLE:
+      TruncateTableNode truncateTable = (TruncateTableNode) root;
+      truncateTable(queryContext, truncateTable);
+      return true;
+
+    case ALTER_TABLE:
+      AlterTableNode alterTable = (AlterTableNode) root;
+      alterTable(context, queryContext, alterTable);
+      return true;
+
+    default:
+      throw new InternalError("updateQuery cannot handle such query: \n" + root.toJson());
+    }
+  }
+
+  /**
+   * Alter a given table
+   */
+  public static void alterTablespace(final TajoMaster.MasterContext context, final QueryContext queryContext,
+                                     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) {
+      AlterTablespaceProto.AlterTablespaceCommand.Builder commandBuilder =
+          AlterTablespaceProto.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());
+  }
+
+  //--------------------------------------------------------------------------
+
+  // Database Section
+  //--------------------------------------------------------------------------
+  public boolean createDatabase(@Nullable QueryContext queryContext, 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(QueryContext queryContext, 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 (queryContext.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;
+  }
+
+  //--------------------------------------------------------------------------
+
+  // Table Section
+  //--------------------------------------------------------------------------
+  private TableDesc createTable(QueryContext queryContext, 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(PlannerUtil.isFileStorageType(createTable.getStorageType()) && createTable.isExternal()){
+      Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
+    }
+
+    return createTable(queryContext, createTable.getTableName(), createTable.getStorageType(),
+        createTable.getTableSchema(), meta, createTable.getPath(), createTable.isExternal(),
+        createTable.getPartitionMethod(), ifNotExists);
+  }
+
+  public TableDesc createTable(QueryContext queryContext, String tableName, CatalogProtos.StoreType storeType,
+                               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 = queryContext.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));
+      }
+    }
+
+    TableDesc desc = new TableDesc(CatalogUtil.buildFQName(databaseName, simpleTableName),
+        schema, meta, (path != null ? path.toUri(): null), isExternal);
+
+    if (partitionDesc != null) {
+      desc.setPartitionMethod(partitionDesc);
+    }
+
+    StorageManager.getStorageManager(queryContext.getConf(), storeType).createTable(desc, ifNotExists);
+
+    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 + "\".");
+    }
+  }
+
+  /**
+   * Drop a given named table
+   *
+   * @param tableName to be dropped
+   * @param purge     Remove all data if purge is true.
+   */
+  public boolean dropTable(QueryContext queryContext, 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 = queryContext.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);
+      }
+    }
+
+    TableDesc tableDesc = catalog.getTableDesc(qualifiedName);
+    catalog.dropTable(qualifiedName);
+
+    if (purge) {
+      try {
+        StorageManager.getStorageManager(queryContext.getConf(),
+            tableDesc.getMeta().getStoreType()).purgeTable(tableDesc);
+      } catch (IOException e) {
+        throw new InternalError(e.getMessage());
+      }
+    }
+
+    LOG.info(String.format("relation \"%s\" is " + (purge ? " purged." : " dropped."), qualifiedName));
+    return true;
+  }
+
+  /**
+   * Truncate table a given table
+   */
+  public void truncateTable(final QueryContext queryContext, final TruncateTableNode truncateTableNode)
+      throws IOException {
+    List<String> tableNames = truncateTableNode.getTableNames();
+    final CatalogService catalog = context.getCatalog();
+
+    String databaseName;
+    String simpleTableName;
+
+    List<TableDesc> tableDescList = new ArrayList<TableDesc>();
+    for (String eachTableName: tableNames) {
+      if (CatalogUtil.isFQTableName(eachTableName)) {
+        String[] split = CatalogUtil.splitFQTableName(eachTableName);
+        databaseName = split[0];
+        simpleTableName = split[1];
+      } else {
+        databaseName = queryContext.getCurrentDatabase();
+        simpleTableName = eachTableName;
+      }
+      final String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
+
+      if (!catalog.existsTable(databaseName, simpleTableName)) {
+        throw new NoSuchTableException(qualifiedName);
+      }
+
+      Path warehousePath = new Path(TajoConf.getWarehouseDir(context.getConf()), databaseName);
+      TableDesc tableDesc = catalog.getTableDesc(databaseName, simpleTableName);
+      Path tablePath = new Path(tableDesc.getPath());
+      if (tablePath.getParent() == null ||
+          !tablePath.getParent().toUri().getPath().equals(warehousePath.toUri().getPath())) {
+        throw new IOException("Can't truncate external table:" + eachTableName + ", data dir=" + tablePath +
+            ", warehouse dir=" + warehousePath);
+      }
+      tableDescList.add(tableDesc);
+    }
+
+    for (TableDesc eachTable: tableDescList) {
+      Path path = new Path(eachTable.getPath());
+      LOG.info("Truncate table: " + eachTable.getName() + ", delete all data files in " + path);
+      FileSystem fs = path.getFileSystem(context.getConf());
+
+      FileStatus[] files = fs.listStatus(path);
+      if (files != null) {
+        for (FileStatus eachFile: files) {
+          fs.delete(eachFile.getPath(), true);
+        }
+      }
+    }
+  }
+
+  /**
+   * ALTER TABLE SET ...
+   */
+  public void alterTable(TajoMaster.MasterContext context, final QueryContext queryContext,
+                         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 = queryContext.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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
new file mode 100644
index 0000000..3585ae7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.exec;
+
+import com.google.protobuf.ByteString;
+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.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+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.planner.physical.EvalExprExec;
+import org.apache.tajo.engine.planner.physical.StoreTableExec;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
+import org.apache.tajo.master.NonForwardQueryResultScanner;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.exec.prehook.CreateTableHook;
+import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
+import org.apache.tajo.master.exec.prehook.InsertIntoHook;
+import org.apache.tajo.master.querymaster.*;
+import org.apache.tajo.master.session.Session;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.plan.Target;
+import org.apache.tajo.plan.expr.EvalNode;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.plan.verifier.VerifyException;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.util.ProtoUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class QueryExecutor {
+  private static final Log LOG = LogFactory.getLog(QueryExecutor.class);
+
+  private final TajoMaster.MasterContext context;
+  private final CatalogService catalog;
+  private final DistributedQueryHookManager hookManager;
+  private final DDLExecutor ddlExecutor;
+
+  public QueryExecutor(TajoMaster.MasterContext context, DDLExecutor ddlExecutor) {
+    this.context = context;
+    this.catalog = context.getCatalog();
+
+    this.ddlExecutor = ddlExecutor;
+    this.hookManager = new DistributedQueryHookManager();
+    this.hookManager.addHook(new CreateTableHook());
+    this.hookManager.addHook(new InsertIntoHook());
+  }
+
+  public SubmitQueryResponse execute(QueryContext queryContext, Session session, String sql, String jsonExpr,
+                      LogicalPlan plan) throws Exception {
+
+    SubmitQueryResponse.Builder response = SubmitQueryResponse.newBuilder();
+    response.setIsForwarded(false);
+    response.setUserName(queryContext.get(SessionVars.USERNAME));
+
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+
+    if (PlannerUtil.checkIfSetSession(rootNode)) {
+      execSetSession(session, plan, response);
+
+
+    } else if (PlannerUtil.checkIfDDLPlan(rootNode)) {
+      context.getSystemMetrics().counter("Query", "numDDLQuery").inc();
+      ddlExecutor.execute(queryContext, plan);
+      response.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      response.setResultCode(ClientProtos.ResultCode.OK);
+
+
+    } else if (plan.isExplain()) { // explain query
+      execExplain(plan, response);
+
+
+      // Simple query indicates a form of 'select * from tb_name [LIMIT X];'.
+    } else if (PlannerUtil.checkIfSimpleQuery(plan)) {
+      execSimpleQuery(queryContext, session, sql, plan, response);
+
+
+      // NonFromQuery indicates a form of 'select a, x+y;'
+    } else if (PlannerUtil.checkIfNonFromQuery(plan)) {
+      execNonFromQuery(queryContext, session, sql, plan, response);
+
+
+    } else { // it requires distributed execution. So, the query is forwarded to a query master.
+      executeDistributedQuery(queryContext, session, plan, sql, jsonExpr, response);
+    }
+
+    response.setSessionVars(ProtoUtil.convertFromMap(session.getAllVariables()));
+
+    return response.build();
+  }
+
+  public void execSetSession(Session session, LogicalPlan plan,
+                             SubmitQueryResponse.Builder response) {
+    SetSessionNode setSessionNode = ((LogicalRootNode)plan.getRootBlock().getRoot()).getChild();
+
+    final String varName = setSessionNode.getName();
+
+    // SET CATALOG 'XXX'
+    if (varName.equals(SessionVars.CURRENT_DATABASE.name())) {
+      String databaseName = setSessionNode.getValue();
+
+      if (catalog.existDatabase(databaseName)) {
+        session.selectDatabase(setSessionNode.getValue());
+      } else {
+        response.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+        response.setResultCode(ClientProtos.ResultCode.ERROR);
+        response.setErrorMessage("database \"" + databaseName + "\" does not exists.");
+      }
+
+      // others
+    } else {
+      if (setSessionNode.isDefaultValue()) {
+        session.removeVariable(varName);
+      } else {
+        session.setVariable(varName, setSessionNode.getValue());
+      }
+    }
+
+    context.getSystemMetrics().counter("Query", "numDDLQuery").inc();
+    response.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+    response.setResultCode(ClientProtos.ResultCode.OK);
+  }
+
+  public void execExplain(LogicalPlan plan, SubmitQueryResponse.Builder response) throws IOException {
+
+    String explainStr = PlannerUtil.buildExplainString(plan.getRootBlock().getRoot());
+    Schema schema = new Schema();
+    schema.addColumn("explain", TajoDataTypes.Type.TEXT);
+    RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(schema);
+
+    ClientProtos.SerializedResultSet.Builder serializedResBuilder = ClientProtos.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);
+
+    response.setResultSet(serializedResBuilder.build());
+    response.setMaxRowNum(lines.length);
+    response.setResultCode(ClientProtos.ResultCode.OK);
+    response.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+  }
+
+  public void execSimpleQuery(QueryContext queryContext, Session session, String query, LogicalPlan plan,
+                              SubmitQueryResponse.Builder response) throws Exception {
+    ScanNode scanNode = plan.getRootBlock().getNode(NodeType.SCAN);
+    if (scanNode == null) {
+      scanNode = plan.getRootBlock().getNode(NodeType.PARTITIONS_SCAN);
+    }
+    TableDesc desc = scanNode.getTableDesc();
+    int maxRow = Integer.MAX_VALUE;
+    if (plan.getRootBlock().hasNode(NodeType.LIMIT)) {
+      LimitNode limitNode = plan.getRootBlock().getNode(NodeType.LIMIT);
+      maxRow = (int) limitNode.getFetchFirstNum();
+    }
+    if (desc.getStats().getNumRows() == 0) {
+      desc.getStats().setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
+    }
+
+    QueryId queryId = QueryIdFactory.newQueryId(context.getResourceManager().getSeedQueryId());
+
+    NonForwardQueryResultScanner queryResultScanner =
+        new NonForwardQueryResultScanner(context.getConf(), session.getSessionId(), queryId, scanNode, desc, maxRow);
+
+    queryResultScanner.init();
+    session.addNonForwardQueryResultScanner(queryResultScanner);
+
+    response.setQueryId(queryId.getProto());
+    response.setMaxRowNum(maxRow);
+    response.setTableDesc(desc.getProto());
+    response.setResultCode(ClientProtos.ResultCode.OK);
+  }
+
+  public void execNonFromQuery(QueryContext queryContext, Session session, String query,
+                               LogicalPlan plan, SubmitQueryResponse.Builder responseBuilder) throws Exception {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+
+    Target[] targets = plan.getRootBlock().getRawTargets();
+    if (targets == null) {
+      throw new PlanningException("No targets");
+    }
+    final 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));
+    }
+    boolean isInsert = rootNode.getChild() != null && rootNode.getChild().getType() == NodeType.INSERT;
+    if (isInsert) {
+      InsertNode insertNode = rootNode.getChild();
+      insertNonFromQuery(queryContext, insertNode, responseBuilder);
+    } else {
+      Schema schema = PlannerUtil.targetToSchema(targets);
+      RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(schema);
+      byte[] serializedBytes = encoder.toBytes(outTuple);
+      ClientProtos.SerializedResultSet.Builder serializedResBuilder = ClientProtos.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);
+    }
+  }
+
+  private void insertNonFromQuery(QueryContext queryContext,
+                                  InsertNode insertNode, SubmitQueryResponse.Builder responseBuilder)
+      throws Exception {
+    String nodeUniqName = insertNode.getTableName() == null ? insertNode.getPath().getName() : insertNode.getTableName();
+    String queryId = nodeUniqName + "_" + System.currentTimeMillis();
+
+    FileSystem fs = TajoConf.getWarehouseDir(context.getConf()).getFileSystem(context.getConf());
+    Path stagingDir = QueryMasterTask.initStagingDir(context.getConf(), queryId.toString(), queryContext);
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+
+    TableDesc tableDesc = null;
+    Path finalOutputDir = null;
+    if (insertNode.getTableName() != null) {
+      tableDesc = this.catalog.getTableDesc(insertNode.getTableName());
+      finalOutputDir = new Path(tableDesc.getPath());
+    } else {
+      finalOutputDir = insertNode.getPath();
+    }
+
+    TaskAttemptContext taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
+    taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
+
+    EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
+    StoreTableExec exec = new StoreTableExec(taskAttemptContext, insertNode, evalExprExec);
+    try {
+      exec.init();
+      exec.next();
+    } finally {
+      exec.close();
+    }
+
+    if (insertNode.isOverwrite()) { // 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(stagingDir, 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 {
+      FileStatus[] files = fs.listStatus(stagingResultDir);
+      for (FileStatus eachFile: files) {
+        Path targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName());
+        if (fs.exists(targetFilePath)) {
+          targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName() + "_" + System.currentTimeMillis());
+        }
+        fs.rename(eachFile.getPath(), targetFilePath);
+      }
+    }
+
+    if (insertNode.hasTargetTable()) {
+      TableStats stats = tableDesc.getStats();
+      long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
+      stats.setNumBytes(volume);
+      stats.setNumRows(1);
+
+      CatalogProtos.UpdateTableStatsProto.Builder builder = CatalogProtos.UpdateTableStatsProto.newBuilder();
+      builder.setTableName(tableDesc.getName());
+      builder.setStats(stats.getProto());
+
+      catalog.updateTableStats(builder.build());
+
+      responseBuilder.setTableDesc(tableDesc.getProto());
+    } else {
+      TableStats stats = new TableStats();
+      long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
+      stats.setNumBytes(volume);
+      stats.setNumRows(1);
+
+      // Empty TableDesc
+      List<CatalogProtos.ColumnProto> columns = new ArrayList<CatalogProtos.ColumnProto>();
+      CatalogProtos.TableDescProto tableDescProto = CatalogProtos.TableDescProto.newBuilder()
+          .setTableName(nodeUniqName)
+          .setMeta(CatalogProtos.TableProto.newBuilder().setStoreType(CatalogProtos.StoreType.CSV).build())
+          .setSchema(CatalogProtos.SchemaProto.newBuilder().addAllFields(columns).build())
+          .setStats(stats.getProto())
+          .build();
+
+      responseBuilder.setTableDesc(tableDescProto);
+    }
+
+    // If queryId == NULL_QUERY_ID and MaxRowNum == -1, TajoCli prints only number of inserted rows.
+    responseBuilder.setMaxRowNum(-1);
+    responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+    responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+  }
+
+  public void executeDistributedQuery(QueryContext queryContext, Session session,
+                                      LogicalPlan plan,
+                                      String sql,
+                                      String jsonExpr,
+                                      SubmitQueryResponse.Builder responseBuilder) throws Exception {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+
+    CatalogProtos.StoreType storeType = PlannerUtil.getStoreType(plan);
+    if (storeType != null) {
+      StorageManager sm = StorageManager.getStorageManager(context.getConf(), storeType);
+      StorageProperty storageProperty = sm.getStorageProperty();
+      if (!storageProperty.isSupportsInsertInto()) {
+        throw new VerifyException("Inserting into non-file storage is not supported.");
+      }
+      sm.beforeInsertOrCATS(rootNode.getChild());
+    }
+    context.getSystemMetrics().counter("Query", "numDMLQuery").inc();
+    hookManager.doHooks(queryContext, plan);
+
+    QueryJobManager queryJobManager = this.context.getQueryJobManager();
+    QueryInfo queryInfo;
+
+    queryInfo = queryJobManager.createNewQueryJob(session, queryContext, sql, jsonExpr, rootNode);
+
+    if(queryInfo == null) {
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
+      responseBuilder.setErrorMessage("Fail starting QueryMaster.");
+      LOG.error("Fail starting QueryMaster: " + sql);
+    } 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 " + queryInfo.getQueryId().toString() + "," + queryInfo.getSql() + "," +
+          " is forwarded to " + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
new file mode 100644
index 0000000..5e3d0b6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.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.exec.prehook;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.logical.CreateTableNode;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.plan.logical.NodeType;
+import org.apache.tajo.storage.StorageUtil;
+
+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(queryContext.getConf()), databaseName, tableName));
+    if(createTableNode.getPartitionMethod() != null) {
+      queryContext.setPartitionMethod(createTableNode.getPartitionMethod());
+    }
+    queryContext.setCreateTable();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHook.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHook.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHook.java
new file mode 100644
index 0000000..77c6ff4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHook.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.master.exec.prehook;
+
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.plan.LogicalPlan;
+
+public interface DistributedQueryHook {
+  boolean isEligible(QueryContext queryContext, LogicalPlan plan);
+  void hook(QueryContext queryContext, LogicalPlan plan) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java
new file mode 100644
index 0000000..3dba176
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.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.exec.prehook;
+
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.plan.LogicalPlan;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public 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();
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a4c34842/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java
new file mode 100644
index 0000000..14c4d8d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.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.exec.prehook;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.logical.InsertNode;
+import org.apache.tajo.plan.logical.NodeType;
+
+public class InsertIntoHook 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());
+      if (insertNode.hasPartition()) {
+        queryContext.setPartitionMethod(insertNode.getPartitionMethod());
+      }
+    } 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();
+    }
+
+  }
+}


[7/9] tajo git commit: TAJO-1262: Rename the prefix 'SubQuery' to 'Stage'.

Posted by ji...@apache.org.
TAJO-1262: Rename the prefix 'SubQuery' to 'Stage'.

Closes #314


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

Branch: refs/heads/index_support
Commit: 3c833e2a8c3ff7ff8a2e1b4497afb390098856bf
Parents: cf66a39
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 22 16:43:39 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 22 16:43:39 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |    2 +
 tajo-client/src/main/proto/ClientProtos.proto   |    4 +-
 .../tajo/master/DefaultTaskScheduler.java       |   36 +-
 .../apache/tajo/master/LazyTaskScheduler.java   |   30 +-
 .../tajo/master/TaskSchedulerFactory.java       |   12 +-
 .../tajo/master/event/QueryCompletedEvent.java  |    8 +-
 .../tajo/master/event/QueryEventType.java       |    4 +-
 .../tajo/master/event/QuerySubQueryEvent.java   |   35 -
 .../tajo/master/event/StageCompletedEvent.java  |   42 +
 .../event/StageContainerAllocationEvent.java    |   38 +
 .../event/StageDiagnosticsUpdateEvent.java      |   34 +
 .../apache/tajo/master/event/StageEvent.java    |   35 +
 .../tajo/master/event/StageEventType.java       |   43 +
 .../tajo/master/event/StageTaskEvent.java       |   43 +
 .../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 -
 .../apache/tajo/master/event/TaskEventType.java |    4 +-
 .../apache/tajo/master/querymaster/Query.java   |  164 +--
 .../querymaster/QueryMasterManagerService.java  |    4 +-
 .../master/querymaster/QueryMasterTask.java     |   28 +-
 .../tajo/master/querymaster/Repartitioner.java  |  172 +--
 .../apache/tajo/master/querymaster/Stage.java   | 1342 +++++++++++++++++
 .../tajo/master/querymaster/StageState.java     |   30 +
 .../tajo/master/querymaster/SubQuery.java       | 1343 ------------------
 .../tajo/master/querymaster/SubQueryState.java  |   30 -
 .../apache/tajo/master/querymaster/Task.java    |   10 +-
 .../tajo/master/querymaster/TaskAttempt.java    |    6 +-
 .../main/java/org/apache/tajo/util/JSPUtil.java |   36 +-
 .../apache/tajo/util/history/HistoryReader.java |    2 +-
 .../apache/tajo/util/history/HistoryWriter.java |   12 +-
 .../apache/tajo/util/history/QueryHistory.java  |   23 +-
 .../apache/tajo/util/history/StageHistory.java  |  270 ++++
 .../tajo/util/history/SubQueryHistory.java      |  270 ----
 .../tajo/worker/TajoResourceAllocator.java      |   14 +-
 .../main/java/org/apache/tajo/worker/Task.java  |   20 +-
 .../apache/tajo/worker/TaskAttemptContext.java  |    2 +-
 .../resources/webapps/admin/querydetail.jsp     |   32 +-
 .../main/resources/webapps/admin/querytasks.jsp |   36 +-
 .../resources/webapps/worker/querydetail.jsp    |   30 +-
 .../main/resources/webapps/worker/queryplan.jsp |   52 +-
 .../resources/webapps/worker/querytasks.jsp     |   18 +-
 .../src/main/resources/webapps/worker/task.jsp  |   10 +-
 .../org/apache/tajo/TajoTestingCluster.java     |    8 +-
 .../org/apache/tajo/TestQueryIdFactory.java     |    8 +-
 .../org/apache/tajo/client/TestTajoClient.java  |   12 +-
 .../tajo/engine/query/TestGroupByQuery.java     |   16 +-
 .../tajo/engine/query/TestTablePartitions.java  |    2 +-
 .../tajo/engine/query/TestUnionQuery.java       |    6 +-
 .../tajo/master/querymaster/TestKillQuery.java  |    8 +-
 .../querymaster/TestTaskStatusUpdate.java       |   18 +-
 .../util/history/TestHistoryWriterReader.java   |   26 +-
 tajo-dist/pom.xml                               |    2 +-
 .../tajo/pullserver/PullServerAuxService.java   |   16 +-
 .../tajo/pullserver/TajoPullServerService.java  |   16 +-
 .../apache/tajo/storage/FileStorageManager.java |    6 +-
 59 files changed, 2335 insertions(+), 2370 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 245918e..29b0c0b 100644
--- a/CHANGES
+++ b/CHANGES
@@ -240,6 +240,8 @@ Release 0.9.1 - unreleased
 
   SUB TASKS
 
+    TAJO-1262: Rename the prefix 'SubQuery' to 'Stage'. (hyunsik)
+
     TAJO-324: Rename the prefix 'QueryUnit' to Task. (hyunsik)
 
     TAJO-1151: Implement the ByteBuffer-based De/Serializer. (jinho)

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-client/src/main/proto/ClientProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/proto/ClientProtos.proto b/tajo-client/src/main/proto/ClientProtos.proto
index 51db763..a741268 100644
--- a/tajo-client/src/main/proto/ClientProtos.proto
+++ b/tajo-client/src/main/proto/ClientProtos.proto
@@ -253,7 +253,7 @@ message QueryInfoProto {
   optional int32 queryMasterInfoPort = 11;
 }
 
-message SubQueryHistoryProto {
+message StageHistoryProto {
   required string executionBlockId =1;
   required string state = 2;
   optional int64 startTime = 3;
@@ -283,7 +283,7 @@ message QueryHistoryProto {
   optional string logicalPlan = 4;
   optional string distributedPlan = 5;
   repeated KeyValueProto sessionVariables = 6;
-  repeated SubQueryHistoryProto subQueryHistories = 7;
+  repeated StageHistoryProto stageHistories = 7;
 }
 
 message GetQueryHistoryResponse {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index d9d496e..dd6233c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
@@ -40,7 +40,7 @@ import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptSched
 import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
 import org.apache.tajo.master.querymaster.Task;
 import org.apache.tajo.master.querymaster.TaskAttempt;
-import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.Stage;
 import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.storage.DataLocation;
 import org.apache.tajo.storage.fragment.FileFragment;
@@ -60,7 +60,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
   private static final Log LOG = LogFactory.getLog(DefaultTaskScheduler.class);
 
   private final TaskSchedulerContext context;
-  private SubQuery subQuery;
+  private Stage stage;
 
   private Thread schedulingThread;
   private AtomicBoolean stopEventHandling = new AtomicBoolean(false);
@@ -71,10 +71,10 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
   private int nextTaskId = 0;
   private int scheduledObjectNum = 0;
 
-  public DefaultTaskScheduler(TaskSchedulerContext context, SubQuery subQuery) {
+  public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) {
     super(DefaultTaskScheduler.class.getName());
     this.context = context;
-    this.subQuery = subQuery;
+    this.stage = stage;
   }
 
   @Override
@@ -117,8 +117,8 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
   private static final TaskAttemptId NULL_ATTEMPT_ID;
   public static final TajoWorkerProtocol.TaskRequestProto stopTaskRunnerReq;
   static {
-    ExecutionBlockId nullSubQuery = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
-    NULL_ATTEMPT_ID = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullSubQuery, 0), 0);
+    ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
+    NULL_ATTEMPT_ID = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, 0), 0);
 
     TajoWorkerProtocol.TaskRequestProto.Builder builder =
         TajoWorkerProtocol.TaskRequestProto.newBuilder();
@@ -192,13 +192,13 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
         FragmentScheduleEvent castEvent = (FragmentScheduleEvent) event;
         if (context.isLeafQuery()) {
           TaskAttemptScheduleContext taskContext = new TaskAttemptScheduleContext();
-          Task task = SubQuery.newEmptyTask(context, taskContext, subQuery, nextTaskId++);
+          Task task = Stage.newEmptyTask(context, taskContext, stage, nextTaskId++);
           task.addFragment(castEvent.getLeftFragment(), true);
           scheduledObjectNum++;
           if (castEvent.hasRightFragments()) {
             task.addFragments(castEvent.getRightFragments());
           }
-          subQuery.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+          stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
         } else {
           fragmentsForNonLeafTask = new FileFragment[2];
           fragmentsForNonLeafTask[0] = castEvent.getLeftFragment();
@@ -217,7 +217,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
         FetchScheduleEvent castEvent = (FetchScheduleEvent) event;
         Map<String, List<FetchImpl>> fetches = castEvent.getFetches();
         TaskAttemptScheduleContext taskScheduleContext = new TaskAttemptScheduleContext();
-        Task task = SubQuery.newEmptyTask(context, taskScheduleContext, subQuery, nextTaskId++);
+        Task task = Stage.newEmptyTask(context, taskScheduleContext, stage, nextTaskId++);
         scheduledObjectNum++;
         for (Entry<String, List<FetchImpl>> eachFetch : fetches.entrySet()) {
           task.addFetches(eachFetch.getKey(), eachFetch.getValue());
@@ -229,7 +229,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
         if (broadcastFragmentsForNonLeafTask != null && broadcastFragmentsForNonLeafTask.length > 0) {
           task.addFragments(Arrays.asList(broadcastFragmentsForNonLeafTask));
         }
-        subQuery.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+        stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
       } else if (event instanceof TaskAttemptToSchedulerEvent) {
         TaskAttemptToSchedulerEvent castEvent = (TaskAttemptToSchedulerEvent) event;
         if (context.isLeafQuery()) {
@@ -239,7 +239,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
         }
       }
     } else if (event.getType() == EventType.T_SCHEDULE_CANCEL) {
-      // when a subquery is killed, unassigned query unit attmpts are canceled from the scheduler.
+      // when a stage is killed, unassigned query unit attmpts are canceled from the scheduler.
       // This event is triggered by TaskAttempt.
       TaskAttemptToSchedulerEvent castedEvent = (TaskAttemptToSchedulerEvent) event;
       scheduledRequests.leafTasks.remove(castedEvent.getTaskAttempt().getId());
@@ -832,7 +832,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
         }
 
         if (attemptId != null) {
-          Task task = subQuery.getTask(attemptId.getTaskId());
+          Task task = stage.getTask(attemptId.getTaskId());
           TaskRequest taskAssign = new TaskRequestImpl(
               attemptId,
               new ArrayList<FragmentProto>(task.getAllFragments()),
@@ -840,8 +840,8 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
               false,
               task.getLogicalPlan().toJson(),
               context.getMasterContext().getQueryContext(),
-              subQuery.getDataChannel(), subQuery.getBlock().getEnforcer());
-          if (checkIfInterQuery(subQuery.getMasterPlan(), subQuery.getBlock())) {
+              stage.getDataChannel(), stage.getBlock().getEnforcer());
+          if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) {
             taskAssign.setInterQuery();
           }
 
@@ -888,7 +888,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
           LOG.debug("Assigned based on * match");
 
           Task task;
-          task = subQuery.getTask(attemptId.getTaskId());
+          task = stage.getTask(attemptId.getTaskId());
           TaskRequest taskAssign = new TaskRequestImpl(
               attemptId,
               Lists.newArrayList(task.getAllFragments()),
@@ -896,9 +896,9 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
               false,
               task.getLogicalPlan().toJson(),
               context.getMasterContext().getQueryContext(),
-              subQuery.getDataChannel(),
-              subQuery.getBlock().getEnforcer());
-          if (checkIfInterQuery(subQuery.getMasterPlan(), subQuery.getBlock())) {
+              stage.getDataChannel(),
+              stage.getBlock().getEnforcer());
+          if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) {
             taskAssign.setInterQuery();
           }
           for(Map.Entry<String, Set<FetchImpl>> entry: task.getFetchMap().entrySet()) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 0ab19db..32af17b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
@@ -37,7 +37,7 @@ import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptSched
 import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
 import org.apache.tajo.master.querymaster.Task;
 import org.apache.tajo.master.querymaster.TaskAttempt;
-import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.Stage;
 import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.fragment.FileFragment;
@@ -57,7 +57,7 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
   private static final Log LOG = LogFactory.getLog(LazyTaskScheduler.class);
 
   private final TaskSchedulerContext context;
-  private final SubQuery subQuery;
+  private final Stage stage;
 
   private Thread schedulingThread;
   private volatile boolean stopEventHandling;
@@ -77,10 +77,10 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
   private int nextTaskId = 0;
   private int containerNum;
 
-  public LazyTaskScheduler(TaskSchedulerContext context, SubQuery subQuery) {
+  public LazyTaskScheduler(TaskSchedulerContext context, Stage stage) {
     super(LazyTaskScheduler.class.getName());
     this.context = context;
-    this.subQuery = subQuery;
+    this.stage = stage;
   }
 
   @Override
@@ -101,8 +101,8 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
 
   @Override
   public void start() {
-    containerNum = subQuery.getContext().getResourceAllocator().calculateNumRequestContainers(
-        subQuery.getContext().getQueryMasterContext().getWorkerContext(),
+    containerNum = stage.getContext().getResourceAllocator().calculateNumRequestContainers(
+        stage.getContext().getQueryMasterContext().getWorkerContext(),
         context.getEstimatedTaskNum(), 512);
 
     LOG.info("Start TaskScheduler");
@@ -129,8 +129,8 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
   private static final TaskAttemptId NULL_ATTEMPT_ID;
   public static final TajoWorkerProtocol.TaskRequestProto stopTaskRunnerReq;
   static {
-    ExecutionBlockId nullSubQuery = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
-    NULL_ATTEMPT_ID = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullSubQuery, 0), 0);
+    ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
+    NULL_ATTEMPT_ID = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, 0), 0);
 
     TajoWorkerProtocol.TaskRequestProto.Builder builder =
         TajoWorkerProtocol.TaskRequestProto.newBuilder();
@@ -362,7 +362,7 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
       String host = container.getTaskHostName();
       TaskAttemptScheduleContext taskContext = new TaskAttemptScheduleContext(container.containerID,
           host, taskRequest.getCallback());
-      Task task = SubQuery.newEmptyTask(context, taskContext, subQuery, nextTaskId++);
+      Task task = Stage.newEmptyTask(context, taskContext, stage, nextTaskId++);
 
       FragmentPair fragmentPair;
       List<FragmentPair> fragmentPairs = new ArrayList<FragmentPair>();
@@ -371,7 +371,7 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
       long taskSize = adjustTaskSize();
       LOG.info("Adjusted task size: " + taskSize);
 
-      TajoConf conf = subQuery.getContext().getConf();
+      TajoConf conf = stage.getContext().getConf();
       // host local, disk local
       String normalized = NetUtils.normalizeHost(host);
       Integer diskId = hostDiskBalancerMap.get(normalized).getDiskId(container.containerID);
@@ -450,7 +450,7 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
       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));
+      stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
     }
   }
 
@@ -469,9 +469,9 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
             taskRequest.getContainerId());
         TaskAttemptScheduleContext taskScheduleContext = new TaskAttemptScheduleContext(container.containerID,
             container.getTaskHostName(), taskRequest.getCallback());
-        Task task = SubQuery.newEmptyTask(context, taskScheduleContext, subQuery, nextTaskId++);
+        Task task = Stage.newEmptyTask(context, taskScheduleContext, stage, nextTaskId++);
         task.setFragment(scheduledFragments.getAllFragments());
-        subQuery.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+        stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
       }
     }
   }
@@ -485,8 +485,8 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
         false,
         taskAttempt.getTask().getLogicalPlan().toJson(),
         context.getMasterContext().getQueryContext(),
-        subQuery.getDataChannel(), subQuery.getBlock().getEnforcer());
-    if (checkIfInterQuery(subQuery.getMasterPlan(), subQuery.getBlock())) {
+        stage.getDataChannel(), stage.getBlock().getEnforcer());
+    if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) {
       taskAssign.setInterQuery();
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 520ecd3..e5291e9 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
@@ -20,7 +20,7 @@ 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 org.apache.tajo.master.querymaster.Stage;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -29,7 +29,7 @@ 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 };
+  private static final Class<?>[] DEFAULT_PARAMS = { TaskSchedulerContext.class, Stage.class };
 
   public static Class<? extends AbstractTaskScheduler> getTaskSchedulerClass(Configuration conf)
       throws IOException {
@@ -46,7 +46,7 @@ public class TaskSchedulerFactory {
   }
 
   public static <T extends AbstractTaskScheduler> T get(Class<T> clazz, TaskSchedulerContext context,
-                                                        SubQuery subQuery) {
+                                                        Stage stage) {
     T result;
     try {
       Constructor<T> constructor = (Constructor<T>) CONSTRUCTOR_CACHE.get(clazz);
@@ -55,15 +55,15 @@ public class TaskSchedulerFactory {
         constructor.setAccessible(true);
         CONSTRUCTOR_CACHE.put(clazz, constructor);
       }
-      result = constructor.newInstance(new Object[]{context, subQuery});
+      result = constructor.newInstance(new Object[]{context, stage});
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
     return result;
   }
 
-  public static AbstractTaskScheduler get(Configuration conf, TaskSchedulerContext context, SubQuery subQuery)
+  public static AbstractTaskScheduler get(Configuration conf, TaskSchedulerContext context, Stage stage)
       throws IOException {
-    return get(getTaskSchedulerClass(conf), context, subQuery);
+    return get(getTaskSchedulerClass(conf), context, stage);
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index dc75a1d..e5a9a32 100644
--- 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
@@ -19,14 +19,14 @@
 package org.apache.tajo.master.event;
 
 import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.master.querymaster.SubQueryState;
+import org.apache.tajo.master.querymaster.StageState;
 
 public class QueryCompletedEvent extends QueryEvent {
   private final ExecutionBlockId executionBlockId;
-  private final SubQueryState finalState;
+  private final StageState finalState;
 
   public QueryCompletedEvent(final ExecutionBlockId executionBlockId,
-                             SubQueryState finalState) {
+                             StageState finalState) {
     super(executionBlockId.getQueryId(), QueryEventType.QUERY_COMPLETED);
     this.executionBlockId = executionBlockId;
     this.finalState = finalState;
@@ -36,7 +36,7 @@ public class QueryCompletedEvent extends QueryEvent {
     return executionBlockId;
   }
 
-  public SubQueryState getState() {
+  public StageState getState() {
     return finalState;
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index edc0cd8..e38a3c4 100644
--- 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
@@ -24,8 +24,8 @@ public enum QueryEventType {
   START,
   KILL,
 
-  // Producer: SubQuery
-  SUBQUERY_COMPLETED,
+  // Producer: Stage
+  STAGE_COMPLETED,
 
   // Producer: Query
   QUERY_COMPLETED,

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
deleted file mode 100644
index ae36a69..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.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/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java
new file mode 100644
index 0000000..2d16fbe
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.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.StageState;
+
+public class StageCompletedEvent extends QueryEvent {
+  private final ExecutionBlockId executionBlockId;
+  private final StageState finalState;
+
+  public StageCompletedEvent(final ExecutionBlockId executionBlockId,
+                             StageState finalState) {
+    super(executionBlockId.getQueryId(), QueryEventType.STAGE_COMPLETED);
+    this.executionBlockId = executionBlockId;
+    this.finalState = finalState;
+  }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
+
+  public StageState getState() {
+    return finalState;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/event/StageContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageContainerAllocationEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageContainerAllocationEvent.java
new file mode 100644
index 0000000..0d29e44
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageContainerAllocationEvent.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.tajo.ExecutionBlockId;
+import org.apache.tajo.master.container.TajoContainer;
+
+import java.util.List;
+
+public class StageContainerAllocationEvent extends StageEvent {
+  private List<TajoContainer> allocatedContainer;
+
+  public StageContainerAllocationEvent(final ExecutionBlockId id,
+                                       List<TajoContainer> allocatedContainer) {
+    super(id, StageEventType.SQ_CONTAINER_ALLOCATED);
+    this.allocatedContainer = allocatedContainer;
+  }
+
+  public List<TajoContainer> getAllocatedContainer() {
+    return this.allocatedContainer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/event/StageDiagnosticsUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageDiagnosticsUpdateEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageDiagnosticsUpdateEvent.java
new file mode 100644
index 0000000..39afc92
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageDiagnosticsUpdateEvent.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 StageDiagnosticsUpdateEvent extends StageEvent {
+  private final String msg;
+
+  public StageDiagnosticsUpdateEvent(final ExecutionBlockId id, String diagnostic) {
+    super(id, StageEventType.SQ_DIAGNOSTIC_UPDATE);
+    this.msg = diagnostic;
+  }
+
+  public String getDiagnosticUpdate() {
+    return msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/event/StageEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageEvent.java
new file mode 100644
index 0000000..6fc4746
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageEvent.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 StageEvent extends AbstractEvent<StageEventType> {
+  private final ExecutionBlockId id;
+
+  public StageEvent(ExecutionBlockId id, StageEventType stageEventType) {
+    super(stageEventType);
+    this.id = id;
+  }
+
+  public ExecutionBlockId getStageId() {
+    return id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.java
new file mode 100644
index 0000000..fa808d4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.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 Stage
+ */
+public enum StageEventType {
+
+  // Producer: Query
+  SQ_INIT,
+  SQ_START,
+  SQ_CONTAINER_ALLOCATED,
+  SQ_KILL,
+  SQ_LAUNCH,
+
+  // Producer: Task
+  SQ_TASK_COMPLETED,
+  SQ_FAILED,
+
+  // Producer: Completed
+  SQ_STAGE_COMPLETED,
+
+  // Producer: Any component
+  SQ_DIAGNOSTIC_UPDATE,
+  SQ_INTERNAL_ERROR
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/event/StageTaskEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageTaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageTaskEvent.java
new file mode 100644
index 0000000..4377881
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageTaskEvent.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.TaskId;
+import org.apache.tajo.master.TaskState;
+
+/**
+ * Event Class: From Task to Stage
+ */
+public class StageTaskEvent extends StageEvent {
+  private TaskId taskId;
+  private TaskState state;
+  public StageTaskEvent(TaskId taskId, TaskState state) {
+    super(taskId.getExecutionBlockId(), StageEventType.SQ_TASK_COMPLETED);
+    this.taskId = taskId;
+    this.state = state;
+  }
+
+  public TaskId getTaskId() {
+    return this.taskId;
+  }
+
+  public TaskState getState() {
+    return state;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
deleted file mode 100644
index 0810e81..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryDiagnosticsUpdateEvent.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package 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/3c833e2a/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
deleted file mode 100644
index 2b3d598..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.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/3c833e2a/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
deleted file mode 100644
index 79b6e2e..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package 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: Task
-  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/3c833e2a/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
deleted file mode 100644
index 816bc48..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.event;
-
-import org.apache.tajo.TaskId;
-import org.apache.tajo.master.TaskState;
-
-/**
- * Event Class: From Task to SubQuery
- */
-public class SubQueryTaskEvent extends SubQueryEvent {
-  private TaskId taskId;
-  private TaskState state;
-  public SubQueryTaskEvent(TaskId taskId, TaskState state) {
-    super(taskId.getExecutionBlockId(), SubQueryEventType.SQ_TASK_COMPLETED);
-    this.taskId = taskId;
-    this.state = state;
-  }
-
-  public TaskId getTaskId() {
-    return this.taskId;
-  }
-
-  public TaskState getState() {
-    return state;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 9448863..0f26821 100644
--- 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
@@ -23,10 +23,10 @@ package org.apache.tajo.master.event;
  */
 public enum TaskEventType {
 
-  //Producer:Client, SubQuery
+  //Producer:Client, Stage
   T_KILL,
 
-  //Producer:SubQuery
+  //Producer:Stage
   T_SCHEDULE,
 
   //Producer:TaskAttempt

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 918cc82..a626df1 100644
--- 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
@@ -50,7 +50,7 @@ import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.util.history.QueryHistory;
-import org.apache.tajo.util.history.SubQueryHistory;
+import org.apache.tajo.util.history.StageHistory;
 
 import java.io.IOException;
 import java.util.*;
@@ -65,7 +65,7 @@ public class Query implements EventHandler<QueryEvent> {
   private final TajoConf systemConf;
   private final Clock clock;
   private String queryStr;
-  private Map<ExecutionBlockId, SubQuery> subqueries;
+  private Map<ExecutionBlockId, Stage> stages;
   private final EventHandler eventHandler;
   private final MasterPlan plan;
   QueryMasterTask.QueryMasterTaskContext context;
@@ -77,11 +77,11 @@ public class Query implements EventHandler<QueryEvent> {
   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 int completedStagesCount = 0;
+  private int successedStagesCount = 0;
+  private int killedStagesCount = 0;
+  private int failedStagesCount = 0;
+  private int erroredStagesCount = 0;
   private final List<String> diagnostics = new ArrayList<String>();
 
   // Internal Variables
@@ -96,7 +96,7 @@ public class Query implements EventHandler<QueryEvent> {
   // 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 StageCompletedTransition STAGE_COMPLETED_TRANSITION = new StageCompletedTransition();
   private static final QueryCompletedTransition QUERY_COMPLETED_TRANSITION = new QueryCompletedTransition();
 
   protected static final StateMachineFactory
@@ -120,8 +120,8 @@ public class Query implements EventHandler<QueryEvent> {
 
           // Transitions from RUNNING state
           .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
-              QueryEventType.SUBQUERY_COMPLETED,
-              SUBQUERY_COMPLETED_TRANSITION)
+              QueryEventType.STAGE_COMPLETED,
+              STAGE_COMPLETED_TRANSITION)
           .addTransition(QueryState.QUERY_RUNNING,
               EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
                   QueryState.QUERY_ERROR),
@@ -132,7 +132,7 @@ public class Query implements EventHandler<QueryEvent> {
               DIAGNOSTIC_UPDATE_TRANSITION)
           .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_KILL_WAIT,
               QueryEventType.KILL,
-              new KillSubQueriesTransition())
+              new KillAllStagesTransition())
           .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
               QueryEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
@@ -143,8 +143,8 @@ public class Query implements EventHandler<QueryEvent> {
               DIAGNOSTIC_UPDATE_TRANSITION)
           // ignore-able transitions
           .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
-              QueryEventType.SUBQUERY_COMPLETED,
-              SUBQUERY_COMPLETED_TRANSITION)
+              QueryEventType.STAGE_COMPLETED,
+              STAGE_COMPLETED_TRANSITION)
           .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
               QueryEventType.KILL)
           .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_ERROR,
@@ -153,8 +153,8 @@ public class Query implements EventHandler<QueryEvent> {
 
           // Transitions from KILL_WAIT state
           .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
-              QueryEventType.SUBQUERY_COMPLETED,
-              SUBQUERY_COMPLETED_TRANSITION)
+              QueryEventType.STAGE_COMPLETED,
+              STAGE_COMPLETED_TRANSITION)
           .addTransition(QueryState.QUERY_KILL_WAIT,
               EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
                   QueryState.QUERY_ERROR),
@@ -191,7 +191,7 @@ public class Query implements EventHandler<QueryEvent> {
               INTERNAL_ERROR_TRANSITION)
           // Ignore-able transitions
           .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
-              EnumSet.of(QueryEventType.KILL, QueryEventType.SUBQUERY_COMPLETED))
+              EnumSet.of(QueryEventType.KILL, QueryEventType.STAGE_COMPLETED))
 
           .installTopology();
 
@@ -206,7 +206,7 @@ public class Query implements EventHandler<QueryEvent> {
     this.clock = context.getClock();
     this.appSubmitTime = appSubmitTime;
     this.queryStr = queryStr;
-    this.subqueries = Maps.newConcurrentMap();
+    this.stages = Maps.newConcurrentMap();
     this.eventHandler = eventHandler;
     this.plan = plan;
     this.cursor = new ExecutionBlockCursor(plan, true);
@@ -237,15 +237,15 @@ public class Query implements EventHandler<QueryEvent> {
       return 1.0f;
     } else {
       int idx = 0;
-      List<SubQuery> tempSubQueries = new ArrayList<SubQuery>();
-      synchronized(subqueries) {
-        tempSubQueries.addAll(subqueries.values());
+      List<Stage> tempStages = new ArrayList<Stage>();
+      synchronized(stages) {
+        tempStages.addAll(stages.values());
       }
 
-      float [] subProgresses = new float[tempSubQueries.size()];
-      for (SubQuery subquery: tempSubQueries) {
-        if (subquery.getState() != SubQueryState.NEW) {
-          subProgresses[idx] = subquery.getProgress();
+      float [] subProgresses = new float[tempStages.size()];
+      for (Stage stage: tempStages) {
+        if (stage.getState() != StageState.NEW) {
+          subProgresses[idx] = stage.getProgress();
         } else {
           subProgresses[idx] = 0.0f;
         }
@@ -285,17 +285,17 @@ public class Query implements EventHandler<QueryEvent> {
 
   public QueryHistory getQueryHistory() {
     QueryHistory queryHistory = makeQueryHistory();
-    queryHistory.setSubQueryHistories(makeSubQueryHistories());
+    queryHistory.setStageHistories(makeStageHistories());
     return queryHistory;
   }
 
-  private List<SubQueryHistory> makeSubQueryHistories() {
-    List<SubQueryHistory> subQueryHistories = new ArrayList<SubQueryHistory>();
-    for(SubQuery eachSubQuery: getSubQueries()) {
-      subQueryHistories.add(eachSubQuery.getSubQueryHistory());
+  private List<StageHistory> makeStageHistories() {
+    List<StageHistory> stageHistories = new ArrayList<StageHistory>();
+    for(Stage eachStage : getStages()) {
+      stageHistories.add(eachStage.getStageHistory());
     }
 
-    return subQueryHistories;
+    return stageHistories;
   }
 
   private QueryHistory makeQueryHistory() {
@@ -348,20 +348,20 @@ public class Query implements EventHandler<QueryEvent> {
     return stateMachine;
   }
   
-  public void addSubQuery(SubQuery subquery) {
-    subqueries.put(subquery.getId(), subquery);
+  public void addStage(Stage stage) {
+    stages.put(stage.getId(), stage);
   }
   
   public QueryId getId() {
     return this.id;
   }
 
-  public SubQuery getSubQuery(ExecutionBlockId id) {
-    return this.subqueries.get(id);
+  public Stage getStage(ExecutionBlockId id) {
+    return this.stages.get(id);
   }
 
-  public Collection<SubQuery> getSubQueries() {
-    return this.subqueries.values();
+  public Collection<Stage> getStages() {
+    return this.stages.values();
   }
 
   public QueryState getSynchronizedState() {
@@ -389,13 +389,13 @@ public class Query implements EventHandler<QueryEvent> {
     public void transition(Query query, QueryEvent queryEvent) {
 
       query.setStartTime();
-      SubQuery subQuery = new SubQuery(query.context, query.getPlan(),
+      Stage stage = new Stage(query.context, query.getPlan(),
           query.getExecutionBlockCursor().nextBlock());
-      subQuery.setPriority(query.priority--);
-      query.addSubQuery(subQuery);
+      stage.setPriority(query.priority--);
+      query.addStage(stage);
 
-      subQuery.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_INIT));
-      LOG.debug("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
+      stage.handle(new StageEvent(stage.getId(), StageEventType.SQ_INIT));
+      LOG.debug("Schedule unit plan: \n" + stage.getBlock().getPlan());
     }
   }
 
@@ -403,20 +403,20 @@ public class Query implements EventHandler<QueryEvent> {
 
     @Override
     public QueryState transition(Query query, QueryEvent queryEvent) {
-      QueryCompletedEvent subQueryEvent = (QueryCompletedEvent) queryEvent;
+      QueryCompletedEvent stageEvent = (QueryCompletedEvent) queryEvent;
       QueryState finalState;
 
-      if (subQueryEvent.getState() == SubQueryState.SUCCEEDED) {
-        finalState = finalizeQuery(query, subQueryEvent);
-      } else if (subQueryEvent.getState() == SubQueryState.FAILED) {
+      if (stageEvent.getState() == StageState.SUCCEEDED) {
+        finalState = finalizeQuery(query, stageEvent);
+      } else if (stageEvent.getState() == StageState.FAILED) {
         finalState = QueryState.QUERY_FAILED;
-      } else if (subQueryEvent.getState() == SubQueryState.KILLED) {
+      } else if (stageEvent.getState() == StageState.KILLED) {
         finalState = QueryState.QUERY_KILLED;
       } else {
         finalState = QueryState.QUERY_ERROR;
       }
       if (finalState != QueryState.QUERY_SUCCEEDED) {
-        SubQuery lastStage = query.getSubQuery(subQueryEvent.getExecutionBlockId());
+        Stage lastStage = query.getStage(stageEvent.getExecutionBlockId());
         if (lastStage != null && lastStage.getTableMeta() != null) {
           StoreType storeType = lastStage.getTableMeta().getStoreType();
           if (storeType != null) {
@@ -436,7 +436,7 @@ public class Query implements EventHandler<QueryEvent> {
     }
 
     private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
-      SubQuery lastStage = query.getSubQuery(event.getExecutionBlockId());
+      Stage lastStage = query.getStage(event.getExecutionBlockId());
       StoreType storeType = lastStage.getTableMeta().getStoreType();
       try {
         LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
@@ -490,7 +490,7 @@ public class Query implements EventHandler<QueryEvent> {
       @Override
       public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
                                 Path finalOutputDir) {
-        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        Stage lastStage = query.getStage(finalExecBlockId);
         NodeType type = lastStage.getBlock().getPlan().getType();
         return type != NodeType.CREATE_TABLE && type != NodeType.INSERT;
       }
@@ -499,7 +499,7 @@ public class Query implements EventHandler<QueryEvent> {
       public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
                           Query query, ExecutionBlockId finalExecBlockId,
                           Path finalOutputDir) throws Exception {
-        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        Stage lastStage = query.getStage(finalExecBlockId);
         TableMeta meta = lastStage.getTableMeta();
 
         String nullChar = queryContext.get(SessionVars.NULL_CHAR);
@@ -526,7 +526,7 @@ public class Query implements EventHandler<QueryEvent> {
       @Override
       public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
                                 Path finalOutputDir) {
-        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        Stage lastStage = query.getStage(finalExecBlockId);
         return lastStage.getBlock().getPlan().getType() == NodeType.CREATE_TABLE;
       }
 
@@ -534,7 +534,7 @@ public class Query implements EventHandler<QueryEvent> {
       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);
+        Stage lastStage = query.getStage(finalExecBlockId);
         TableStats stats = lastStage.getResultStats();
 
         CreateTableNode createTableNode = (CreateTableNode) lastStage.getBlock().getPlan();
@@ -565,7 +565,7 @@ public class Query implements EventHandler<QueryEvent> {
       @Override
       public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
                                 Path finalOutputDir) {
-        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        Stage lastStage = query.getStage(finalExecBlockId);
         return lastStage.getBlock().getPlan().getType() == NodeType.INSERT;
       }
 
@@ -575,7 +575,7 @@ public class Query implements EventHandler<QueryEvent> {
           throws Exception {
 
         CatalogService catalog = context.getWorkerContext().getCatalog();
-        SubQuery lastStage = query.getSubQuery(finalExecBlockId);
+        Stage lastStage = query.getStage(finalExecBlockId);
         TableMeta meta = lastStage.getTableMeta();
         TableStats stats = lastStage.getResultStats();
 
@@ -613,7 +613,7 @@ public class Query implements EventHandler<QueryEvent> {
     return directorySummary.getLength();
   }
 
-  public static class SubQueryCompletedTransition implements SingleArcTransition<Query, QueryEvent> {
+  public static class StageCompletedTransition implements SingleArcTransition<Query, QueryEvent> {
 
     private boolean hasNext(Query query) {
       ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
@@ -624,43 +624,43 @@ public class Query implements EventHandler<QueryEvent> {
     private void executeNextBlock(Query query) {
       ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
       ExecutionBlock nextBlock = cursor.nextBlock();
-      SubQuery nextSubQuery = new SubQuery(query.context, query.getPlan(), nextBlock);
-      nextSubQuery.setPriority(query.priority--);
-      query.addSubQuery(nextSubQuery);
-      nextSubQuery.handle(new SubQueryEvent(nextSubQuery.getId(), SubQueryEventType.SQ_INIT));
+      Stage nextStage = new Stage(query.context, query.getPlan(), nextBlock);
+      nextStage.setPriority(query.priority--);
+      query.addStage(nextStage);
+      nextStage.handle(new StageEvent(nextStage.getId(), StageEventType.SQ_INIT));
 
-      LOG.info("Scheduling SubQuery:" + nextSubQuery.getId());
+      LOG.info("Scheduling Stage:" + nextStage.getId());
       if(LOG.isDebugEnabled()) {
-        LOG.debug("Scheduling SubQuery's Priority: " + nextSubQuery.getPriority());
-        LOG.debug("Scheduling SubQuery's Plan: \n" + nextSubQuery.getBlock().getPlan());
+        LOG.debug("Scheduling Stage's Priority: " + nextStage.getPriority());
+        LOG.debug("Scheduling Stage's Plan: \n" + nextStage.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++;
+        query.completedStagesCount++;
+        StageCompletedEvent castEvent = (StageCompletedEvent) event;
+
+        if (castEvent.getState() == StageState.SUCCEEDED) {
+          query.successedStagesCount++;
+        } else if (castEvent.getState() == StageState.KILLED) {
+          query.killedStagesCount++;
+        } else if (castEvent.getState() == StageState.FAILED) {
+          query.failedStagesCount++;
+        } else if (castEvent.getState() == StageState.ERROR) {
+          query.erroredStagesCount++;
         } else {
-          LOG.error(String.format("Invalid SubQuery (%s) State %s at %s",
+          LOG.error(String.format("Invalid Stage (%s) State %s at %s",
               castEvent.getExecutionBlockId().toString(), castEvent.getState().name(), query.getSynchronizedState().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
+        // if a stage is succeeded and a query is running
+        if (castEvent.getState() == StageState.SUCCEEDED &&  // latest stage succeeded
             query.getSynchronizedState() == QueryState.QUERY_RUNNING &&     // current state is not in KILL_WAIT, FAILED, or ERROR.
-            hasNext(query)) {                                   // there remains at least one subquery.
-          query.getSubQuery(castEvent.getExecutionBlockId()).waitingIntermediateReport();
+            hasNext(query)) {                                   // there remains at least one stage.
+          query.getStage(castEvent.getExecutionBlockId()).waitingIntermediateReport();
           executeNextBlock(query);
         } else { // if a query is completed due to finished, kill, failure, or error
           query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState()));
@@ -687,12 +687,12 @@ public class Query implements EventHandler<QueryEvent> {
     }
   }
 
-  private static class KillSubQueriesTransition implements SingleArcTransition<Query, QueryEvent> {
+  private static class KillAllStagesTransition 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));
+      synchronized (query.stages) {
+        for (Stage stage : query.stages.values()) {
+          query.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index e7e2bc0..c2e1009 100644
--- 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
@@ -150,7 +150,7 @@ public class QueryMasterManagerService extends CompositeService
       if (queryMasterTask == null) {
         queryMasterTask = queryMaster.getQueryMasterTask(queryId, true);
       }
-      SubQuery sq = queryMasterTask.getQuery().getSubQuery(attemptId.getTaskId().getExecutionBlockId());
+      Stage sq = queryMasterTask.getQuery().getStage(attemptId.getTaskId().getExecutionBlockId());
       Task task = sq.getTask(attemptId.getTaskId());
       TaskAttempt attempt = task.getAttempt(attemptId.getId());
 
@@ -221,7 +221,7 @@ public class QueryMasterManagerService extends CompositeService
     QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(new QueryId(request.getEbId().getQueryId()));
     if (queryMasterTask != null) {
       ExecutionBlockId ebId = new ExecutionBlockId(request.getEbId());
-      queryMasterTask.getQuery().getSubQuery(ebId).receiveExecutionBlockReport(request);
+      queryMasterTask.getQuery().getStage(ebId).receiveExecutionBlockReport(request);
     }
     done.run(TajoWorker.TRUE_PROTO);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index c96b86e..e3d3d79 100644
--- 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
@@ -160,7 +160,7 @@ public class QueryMasterTask extends CompositeService {
       dispatcher = new TajoAsyncDispatcher(queryId.toString());
       addService(dispatcher);
 
-      dispatcher.register(SubQueryEventType.class, new SubQueryEventDispatcher());
+      dispatcher.register(StageEventType.class, new StageEventDispatcher());
       dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
       dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
       dispatcher.register(QueryMasterQueryCompletedEvent.EventType.class, new QueryFinishEventHandler());
@@ -255,7 +255,7 @@ public class QueryMasterTask extends CompositeService {
 
   public void handleTaskRequestEvent(TaskRequestEvent event) {
     ExecutionBlockId id = event.getExecutionBlockId();
-    query.getSubQuery(id).handleTaskRequestEvent(event);
+    query.getStage(id).handleTaskRequestEvent(event);
   }
 
   public void handleTaskFailed(TajoWorkerProtocol.TaskFatalErrorReport report) {
@@ -274,13 +274,13 @@ public class QueryMasterTask extends CompositeService {
     }
   }
 
-  private class SubQueryEventDispatcher implements EventHandler<SubQueryEvent> {
-    public void handle(SubQueryEvent event) {
-      ExecutionBlockId id = event.getSubQueryId();
+  private class StageEventDispatcher implements EventHandler<StageEvent> {
+    public void handle(StageEvent event) {
+      ExecutionBlockId id = event.getStageId();
       if(LOG.isDebugEnabled()) {
-        LOG.debug("SubQueryEventDispatcher:" + id + "," + event.getType());
+        LOG.debug("StageEventDispatcher:" + id + "," + event.getType());
       }
-      query.getSubQuery(id).handle(event);
+      query.getStage(id).handle(event);
     }
   }
 
@@ -291,7 +291,7 @@ public class QueryMasterTask extends CompositeService {
       if(LOG.isDebugEnabled()) {
         LOG.debug("TaskEventDispatcher>" + taskId + "," + event.getType());
       }
-      Task task = query.getSubQuery(taskId.getExecutionBlockId()).
+      Task task = query.getStage(taskId.getExecutionBlockId()).
           getTask(taskId);
       task.handle(event);
     }
@@ -301,8 +301,8 @@ public class QueryMasterTask extends CompositeService {
       implements EventHandler<TaskAttemptEvent> {
     public void handle(TaskAttemptEvent event) {
       TaskAttemptId attemptId = event.getTaskAttemptId();
-      SubQuery subQuery = query.getSubQuery(attemptId.getTaskId().getExecutionBlockId());
-      Task task = subQuery.getTask(attemptId.getTaskId());
+      Stage stage = query.getStage(attemptId.getTaskId().getExecutionBlockId());
+      Task task = stage.getTask(attemptId.getTaskId());
       TaskAttempt attempt = task.getAttempt(attemptId);
       attempt.handle(event);
     }
@@ -311,8 +311,8 @@ public class QueryMasterTask extends CompositeService {
   private class TaskSchedulerDispatcher
       implements EventHandler<TaskSchedulerEvent> {
     public void handle(TaskSchedulerEvent event) {
-      SubQuery subQuery = query.getSubQuery(event.getExecutionBlockId());
-      subQuery.getTaskScheduler().handle(event);
+      Stage stage = query.getStage(event.getExecutionBlockId());
+      stage.getTaskScheduler().handle(event);
     }
   }
 
@@ -627,8 +627,8 @@ public class QueryMasterTask extends CompositeService {
       return dispatcher;
     }
 
-    public SubQuery getSubQuery(ExecutionBlockId id) {
-      return query.getSubQuery(id);
+    public Stage getStage(ExecutionBlockId id) {
+      return query.getStage(id);
     }
 
     public Map<String, TableDesc> getTableDescMap() {


[8/9] tajo git commit: Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/tajo into index_support

Posted by ji...@apache.org.
Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/tajo into index_support


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

Branch: refs/heads/index_support
Commit: 829fcd96377926e77c3161639037ff3e88f62203
Parents: 5893d2c 3c833e2
Author: Jihoon Son <ji...@apache.org>
Authored: Mon Dec 22 17:10:25 2014 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Mon Dec 22 17:10:25 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   27 +
 .../tajo/catalog/AbstractCatalogClient.java     |    2 +-
 .../org/apache/tajo/catalog/CatalogUtil.java    |   24 +-
 .../org/apache/tajo/catalog/DDLBuilder.java     |    6 +-
 .../apache/tajo/catalog/store/HCatalogUtil.java |    3 +-
 .../org/apache/tajo/cli/tools/TajoAdmin.java    |    2 +-
 .../org/apache/tajo/cli/tools/TajoHAAdmin.java  |    2 +-
 .../apache/tajo/client/SessionConnection.java   |    2 +-
 .../org/apache/tajo/client/TajoClientImpl.java  |    5 +-
 .../apache/tajo/client/TajoHAClientUtil.java    |   20 +-
 tajo-client/src/main/proto/ClientProtos.proto   |    4 +-
 .../java/org/apache/tajo/QueryIdFactory.java    |   12 +-
 .../org/apache/tajo/QueryUnitAttemptId.java     |   94 --
 .../main/java/org/apache/tajo/QueryUnitId.java  |   90 --
 .../java/org/apache/tajo/TaskAttemptId.java     |   94 ++
 .../src/main/java/org/apache/tajo/TaskId.java   |   90 ++
 .../java/org/apache/tajo/ha/HAConstants.java    |   27 +
 .../java/org/apache/tajo/ha/HAServiceUtil.java  |  292 ++++
 .../org/apache/tajo/util/HAServiceUtil.java     |  293 ----
 .../java/org/apache/tajo/util/TajoIdUtils.java  |    8 +-
 .../apache/tajo/validation/PathValidator.java   |    2 +-
 tajo-common/src/main/proto/TajoIdProtos.proto   |    6 +-
 .../apache/tajo/validation/TestValidators.java  |   12 +
 .../physical/HashShuffleFileWriteExec.java      |    3 +-
 .../engine/planner/physical/PhysicalExec.java   |    2 +-
 .../engine/planner/physical/SeqScanExec.java    |    2 +-
 .../tajo/engine/query/QueryUnitRequest.java     |   51 -
 .../tajo/engine/query/QueryUnitRequestImpl.java |  328 -----
 .../apache/tajo/engine/query/TaskRequest.java   |   51 +
 .../tajo/engine/query/TaskRequestImpl.java      |  328 +++++
 .../tajo/master/DefaultTaskScheduler.java       |  188 +--
 .../org/apache/tajo/master/GlobalEngine.java    |  789 +---------
 .../apache/tajo/master/LazyTaskScheduler.java   |   70 +-
 .../master/NonForwardQueryResultScanner.java    |    6 +-
 .../apache/tajo/master/TajoContainerProxy.java  |    6 +-
 .../java/org/apache/tajo/master/TajoMaster.java |    2 +-
 .../tajo/master/TajoMasterClientService.java    |    9 +-
 .../tajo/master/TaskSchedulerFactory.java       |   12 +-
 .../event/ContainerAllocatorEventType.java      |    2 +-
 .../tajo/master/event/LocalTaskEvent.java       |    8 +-
 .../tajo/master/event/QueryCompletedEvent.java  |    8 +-
 .../tajo/master/event/QueryEventType.java       |    4 +-
 .../tajo/master/event/QuerySubQueryEvent.java   |   35 -
 .../event/QueryUnitAttemptScheduleEvent.java    |   87 --
 .../tajo/master/event/StageCompletedEvent.java  |   42 +
 .../event/StageContainerAllocationEvent.java    |   38 +
 .../event/StageDiagnosticsUpdateEvent.java      |   34 +
 .../apache/tajo/master/event/StageEvent.java    |   35 +
 .../tajo/master/event/StageEventType.java       |   43 +
 .../tajo/master/event/StageTaskEvent.java       |   43 +
 .../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  |    4 +-
 .../tajo/master/event/TaskAttemptEvent.java     |    8 +-
 .../master/event/TaskAttemptScheduleEvent.java  |    4 +-
 .../event/TaskAttemptStatusUpdateEvent.java     |    4 +-
 .../event/TaskAttemptToSchedulerEvent.java      |   87 ++
 .../tajo/master/event/TaskCompletionEvent.java  |    4 +-
 .../org/apache/tajo/master/event/TaskEvent.java |    8 +-
 .../apache/tajo/master/event/TaskEventType.java |    4 +-
 .../tajo/master/event/TaskFatalErrorEvent.java  |    6 +-
 .../tajo/master/event/TaskRequestEvent.java     |    9 +-
 .../tajo/master/event/TaskTAttemptEvent.java    |   10 +-
 .../apache/tajo/master/exec/DDLExecutor.java    |  434 ++++++
 .../apache/tajo/master/exec/QueryExecutor.java  |  391 +++++
 .../master/exec/prehook/CreateTableHook.java    |   53 +
 .../exec/prehook/DistributedQueryHook.java      |   27 +
 .../prehook/DistributedQueryHookManager.java    |   45 +
 .../master/exec/prehook/InsertIntoHook.java     |   61 +
 .../tajo/master/ha/HAServiceHDFSImpl.java       |  100 +-
 .../apache/tajo/master/querymaster/Query.java   |  164 +--
 .../master/querymaster/QueryJobManager.java     |    5 +-
 .../tajo/master/querymaster/QueryMaster.java    |    2 +-
 .../querymaster/QueryMasterManagerService.java  |   24 +-
 .../master/querymaster/QueryMasterTask.java     |   44 +-
 .../tajo/master/querymaster/QueryUnit.java      |  907 ------------
 .../master/querymaster/QueryUnitAttempt.java    |  443 ------
 .../tajo/master/querymaster/Repartitioner.java  |  186 +--
 .../apache/tajo/master/querymaster/Stage.java   | 1342 +++++++++++++++++
 .../tajo/master/querymaster/StageState.java     |   30 +
 .../tajo/master/querymaster/SubQuery.java       | 1345 ------------------
 .../tajo/master/querymaster/SubQueryState.java  |   30 -
 .../apache/tajo/master/querymaster/Task.java    |  907 ++++++++++++
 .../tajo/master/querymaster/TaskAttempt.java    |  443 ++++++
 .../main/java/org/apache/tajo/util/JSPUtil.java |  120 +-
 .../apache/tajo/util/history/HistoryReader.java |   22 +-
 .../apache/tajo/util/history/HistoryWriter.java |   14 +-
 .../apache/tajo/util/history/QueryHistory.java  |   23 +-
 .../tajo/util/history/QueryUnitHistory.java     |  167 ---
 .../apache/tajo/util/history/StageHistory.java  |  270 ++++
 .../tajo/util/history/SubQueryHistory.java      |  270 ----
 .../apache/tajo/util/history/TaskHistory.java   |  167 +++
 .../tajo/worker/ExecutionBlockContext.java      |   12 +-
 .../java/org/apache/tajo/worker/FetchImpl.java  |   19 +-
 .../apache/tajo/worker/InterDataRetriever.java  |    8 +-
 .../tajo/worker/TajoResourceAllocator.java      |   16 +-
 .../java/org/apache/tajo/worker/TajoWorker.java |    1 +
 .../tajo/worker/TajoWorkerManagerService.java   |    8 +-
 .../main/java/org/apache/tajo/worker/Task.java  |   51 +-
 .../apache/tajo/worker/TaskAttemptContext.java  |   14 +-
 .../org/apache/tajo/worker/TaskHistory.java     |   18 +-
 .../java/org/apache/tajo/worker/TaskRunner.java |   18 +-
 .../apache/tajo/worker/TaskRunnerHistory.java   |   16 +-
 .../apache/tajo/worker/TaskRunnerManager.java   |   10 +-
 .../tajo/worker/WorkerHeartbeatService.java     |    2 +-
 .../retriever/AdvancedDataRetriever.java        |   14 +-
 .../ConnectivityCheckerRuleForTajoWorker.java   |    2 +-
 .../src/main/proto/QueryMasterProtocol.proto    |    2 +-
 .../src/main/proto/TajoWorkerProtocol.proto     |   22 +-
 .../resources/webapps/admin/querydetail.jsp     |   32 +-
 .../main/resources/webapps/admin/querytasks.jsp |  106 +-
 .../main/resources/webapps/admin/queryunit.jsp  |  134 --
 .../src/main/resources/webapps/admin/task.jsp   |  134 ++
 .../resources/webapps/worker/querydetail.jsp    |   30 +-
 .../main/resources/webapps/worker/queryplan.jsp |   52 +-
 .../resources/webapps/worker/querytasks.jsp     |   86 +-
 .../main/resources/webapps/worker/queryunit.jsp |  175 ---
 .../src/main/resources/webapps/worker/task.jsp  |  174 +++
 .../resources/webapps/worker/taskdetail.jsp     |   14 +-
 .../resources/webapps/worker/taskhistory.jsp    |    6 +-
 .../src/main/resources/webapps/worker/tasks.jsp |   14 +-
 .../apache/tajo/LocalTajoTestingUtility.java    |   10 +-
 .../org/apache/tajo/TajoTestingCluster.java     |   11 +-
 .../org/apache/tajo/TestQueryIdFactory.java     |   14 +-
 .../test/java/org/apache/tajo/TestTajoIds.java  |   32 +-
 .../org/apache/tajo/client/TestTajoClient.java  |   20 +-
 .../planner/physical/TestBNLJoinExec.java       |    4 +-
 .../planner/physical/TestBSTIndexExec.java      |    2 +-
 .../planner/physical/TestExternalSortExec.java  |    2 +-
 .../physical/TestFullOuterHashJoinExec.java     |    8 +-
 .../physical/TestFullOuterMergeJoinExec.java    |   12 +-
 .../planner/physical/TestHashAntiJoinExec.java  |    2 +-
 .../planner/physical/TestHashJoinExec.java      |    4 +-
 .../planner/physical/TestHashSemiJoinExec.java  |    2 +-
 .../physical/TestLeftOuterHashJoinExec.java     |   10 +-
 .../physical/TestLeftOuterNLJoinExec.java       |   10 +-
 .../planner/physical/TestMergeJoinExec.java     |    2 +-
 .../engine/planner/physical/TestNLJoinExec.java |    4 +-
 .../planner/physical/TestPhysicalPlanner.java   |   60 +-
 .../physical/TestProgressExternalSortExec.java  |    2 +-
 .../physical/TestRightOuterHashJoinExec.java    |    6 +-
 .../physical/TestRightOuterMergeJoinExec.java   |   12 +-
 .../engine/planner/physical/TestSortExec.java   |    2 +-
 .../tajo/engine/query/TestGroupByQuery.java     |   39 +-
 .../tajo/engine/query/TestTablePartitions.java  |    2 +-
 .../tajo/engine/query/TestUnionQuery.java       |    6 +-
 .../apache/tajo/master/TestRepartitioner.java   |   21 +-
 .../tajo/master/ha/TestHAServiceHDFSImpl.java   |  115 +-
 .../querymaster/TestIntermediateEntry.java      |    2 +-
 .../tajo/master/querymaster/TestKillQuery.java  |    8 +-
 .../querymaster/TestQueryUnitStatusUpdate.java  |  194 ---
 .../querymaster/TestTaskStatusUpdate.java       |  194 +++
 .../java/org/apache/tajo/util/TestJSPUtil.java  |   54 +-
 .../util/history/TestHistoryWriterReader.java   |   59 +-
 .../org/apache/tajo/worker/TestHistory.java     |    8 +-
 .../tajo/worker/TestRangeRetrieverHandler.java  |    4 +-
 .../worker/dataserver/TestHttpDataServer.java   |   12 +-
 .../TestGroupByQuery/testGroupbyWithLimit1.sql  |    1 +
 .../TestGroupByQuery/testGroupbyWithLimit2.sql  |    1 +
 .../TestGroupByQuery/testGroupbyWithLimit3.sql  |    1 +
 .../queries/TestQueryUnitStatusUpdate/case1.sql |    1 -
 .../queries/TestQueryUnitStatusUpdate/case2.sql |    5 -
 .../queries/TestQueryUnitStatusUpdate/case3.sql |   10 -
 .../TestSelectQuery/datetime_table_ddl.sql      |    2 +-
 .../datetime_table_timezoned_ddl.sql            |    2 +-
 .../queries/TestTaskStatusUpdate/case1.sql      |    1 +
 .../queries/TestTaskStatusUpdate/case2.sql      |    5 +
 .../queries/TestTaskStatusUpdate/case3.sql      |   10 +
 .../testGroupbyWithLimit1.result                |    3 +
 .../testGroupbyWithLimit2.result                |    3 +
 .../testGroupbyWithLimit3.result                |    3 +
 tajo-dist/pom.xml                               |    6 +-
 .../configuration/catalog_configuration.rst     |    4 +-
 .../src/main/sphinx/table_management/csv.rst    |    4 +-
 .../sphinx/table_management/table_overview.rst  |    4 +-
 tajo-docs/src/main/sphinx/time_zone.rst         |    6 +-
 .../org/apache/tajo/plan/LogicalPlanner.java    |    7 +
 .../tajo/pullserver/HttpDataServerHandler.java  |    2 +-
 .../tajo/pullserver/PullServerAuxService.java   |   16 +-
 .../tajo/pullserver/TajoPullServerService.java  |   16 +-
 .../retriever/AdvancedDataRetriever.java        |   16 +-
 .../org/apache/tajo/storage/StorageManager.java |   19 +-
 .../org/apache/tajo/storage/StorageUtil.java    |    4 +-
 .../src/main/resources/storage-default.xml      |   22 +
 .../src/test/resources/storage-default.xml      |   22 +
 .../storage/hbase/AbstractHBaseAppender.java    |    6 +-
 .../tajo/storage/hbase/HBasePutAppender.java    |    4 +-
 .../tajo/storage/hbase/HBaseStorageManager.java |    2 +-
 .../tajo/storage/hbase/HFileAppender.java       |    8 +-
 .../java/org/apache/tajo/storage/CSVFile.java   |    4 +-
 .../org/apache/tajo/storage/FileAppender.java   |    6 +-
 .../apache/tajo/storage/FileStorageManager.java |   12 +-
 .../tajo/storage/HashShuffleAppender.java       |   12 +-
 .../storage/HashShuffleAppenderManager.java     |    6 +-
 .../java/org/apache/tajo/storage/RawFile.java   |   24 +-
 .../java/org/apache/tajo/storage/RowFile.java   |    4 +-
 .../apache/tajo/storage/avro/AvroAppender.java  |    4 +-
 .../tajo/storage/parquet/ParquetAppender.java   |    4 +-
 .../org/apache/tajo/storage/rcfile/RCFile.java  |    4 +-
 .../sequencefile/SequenceFileAppender.java      |    4 +-
 .../tajo/storage/text/DelimitedLineReader.java  |   14 +-
 .../tajo/storage/text/DelimitedTextFile.java    |   21 +-
 .../org/apache/tajo/storage/TestStorages.java   |   77 +
 .../src/test/resources/storage-default.xml      |   22 +
 208 files changed, 7475 insertions(+), 6860 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/829fcd96/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/829fcd96/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index ac00c15,6db76ae..a4e49f7
--- 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
@@@ -987,15 -987,9 +987,15 @@@ public class TestPhysicalPlanner 
      FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
          new Path(employee.getPath()), Integer.MAX_VALUE);
      Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateIndex");
 +    Path indexPath = StorageUtil.concatPath(TajoConf.getWarehouseDir(conf), "default/idx_employee");
 +    if (sm.getFileSystem().exists(indexPath)) {
 +      sm.getFileSystem().delete(indexPath, true);
 +    }
 +
      TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
-         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
+         LocalTajoTestingUtility.newTaskAttemptId(masterPlan),
          new FileFragment[] {frags[0]}, workDir);
 +    ctx.setEnforcer(new Enforcer());
      Expr context = analyzer.parse(createIndexStmt[0]);
      LogicalPlan plan = planner.createPlan(defaultContext, context);
      LogicalNode rootNode = optimizer.optimize(plan);

http://git-wip-us.apache.org/repos/asf/tajo/blob/829fcd96/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------


[5/9] tajo git commit: TAJO-1262: Rename the prefix 'SubQuery' to 'Stage'.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
deleted file mode 100644
index d4cc6e7..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ /dev/null
@@ -1,1343 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.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.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.*;
-import org.apache.tajo.catalog.CatalogUtil;
-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.StoreType;
-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.json.CoreGsonHelper;
-import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
-import org.apache.tajo.engine.planner.enforce.Enforcer;
-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.ipc.TajoMasterProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage;
-import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
-import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto;
-import org.apache.tajo.master.*;
-import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
-import org.apache.tajo.master.querymaster.Task.IntermediateEntry;
-import org.apache.tajo.master.container.TajoContainer;
-import org.apache.tajo.master.container.TajoContainerId;
-import org.apache.tajo.storage.FileStorageManager;
-import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.unit.StorageUnit;
-import org.apache.tajo.util.KeyValueSet;
-import org.apache.tajo.util.history.TaskHistory;
-import org.apache.tajo.util.history.SubQueryHistory;
-import org.apache.tajo.worker.FetchImpl;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-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.plan.serder.PlanProto.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 AbstractTaskScheduler taskScheduler;
-  private QueryMasterTask.QueryMasterTaskContext context;
-  private final List<String> diagnostics = new ArrayList<String>();
-  private SubQueryState subQueryState;
-
-  private long startTime;
-  private long finishTime;
-
-  volatile Map<TaskId, Task> tasks = new ConcurrentHashMap<TaskId, Task>();
-  volatile Map<TajoContainerId, TajoContainer> containers = new ConcurrentHashMap<TajoContainerId,
-    TajoContainer>();
-
-  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, new KillTasksTransition())
-          .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), new KillTasksTransition())
-          .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 KILLED state
-          .addTransition(SubQueryState.KILLED, SubQueryState.KILLED,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
-              CONTAINERS_CANCEL_TRANSITION)
-          .addTransition(SubQueryState.KILLED, SubQueryState.KILLED,
-              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
-              DIAGNOSTIC_UPDATE_TRANSITION)
-          .addTransition(SubQueryState.KILLED, SubQueryState.ERROR,
-              SubQueryEventType.SQ_INTERNAL_ERROR,
-              INTERNAL_ERROR_TRANSITION)
-              // Ignore-able transitions
-          .addTransition(SubQueryState.KILLED, SubQueryState.KILLED,
-              EnumSet.of(
-                  SubQueryEventType.SQ_START,
-                  SubQueryEventType.SQ_KILL,
-                  SubQueryEventType.SQ_CONTAINER_ALLOCATED,
-                  SubQueryEventType.SQ_FAILED))
-
-          // 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 ERROR 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,
-                  SubQueryEventType.SQ_SUBQUERY_COMPLETED))
-
-          .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;
-  private List<IntermediateEntry> hashShuffleIntermediateEntries = new ArrayList<IntermediateEntry>();
-  private AtomicInteger completeReportReceived = new AtomicInteger(0);
-  private SubQueryHistory finalSubQueryHistory;
-
-  public SubQuery(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block) {
-    this.context = context;
-    this.masterPlan = masterPlan;
-    this.block = block;
-    this.eventHandler = context.getEventHandler();
-
-    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
-    this.readLock = readWriteLock.readLock();
-    this.writeLock = readWriteLock.writeLock();
-    stateMachine = stateMachineFactory.make(this);
-    subQueryState = stateMachine.getCurrentState();
-  }
-
-  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<Task> tempTasks = null;
-    readLock.lock();
-    try {
-      if (getState() == SubQueryState.NEW) {
-        return 0.0f;
-      } else {
-        tempTasks = new ArrayList<Task>(tasks.values());
-      }
-    } finally {
-      readLock.unlock();
-    }
-
-    float totalProgress = 0.0f;
-    for (Task eachTask : tempTasks) {
-      if (eachTask.getLastAttempt() != null) {
-        totalProgress += eachTask.getLastAttempt().getProgress();
-      }
-    }
-
-    if (totalProgress > 0.0f) {
-      return (float) Math.floor((totalProgress / (float) Math.max(tempTasks.size(), 1)) * 1000.0f) / 1000.0f;
-    } else {
-      return 0.0f;
-    }
-  }
-
-  public int getSucceededObjectCount() {
-    return succeededObjectCount;
-  }
-
-  public int getTotalScheduledObjectsCount() {
-    return totalScheduledObjectsCount;
-  }
-
-  public ExecutionBlock getBlock() {
-    return block;
-  }
-
-  public void addTask(Task task) {
-    tasks.put(task.getId(), task);
-  }
-
-  public SubQueryHistory getSubQueryHistory() {
-    if (finalSubQueryHistory != null) {
-      if (finalSubQueryHistory.getFinishTime() == 0) {
-        finalSubQueryHistory = makeSubQueryHistory();
-        finalSubQueryHistory.setTasks(makeTaskHistories());
-      }
-      return finalSubQueryHistory;
-    } else {
-      return makeSubQueryHistory();
-    }
-  }
-
-  private List<TaskHistory> makeTaskHistories() {
-    List<TaskHistory> taskHistories = new ArrayList<TaskHistory>();
-
-    for(Task eachTask : getTasks()) {
-      taskHistories.add(eachTask.getTaskHistory());
-    }
-
-    return taskHistories;
-  }
-
-  private SubQueryHistory makeSubQueryHistory() {
-    SubQueryHistory subQueryHistory = new SubQueryHistory();
-
-    subQueryHistory.setExecutionBlockId(getId().toString());
-    subQueryHistory.setPlan(PlannerUtil.buildExplainString(block.getPlan()));
-    subQueryHistory.setState(getState().toString());
-    subQueryHistory.setStartTime(startTime);
-    subQueryHistory.setFinishTime(finishTime);
-    subQueryHistory.setSucceededObjectCount(succeededObjectCount);
-    subQueryHistory.setKilledObjectCount(killedObjectCount);
-    subQueryHistory.setFailedObjectCount(failedObjectCount);
-    subQueryHistory.setTotalScheduledObjectsCount(totalScheduledObjectsCount);
-    subQueryHistory.setHostLocalAssigned(getTaskScheduler().getHostLocalAssigned());
-    subQueryHistory.setRackLocalAssigned(getTaskScheduler().getRackLocalAssigned());
-
-    long totalInputBytes = 0;
-    long totalReadBytes = 0;
-    long totalReadRows = 0;
-    long totalWriteBytes = 0;
-    long totalWriteRows = 0;
-    int numShuffles = 0;
-    for(Task eachTask : getTasks()) {
-      numShuffles = eachTask.getShuffleOutpuNum();
-      if (eachTask.getLastAttempt() != null) {
-        TableStats inputStats = eachTask.getLastAttempt().getInputStats();
-        if (inputStats != null) {
-          totalInputBytes += inputStats.getNumBytes();
-          totalReadBytes += inputStats.getReadBytes();
-          totalReadRows += inputStats.getNumRows();
-        }
-        TableStats outputStats = eachTask.getLastAttempt().getResultStats();
-        if (outputStats != null) {
-          totalWriteBytes += outputStats.getNumBytes();
-          totalWriteRows += outputStats.getNumRows();
-        }
-      }
-    }
-
-    subQueryHistory.setTotalInputBytes(totalInputBytes);
-    subQueryHistory.setTotalReadBytes(totalReadBytes);
-    subQueryHistory.setTotalReadRows(totalReadRows);
-    subQueryHistory.setTotalWriteBytes(totalWriteBytes);
-    subQueryHistory.setTotalWriteRows(totalWriteRows);
-    subQueryHistory.setNumShuffles(numShuffles);
-    subQueryHistory.setProgress(getProgress());
-    return subQueryHistory;
-  }
-
-  /**
-   * 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 ExecutionBlockId getId() {
-    return block.getId();
-  }
-  
-  public Task[] getTasks() {
-    return tasks.values().toArray(new Task[tasks.size()]);
-  }
-  
-  public Task getTask(TaskId 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 getSynchronizedState() {
-    readLock.lock();
-    try {
-      return stateMachine.getCurrentState();
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  /* non-blocking call for client API */
-  public SubQueryState getState() {
-    return subQueryState;
-  }
-
-  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 (Task unit : getTasks()) {
-      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()));
-  }
-
-  /**
-   * 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);
-
-    // if store plan (i.e., CREATE or INSERT OVERWRITE)
-    StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
-    if (storeType == null) {
-      // get default or store type
-      storeType = StoreType.CSV;
-    }
-
-    schema = channel.getSchema();
-    meta = CatalogUtil.newTableMeta(storeType, new KeyValueSet());
-    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=" + getSynchronizedState());
-    }
-
-    try {
-      writeLock.lock();
-      SubQueryState oldState = getSynchronizedState();
-      try {
-        getStateMachine().doTransition(event.getType(), event);
-        subQueryState = getSynchronizedState();
-      } catch (InvalidStateTransitonException e) {
-        LOG.error("Can't handle this event at current state"
-            + ", eventType:" + event.getType().name()
-            + ", oldState:" + oldState.name()
-            + ", nextState:" + getSynchronizedState().name()
-            , e);
-        eventHandler.handle(new SubQueryEvent(getId(),
-            SubQueryEventType.SQ_INTERNAL_ERROR));
-      }
-
-      // notify the eventhandler of state change
-      if (LOG.isDebugEnabled()) {
-        if (oldState != getSynchronizedState()) {
-          LOG.debug(getId() + " SubQuery Transitioned from " + oldState + " to "
-              + getSynchronizedState());
-        }
-      }
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void handleTaskRequestEvent(TaskRequestEvent event) {
-    taskScheduler.handleTaskRequestEvent(event);
-  }
-
-  private static class InitAndRequestContainer implements MultipleArcTransition<SubQuery,
-      SubQueryEvent, SubQueryState> {
-
-    @Override
-    public SubQueryState transition(final 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 {
-          // execute pre-processing asyncronously
-          subQuery.getContext().getQueryMasterContext().getEventExecutor()
-              .submit(new Runnable() {
-                        @Override
-                        public void run() {
-                          try {
-                            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.complete();
-                            } else {
-                              if(subQuery.getSynchronizedState() == SubQueryState.INITED) {
-                                subQuery.taskScheduler.start();
-                                allocateContainers(subQuery);
-                              } else {
-                                subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_KILL));
-                              }
-                            }
-                          } catch (Throwable 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));
-                          }
-                        }
-                      }
-              );
-          state = SubQueryState.INITED;
-        }
-      } catch (Throwable 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());
-
-      LogicalNode grpNode = null;
-      if (parent != null) {
-        grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.GROUP_BY);
-        if (grpNode == null) {
-          grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.DISTINCT_GROUP_BY);
-        }
-      }
-
-      // We assume this execution block the first stage of join if two or more tables are included in this block,
-      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 / masterPlan.getContext().getInt(SessionVars.JOIN_PER_SHUFFLE_SIZE));
-
-        if (masterPlan.getContext().containsKey(SessionVars.TEST_MIN_TASK_NUM)) {
-          taskNum = masterPlan.getContext().getInt(SessionVars.TEST_MIN_TASK_NUM);
-          LOG.warn("!!!!! TESTCASE MODE !!!!!");
-        }
-
-        // The shuffle output numbers of join may be inconsistent by execution block order.
-        // Thus, we need to compare the number with DataChannel output numbers.
-        // If the number is right, the number and DataChannel output numbers will be consistent.
-        int outerShuffleOutputNum = 0, innerShuffleOutputNum = 0;
-        for (DataChannel eachChannel : masterPlan.getOutgoingChannels(outer.getId())) {
-          outerShuffleOutputNum = Math.max(outerShuffleOutputNum, eachChannel.getShuffleOutputNum());
-        }
-        for (DataChannel eachChannel : masterPlan.getOutgoingChannels(inner.getId())) {
-          innerShuffleOutputNum = Math.max(innerShuffleOutputNum, eachChannel.getShuffleOutputNum());
-        }
-        if (outerShuffleOutputNum != innerShuffleOutputNum
-            && taskNum != outerShuffleOutputNum
-            && taskNum != innerShuffleOutputNum) {
-          LOG.info(subQuery.getId() + ", Change determined number of join partitions cause difference of outputNum" +
-                  ", originTaskNum=" + taskNum + ", changedTaskNum=" + Math.max(outerShuffleOutputNum, innerShuffleOutputNum) +
-                  ", outerShuffleOutptNum=" + outerShuffleOutputNum +
-                  ", innerShuffleOutputNum=" + innerShuffleOutputNum);
-          taskNum = Math.max(outerShuffleOutputNum, innerShuffleOutputNum);
-        }
-
-        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) {
-        boolean hasGroupColumns = true;
-        if (grpNode.getType() == NodeType.GROUP_BY) {
-          hasGroupColumns = ((GroupbyNode)grpNode).getGroupingColumns().length > 0;
-        } else if (grpNode.getType() == NodeType.DISTINCT_GROUP_BY) {
-          // Find current distinct stage node.
-          DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(subQuery.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
-          if (distinctNode == null) {
-            LOG.warn(subQuery.getId() + ", Can't find current DistinctGroupbyNode");
-            distinctNode = (DistinctGroupbyNode)grpNode;
-          }
-          hasGroupColumns = distinctNode.getGroupingColumns().length > 0;
-
-          Enforcer enforcer = subQuery.getBlock().getEnforcer();
-          if (enforcer == null) {
-            LOG.warn(subQuery.getId() + ", DistinctGroupbyNode's enforcer is null.");
-          }
-          EnforceProperty property = PhysicalPlannerImpl.getAlgorithmEnforceProperty(enforcer, distinctNode);
-          if (property != null) {
-            if (property.getDistinct().getIsMultipleAggregation()) {
-              MultipleAggregationStage stage = property.getDistinct().getMultipleAggregationStage();
-              if (stage != MultipleAggregationStage.THRID_STAGE) {
-                hasGroupColumns = true;
-              }
-            }
-          }
-        }
-        if (!hasGroupColumns) {
-          LOG.info(subQuery.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
-          return 1;
-        } else {
-          long volume = getInputVolume(subQuery.masterPlan, subQuery.context, subQuery.block);
-
-          int volumeByMB = (int) Math.ceil((double) volume / StorageUnit.MB);
-          LOG.info(subQuery.getId() + ", Table's volume is approximately " + volumeByMB + " MB");
-          // determine the number of task
-          int taskNum = (int) Math.ceil((double) volumeByMB /
-              masterPlan.getContext().getInt(SessionVars.GROUPBY_PER_SHUFFLE_SIZE));
-          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(subQuery.getId() + ", 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(subQuery.getId() + ", 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.getSynchronizedState() != 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<Fragment> 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) {
-        // After calling this method, partition paths are removed from the physical plan.
-        FileStorageManager storageManager =
-            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
-        fragments = Repartitioner.getFragmentsFromPartitionedTable(storageManager, scan, table);
-      } else {
-        StorageManager storageManager =
-            StorageManager.getStorageManager(subQuery.getContext().getConf(), meta.getStoreType());
-        fragments = storageManager.getSplits(scan.getCanonicalName(), table, scan);
-      }
-
-      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.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, Fragment fragment) {
-    subQuery.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
-        subQuery.getId(), fragment));
-  }
-
-
-  public static void scheduleFragments(SubQuery subQuery, Collection<Fragment> fragments) {
-    for (Fragment eachFragment : fragments) {
-      scheduleFragment(subQuery, eachFragment);
-    }
-  }
-
-  public static void scheduleFragments(SubQuery subQuery, Collection<Fragment> leftFragments,
-                                       Collection<Fragment> broadcastFragments) {
-    for (Fragment eachLeafFragment : leftFragments) {
-      scheduleFragment(subQuery, eachLeafFragment, broadcastFragments);
-    }
-  }
-
-  public static void scheduleFragment(SubQuery subQuery,
-                                      Fragment leftFragment, Collection<Fragment> rightFragments) {
-    subQuery.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
-        subQuery.getId(), leftFragment, rightFragments));
-  }
-
-  public static void scheduleFetches(SubQuery subQuery, Map<String, List<FetchImpl>> fetches) {
-    subQuery.taskScheduler.handle(new FetchScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
-        subQuery.getId(), fetches));
-  }
-
-  public static Task newEmptyTask(TaskSchedulerContext schedulerContext,
-                                  TaskAttemptScheduleContext taskContext,
-                                  SubQuery subQuery, int taskId) {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    Task unit = new Task(schedulerContext.getMasterContext().getConf(),
-        taskContext,
-        QueryIdFactory.newTaskId(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 (TajoContainer container : allocationEvent.getAllocatedContainer()) {
-          TajoContainerId 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 LaunchTaskRunnersEvent(subQuery.getId(), allocationEvent.getAllocatedContainer(),
-                subQuery.getContext().getQueryContext(),
-                CoreGsonHelper.toJson(subQuery.getBlock().getPlan(), LogicalNode.class))
-        );
-
-        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;
-      Task task = subQuery.getTask(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) {
-          subQuery.succeededObjectCount++;
-        } else if (task.getState() == TaskState.KILLED) {
-          subQuery.killedObjectCount++;
-        } else if (task.getState() == TaskState.FAILED) {
-          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) {
-      if(subQuery.getTaskScheduler() != null){
-        subQuery.getTaskScheduler().stop();
-      }
-
-      for (Task task : subQuery.getTasks()) {
-        subQuery.eventHandler.handle(new TaskEvent(task.getId(), TaskEventType.T_KILL));
-      }
-    }
-  }
-
-  private void cleanup() {
-    stopScheduler();
-    releaseContainers();
-
-    if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) {
-      List<ExecutionBlock> childs = getMasterPlan().getChilds(getId());
-      List<TajoIdProtos.ExecutionBlockIdProto> ebIds = Lists.newArrayList();
-
-      for (ExecutionBlock executionBlock : childs) {
-        ebIds.add(executionBlock.getId().getProto());
-      }
-
-      getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds);
-    }
-
-    this.finalSubQueryHistory = makeSubQueryHistory();
-    this.finalSubQueryHistory.setTasks(makeTaskHistories());
-  }
-
-  public List<IntermediateEntry> getHashShuffleIntermediateEntries() {
-    return hashShuffleIntermediateEntries;
-  }
-
-  protected void waitingIntermediateReport() {
-    LOG.info(getId() + ", waiting IntermediateReport: expectedTaskNum=" + completeReportReceived.get());
-    synchronized(completeReportReceived) {
-      long startTime = System.currentTimeMillis();
-      while (true) {
-        if (completeReportReceived.get() >= tasks.size()) {
-          LOG.info(getId() + ", completed waiting IntermediateReport");
-          return;
-        } else {
-          try {
-            completeReportReceived.wait(10 * 1000);
-          } catch (InterruptedException e) {
-          }
-          long elapsedTime = System.currentTimeMillis() - startTime;
-          if (elapsedTime >= 120 * 1000) {
-            LOG.error(getId() + ", Timeout while receiving intermediate reports: " + elapsedTime + " ms");
-            abort(SubQueryState.FAILED);
-            return;
-          }
-        }
-      }
-    }
-  }
-
-  public void receiveExecutionBlockReport(TajoWorkerProtocol.ExecutionBlockReport report) {
-    LOG.info(getId() + ", receiveExecutionBlockReport:" +  report.getSucceededTasks());
-    if (!report.getReportSuccess()) {
-      LOG.error(getId() + ", ExecutionBlock final report fail cause:" + report.getReportErrorMessage());
-      abort(SubQueryState.FAILED);
-      return;
-    }
-    if (report.getIntermediateEntriesCount() > 0) {
-      synchronized (hashShuffleIntermediateEntries) {
-        for (IntermediateEntryProto eachInterm: report.getIntermediateEntriesList()) {
-          hashShuffleIntermediateEntries.add(new IntermediateEntry(eachInterm));
-        }
-      }
-    }
-    synchronized(completeReportReceived) {
-      completeReportReceived.addAndGet(report.getSucceededTasks());
-      completeReportReceived.notifyAll();
-    }
-  }
-
-  private static class SubQueryCompleteTransition
-      implements MultipleArcTransition<SubQuery, SubQueryEvent, SubQueryState> {
-
-    @Override
-    public SubQueryState transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
-      // TODO - Commit subQuery
-      // 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.getSynchronizedState() + " State");
-            subQuery.abort(SubQueryState.ERROR);
-            return SubQueryState.ERROR;
-          }
-        } else {
-          subQuery.complete();
-          return SubQueryState.SUCCEEDED;
-        }
-      } catch (Throwable t) {
-        LOG.error(t.getMessage(), 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/3c833e2a/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
deleted file mode 100644
index effcfde..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.querymaster;
-
-public enum SubQueryState {
-  NEW,
-  INITED,
-  RUNNING,
-  SUCCEEDED,
-  FAILED,
-  KILL_WAIT,
-  KILLED,
-  ERROR
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java
index 7de3933..5475791 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java
@@ -580,7 +580,7 @@ public class Task implements EventHandler<TaskEvent> {
 
     @Override
     public void transition(Task task, TaskEvent taskEvent) {
-      task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.KILLED));
+      task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.KILLED));
     }
   }
 
@@ -598,7 +598,7 @@ public class Task implements EventHandler<TaskEvent> {
     @Override
     public void transition(Task task, TaskEvent event) {
       task.finishTask();
-      task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.KILLED));
+      task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.KILLED));
     }
   }
 
@@ -617,7 +617,7 @@ public class Task implements EventHandler<TaskEvent> {
       task.succeededPullServerPort = attempt.getWorkerConnectionInfo().getPullServerPort();
 
       task.finishTask();
-      task.eventHandler.handle(new SubQueryTaskEvent(event.getTaskId(), TaskState.SUCCEEDED));
+      task.eventHandler.handle(new StageTaskEvent(event.getTaskId(), TaskState.SUCCEEDED));
     }
   }
 
@@ -644,7 +644,7 @@ public class Task implements EventHandler<TaskEvent> {
       task.finishedAttempts++;
 
       task.finishTask();
-      task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.FAILED));
+      task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.FAILED));
     }
   }
 
@@ -669,7 +669,7 @@ public class Task implements EventHandler<TaskEvent> {
         }
       } else {
         task.finishTask();
-        task.eventHandler.handle(new SubQueryTaskEvent(task.getId(), TaskState.FAILED));
+        task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.FAILED));
         return TaskState.FAILED;
       }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java
index 63c6dbb..0f161ff 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java
@@ -420,11 +420,11 @@ public class TaskAttempt implements EventHandler<TaskAttemptEvent> {
             + ", nextState:" + getState().name()
             , e);
         eventHandler.handle(
-            new SubQueryDiagnosticsUpdateEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
+            new StageDiagnosticsUpdateEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
                 "Can't handle this event at current state of " + event.getTaskAttemptId() + ")"));
         eventHandler.handle(
-            new SubQueryEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
-                SubQueryEventType.SQ_INTERNAL_ERROR));
+            new StageEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
+                StageEventType.SQ_INTERNAL_ERROR));
       }
 
       //notify the eventhandler of state change

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 0573197..6050617 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
@@ -28,9 +28,9 @@ import org.apache.tajo.master.ha.HAService;
 import org.apache.tajo.master.querymaster.QueryInProgress;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.master.querymaster.Task;
-import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.Stage;
 import org.apache.tajo.util.history.TaskHistory;
-import org.apache.tajo.util.history.SubQueryHistory;
+import org.apache.tajo.util.history.StageHistory;
 import org.apache.tajo.worker.TaskRunnerHistory;
 import org.apache.tajo.worker.TaskRunner;
 
@@ -144,50 +144,50 @@ public class JSPUtil {
     return queryProgressList;
   }
 
-  public static List<SubQuery> sortSubQuery(Collection<SubQuery> subQueries) {
-    List<SubQuery> subQueryList = new ArrayList<SubQuery>(subQueries);
-    Collections.sort(subQueryList, new Comparator<SubQuery>() {
+  public static List<Stage> sortStages(Collection<Stage> stages) {
+    List<Stage> stageList = new ArrayList<Stage>(stages);
+    Collections.sort(stageList, new Comparator<Stage>() {
       @Override
-      public int compare(SubQuery subQuery1, SubQuery subQuery2) {
-        long q1StartTime = subQuery1.getStartTime();
-        long q2StartTime = subQuery2.getStartTime();
+      public int compare(Stage stage1, Stage stage2) {
+        long q1StartTime = stage1.getStartTime();
+        long q2StartTime = stage2.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());
+          return stage1.getId().toString().compareTo(stage2.getId().toString());
         } else {
           return result;
         }
       }
     });
 
-    return subQueryList;
+    return stageList;
   }
 
-  public static List<SubQueryHistory> sortSubQueryHistory(Collection<SubQueryHistory> subQueries) {
-    List<SubQueryHistory> subQueryList = new ArrayList<SubQueryHistory>(subQueries);
-    Collections.sort(subQueryList, new Comparator<SubQueryHistory>() {
+  public static List<StageHistory> sortStageHistories(Collection<StageHistory> stages) {
+    List<StageHistory> stageList = new ArrayList<StageHistory>(stages);
+    Collections.sort(stageList, new Comparator<StageHistory>() {
       @Override
-      public int compare(SubQueryHistory subQuery1, SubQueryHistory subQuery2) {
-        long q1StartTime = subQuery1.getStartTime();
-        long q2StartTime = subQuery2.getStartTime();
+      public int compare(StageHistory stage1, StageHistory stage2) {
+        long q1StartTime = stage1.getStartTime();
+        long q2StartTime = stage2.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.getExecutionBlockId().compareTo(subQuery2.getExecutionBlockId());
+          return stage1.getExecutionBlockId().compareTo(stage2.getExecutionBlockId());
         } else {
           return result;
         }
       }
     });
 
-    return subQueryList;
+    return stageList;
   }
 
   public static String getMasterActiveLabel(MasterContext context) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
index 9fb427f..932f584 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
@@ -210,7 +210,7 @@ public class HistoryReader {
 
       in.readFully(buf, 0, buf.length);
 
-      return SubQueryHistory.fromJsonTasks(new String(buf));
+      return StageHistory.fromJsonTasks(new String(buf));
     } finally {
       if (in != null) {
         in.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
index 7e30f9c..5934885 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
@@ -40,7 +40,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
  *
  *   <tajo.history.query.dir>/<yyyyMMdd>/query-list/query-list-<HHmmss>.hist (TajoMaster's query list, hourly rolling)
  *                                      /query-detail/<QUERY_ID>/query.hist    (QueryMaster's query detail)
- *                                                               /<EB_ID>.hist  (QueryMaster's subquery detail)
+ *                                                               /<EB_ID>.hist  (QueryMaster's stage detail)
  *   <tajo.history.task.dir>/<yyyyMMdd>/tasks/<WORKER_HOST>_<WORKER_PORT>/<WORKER_HOST>_<WORKER_PORT>_<HH>_<seq>.hist
  * History files are kept for "tajo.history.expiry-time-day" (default value is 7 days)
  */
@@ -267,7 +267,7 @@ public class HistoryWriter extends AbstractService {
       // QueryMaster's query detail history (json format)
       // <tajo.query-history.path>/<yyyyMMdd>/query-detail/<QUERY_ID>/query.hist
 
-      // QueryMaster's subquery detail history(proto binary format)
+      // QueryMaster's stage detail history(proto binary format)
       // <tajo.query-history.path>/<yyyyMMdd>/query-detail/<QUERY_ID>/<EB_ID>.hist
 
       Path queryHistoryFile = getQueryHistoryFilePath(historyParentPath, queryHistory.getQueryId());
@@ -295,13 +295,13 @@ public class HistoryWriter extends AbstractService {
         }
       }
 
-      if (queryHistory.getSubQueryHistories() != null) {
-        for (SubQueryHistory subQueryHistory : queryHistory.getSubQueryHistories()) {
-          Path path = new Path(queryHistoryFile.getParent(), subQueryHistory.getExecutionBlockId() + HISTORY_FILE_POSTFIX);
+      if (queryHistory.getStageHistories() != null) {
+        for (StageHistory stageHistory : queryHistory.getStageHistories()) {
+          Path path = new Path(queryHistoryFile.getParent(), stageHistory.getExecutionBlockId() + HISTORY_FILE_POSTFIX);
           out = null;
           try {
             out = fs.create(path);
-            out.write(subQueryHistory.toTasksJson().getBytes());
+            out.write(stageHistory.toTasksJson().getBytes());
             LOG.info("Saving query unit: " + path);
           } finally {
             if (out != null) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
index 7a81b4b..fdc45a1 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
@@ -20,8 +20,9 @@ package org.apache.tajo.util.history;
 
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.QueryHistoryProto;
-import org.apache.tajo.ipc.ClientProtos.SubQueryHistoryProto;
+import org.apache.tajo.ipc.ClientProtos.StageHistoryProto;
 import org.apache.tajo.json.GsonObject;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto;
 
@@ -42,7 +43,7 @@ public class QueryHistory implements GsonObject, History {
   @Expose
   private String distributedPlan;
   @Expose
-  private List<SubQueryHistory> subQueryHistories;
+  private List<StageHistory> stageHistories;
 
   public String getQueryId() {
     return queryId;
@@ -56,8 +57,8 @@ public class QueryHistory implements GsonObject, History {
     this.queryMaster = queryMaster;
   }
 
-  public void setSubQueryHistories(List<SubQueryHistory> subQueryHistories) {
-    this.subQueryHistories = subQueryHistories;
+  public void setStageHistories(List<StageHistory> stageHistories) {
+    this.stageHistories = stageHistories;
   }
 
   public String getQueryMaster() {
@@ -72,8 +73,8 @@ public class QueryHistory implements GsonObject, History {
     this.httpPort = httpPort;
   }
 
-  public List<SubQueryHistory> getSubQueryHistories() {
-    return subQueryHistories;
+  public List<StageHistory> getStageHistories() {
+    return stageHistories;
   }
 
   public List<String[]> getSessionVariables() {
@@ -138,13 +139,13 @@ public class QueryHistory implements GsonObject, History {
     builder.addAllSessionVariables(sessionProtos);
 
 
-    List<SubQueryHistoryProto> subQueryHistoryProtos = new ArrayList<SubQueryHistoryProto>();
-    if (subQueryHistories != null) {
-      for (SubQueryHistory eachSubQuery: subQueryHistories) {
-        subQueryHistoryProtos.add((eachSubQuery.getProto()));
+    List<StageHistoryProto> stageHistoryProtos = new ArrayList<ClientProtos.StageHistoryProto>();
+    if (stageHistories != null) {
+      for (StageHistory eachStage: stageHistories) {
+        stageHistoryProtos.add((eachStage.getProto()));
       }
     }
-    builder.addAllSubQueryHistories(subQueryHistoryProtos);
+    builder.addAllStageHistories(stageHistoryProtos);
 
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/util/history/StageHistory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/StageHistory.java b/tajo-core/src/main/java/org/apache/tajo/util/history/StageHistory.java
new file mode 100644
index 0000000..e760f86
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/StageHistory.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.util.history;
+
+import com.google.gson.annotations.Expose;
+import com.google.gson.reflect.TypeToken;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.ipc.ClientProtos.StageHistoryProto;
+import org.apache.tajo.json.GsonObject;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class StageHistory implements GsonObject {
+  @Expose
+  private String executionBlockId;
+  @Expose
+  private String state;
+  @Expose
+  private long startTime;
+  @Expose
+  private long finishTime;
+  @Expose
+  private int succeededObjectCount;
+  @Expose
+  private int failedObjectCount;
+  @Expose
+  private int killedObjectCount;
+  @Expose
+  private int totalScheduledObjectsCount;
+
+  @Expose
+  private long totalInputBytes;
+  @Expose
+  private long totalReadBytes;
+  @Expose
+  private long totalReadRows;
+  @Expose
+  private long totalWriteBytes;
+  @Expose
+  private long totalWriteRows;
+  @Expose
+  private int numShuffles;
+  @Expose
+  private float progress;
+
+  @Expose
+  private String plan;
+  @Expose
+  private int hostLocalAssigned;
+  @Expose
+  private int rackLocalAssigned;
+
+  private List<TaskHistory> tasks;
+
+  public String getExecutionBlockId() {
+    return executionBlockId;
+  }
+
+  public void setExecutionBlockId(String executionBlockId) {
+    this.executionBlockId = executionBlockId;
+  }
+
+  public String getState() {
+    return state;
+  }
+
+  public void setState(String state) {
+    this.state = state;
+  }
+
+  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 int getSucceededObjectCount() {
+    return succeededObjectCount;
+  }
+
+  public void setSucceededObjectCount(int succeededObjectCount) {
+    this.succeededObjectCount = succeededObjectCount;
+  }
+
+  public int getTotalScheduledObjectsCount() {
+    return totalScheduledObjectsCount;
+  }
+
+  public void setTotalScheduledObjectsCount(int totalScheduledObjectsCount) {
+    this.totalScheduledObjectsCount = totalScheduledObjectsCount;
+  }
+
+  public long getTotalInputBytes() {
+    return totalInputBytes;
+  }
+
+  public void setTotalInputBytes(long totalInputBytes) {
+    this.totalInputBytes = totalInputBytes;
+  }
+
+  public long getTotalReadBytes() {
+    return totalReadBytes;
+  }
+
+  public void setTotalReadBytes(long totalReadBytes) {
+    this.totalReadBytes = totalReadBytes;
+  }
+
+  public long getTotalReadRows() {
+    return totalReadRows;
+  }
+
+  public void setTotalReadRows(long totalReadRows) {
+    this.totalReadRows = totalReadRows;
+  }
+
+  public long getTotalWriteBytes() {
+    return totalWriteBytes;
+  }
+
+  public void setTotalWriteBytes(long totalWriteBytes) {
+    this.totalWriteBytes = totalWriteBytes;
+  }
+
+  public long getTotalWriteRows() {
+    return totalWriteRows;
+  }
+
+  public void setTotalWriteRows(long totalWriteRows) {
+    this.totalWriteRows = totalWriteRows;
+  }
+
+  public int getNumShuffles() {
+    return numShuffles;
+  }
+
+  public void setNumShuffles(int numShuffles) {
+    this.numShuffles = numShuffles;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  public void setProgress(float progress) {
+    this.progress = progress;
+  }
+
+  public String getPlan() {
+    return plan;
+  }
+
+  public void setPlan(String plan) {
+    this.plan = plan;
+  }
+
+  public int getHostLocalAssigned() {
+    return hostLocalAssigned;
+  }
+
+  public void setHostLocalAssigned(int hostLocalAssigned) {
+    this.hostLocalAssigned = hostLocalAssigned;
+  }
+
+  public int getRackLocalAssigned() {
+    return rackLocalAssigned;
+  }
+
+  public void setRackLocalAssigned(int rackLocalAssigned) {
+    this.rackLocalAssigned = rackLocalAssigned;
+  }
+
+  public int getFailedObjectCount() {
+    return failedObjectCount;
+  }
+
+  public void setFailedObjectCount(int failedObjectCount) {
+    this.failedObjectCount = failedObjectCount;
+  }
+
+  public int getKilledObjectCount() {
+    return killedObjectCount;
+  }
+
+  public void setKilledObjectCount(int killedObjectCount) {
+    this.killedObjectCount = killedObjectCount;
+  }
+
+  public List<TaskHistory> getTasks() {
+    return tasks;
+  }
+
+  public void setTasks(List<TaskHistory> tasks) {
+    this.tasks = tasks;
+  }
+
+  @Override
+  public String toJson() {
+    return CoreGsonHelper.toJson(this, StageHistory.class);
+  }
+
+  public String toTasksJson() {
+    if (tasks == null) {
+      return "";
+    }
+    return CoreGsonHelper.getInstance().toJson(tasks, new TypeToken<List<TaskHistory>>() {
+    }.getType());
+  }
+
+  public static List<TaskHistory> fromJsonTasks(String json) {
+    if (json == null || json.trim().isEmpty()) {
+      return new ArrayList<TaskHistory>();
+    }
+    return CoreGsonHelper.getInstance().fromJson(json, new TypeToken<List<TaskHistory>>() {
+    }.getType());
+  }
+
+  public StageHistoryProto getProto() {
+    StageHistoryProto.Builder builder = StageHistoryProto.newBuilder();
+    builder.setExecutionBlockId(executionBlockId)
+      .setState(state)
+      .setStartTime(startTime)
+      .setFinishTime(finishTime)
+      .setSucceededObjectCount(succeededObjectCount)
+      .setFailedObjectCount(failedObjectCount)
+      .setKilledObjectCount(killedObjectCount)
+      .setTotalScheduledObjectsCount(totalScheduledObjectsCount)
+
+      .setTotalInputBytes(totalInputBytes)
+      .setTotalReadBytes(totalReadBytes)
+      .setTotalReadRows(totalReadRows)
+      .setTotalWriteBytes(totalWriteBytes)
+      .setTotalWriteRows(totalWriteRows)
+      .setNumShuffles(numShuffles)
+      .setProgress(progress)
+
+      .setPlan(plan)
+      .setHostLocalAssigned(hostLocalAssigned)
+      .setRackLocalAssigned(rackLocalAssigned);
+
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java b/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
deleted file mode 100644
index 0afdf5a..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.util.history;
-
-import com.google.gson.annotations.Expose;
-import com.google.gson.reflect.TypeToken;
-import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.ipc.ClientProtos.SubQueryHistoryProto;
-import org.apache.tajo.json.GsonObject;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class SubQueryHistory implements GsonObject {
-  @Expose
-  private String executionBlockId;
-  @Expose
-  private String state;
-  @Expose
-  private long startTime;
-  @Expose
-  private long finishTime;
-  @Expose
-  private int succeededObjectCount;
-  @Expose
-  private int failedObjectCount;
-  @Expose
-  private int killedObjectCount;
-  @Expose
-  private int totalScheduledObjectsCount;
-
-  @Expose
-  private long totalInputBytes;
-  @Expose
-  private long totalReadBytes;
-  @Expose
-  private long totalReadRows;
-  @Expose
-  private long totalWriteBytes;
-  @Expose
-  private long totalWriteRows;
-  @Expose
-  private int numShuffles;
-  @Expose
-  private float progress;
-
-  @Expose
-  private String plan;
-  @Expose
-  private int hostLocalAssigned;
-  @Expose
-  private int rackLocalAssigned;
-
-  private List<TaskHistory> tasks;
-
-  public String getExecutionBlockId() {
-    return executionBlockId;
-  }
-
-  public void setExecutionBlockId(String executionBlockId) {
-    this.executionBlockId = executionBlockId;
-  }
-
-  public String getState() {
-    return state;
-  }
-
-  public void setState(String state) {
-    this.state = state;
-  }
-
-  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 int getSucceededObjectCount() {
-    return succeededObjectCount;
-  }
-
-  public void setSucceededObjectCount(int succeededObjectCount) {
-    this.succeededObjectCount = succeededObjectCount;
-  }
-
-  public int getTotalScheduledObjectsCount() {
-    return totalScheduledObjectsCount;
-  }
-
-  public void setTotalScheduledObjectsCount(int totalScheduledObjectsCount) {
-    this.totalScheduledObjectsCount = totalScheduledObjectsCount;
-  }
-
-  public long getTotalInputBytes() {
-    return totalInputBytes;
-  }
-
-  public void setTotalInputBytes(long totalInputBytes) {
-    this.totalInputBytes = totalInputBytes;
-  }
-
-  public long getTotalReadBytes() {
-    return totalReadBytes;
-  }
-
-  public void setTotalReadBytes(long totalReadBytes) {
-    this.totalReadBytes = totalReadBytes;
-  }
-
-  public long getTotalReadRows() {
-    return totalReadRows;
-  }
-
-  public void setTotalReadRows(long totalReadRows) {
-    this.totalReadRows = totalReadRows;
-  }
-
-  public long getTotalWriteBytes() {
-    return totalWriteBytes;
-  }
-
-  public void setTotalWriteBytes(long totalWriteBytes) {
-    this.totalWriteBytes = totalWriteBytes;
-  }
-
-  public long getTotalWriteRows() {
-    return totalWriteRows;
-  }
-
-  public void setTotalWriteRows(long totalWriteRows) {
-    this.totalWriteRows = totalWriteRows;
-  }
-
-  public int getNumShuffles() {
-    return numShuffles;
-  }
-
-  public void setNumShuffles(int numShuffles) {
-    this.numShuffles = numShuffles;
-  }
-
-  public float getProgress() {
-    return progress;
-  }
-
-  public void setProgress(float progress) {
-    this.progress = progress;
-  }
-
-  public String getPlan() {
-    return plan;
-  }
-
-  public void setPlan(String plan) {
-    this.plan = plan;
-  }
-
-  public int getHostLocalAssigned() {
-    return hostLocalAssigned;
-  }
-
-  public void setHostLocalAssigned(int hostLocalAssigned) {
-    this.hostLocalAssigned = hostLocalAssigned;
-  }
-
-  public int getRackLocalAssigned() {
-    return rackLocalAssigned;
-  }
-
-  public void setRackLocalAssigned(int rackLocalAssigned) {
-    this.rackLocalAssigned = rackLocalAssigned;
-  }
-
-  public int getFailedObjectCount() {
-    return failedObjectCount;
-  }
-
-  public void setFailedObjectCount(int failedObjectCount) {
-    this.failedObjectCount = failedObjectCount;
-  }
-
-  public int getKilledObjectCount() {
-    return killedObjectCount;
-  }
-
-  public void setKilledObjectCount(int killedObjectCount) {
-    this.killedObjectCount = killedObjectCount;
-  }
-
-  public List<TaskHistory> getTasks() {
-    return tasks;
-  }
-
-  public void setTasks(List<TaskHistory> tasks) {
-    this.tasks = tasks;
-  }
-
-  @Override
-  public String toJson() {
-    return CoreGsonHelper.toJson(this, SubQueryHistory.class);
-  }
-
-  public String toTasksJson() {
-    if (tasks == null) {
-      return "";
-    }
-    return CoreGsonHelper.getInstance().toJson(tasks, new TypeToken<List<TaskHistory>>() {
-    }.getType());
-  }
-
-  public static List<TaskHistory> fromJsonTasks(String json) {
-    if (json == null || json.trim().isEmpty()) {
-      return new ArrayList<TaskHistory>();
-    }
-    return CoreGsonHelper.getInstance().fromJson(json, new TypeToken<List<TaskHistory>>() {
-    }.getType());
-  }
-
-  public SubQueryHistoryProto getProto() {
-    SubQueryHistoryProto.Builder builder = SubQueryHistoryProto.newBuilder();
-    builder.setExecutionBlockId(executionBlockId)
-      .setState(state)
-      .setStartTime(startTime)
-      .setFinishTime(finishTime)
-      .setSucceededObjectCount(succeededObjectCount)
-      .setFailedObjectCount(failedObjectCount)
-      .setKilledObjectCount(killedObjectCount)
-      .setTotalScheduledObjectsCount(totalScheduledObjectsCount)
-
-      .setTotalInputBytes(totalInputBytes)
-      .setTotalReadBytes(totalReadBytes)
-      .setTotalReadRows(totalReadRows)
-      .setTotalWriteBytes(totalWriteBytes)
-      .setTotalWriteRows(totalWriteRows)
-      .setNumShuffles(numShuffles)
-      .setProgress(progress)
-
-      .setPlan(plan)
-      .setHostLocalAssigned(hostLocalAssigned)
-      .setRackLocalAssigned(rackLocalAssigned);
-
-    return builder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index f055733..8944eae 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
@@ -37,10 +37,10 @@ import org.apache.tajo.master.container.TajoContainer;
 import org.apache.tajo.master.container.TajoContainerId;
 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.event.StageContainerAllocationEvent;
 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.querymaster.Stage;
+import org.apache.tajo.master.querymaster.StageState;
 import org.apache.tajo.master.rm.*;
 import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.rpc.NettyClientBase;
@@ -352,8 +352,8 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
           containers.add(container);
         }
 
-        SubQueryState state = queryTaskContext.getSubQuery(executionBlockId).getSynchronizedState();
-        if (!SubQuery.isRunningState(state)) {
+        StageState state = queryTaskContext.getStage(executionBlockId).getSynchronizedState();
+        if (!Stage.isRunningState(state)) {
           try {
             List<TajoContainerId> containerIds = new ArrayList<TajoContainerId>();
             for(TajoContainer eachContainer: containers) {
@@ -368,9 +368,9 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
 
         if (allocatedResources.size() > 0) {
           if(LOG.isDebugEnabled()) {
-            LOG.debug("SubQueryContainerAllocationEvent fire:" + executionBlockId);
+            LOG.debug("StageContainerAllocationEvent fire:" + executionBlockId);
           }
-          queryTaskContext.getEventHandler().handle(new SubQueryContainerAllocationEvent(executionBlockId, containers));
+          queryTaskContext.getEventHandler().handle(new StageContainerAllocationEvent(executionBlockId, containers));
         }
         numAllocatedContainers += allocatedResources.size();
 


[4/9] tajo git commit: TAJO-1262: Rename the prefix 'SubQuery' to 'Stage'.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 0920619..70a3202 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
@@ -162,7 +162,7 @@ public class Task {
     
     context.setState(TaskAttemptState.TA_PENDING);
     LOG.info("==================================");
-    LOG.info("* Subquery " + request.getId() + " is initialized");
+    LOG.info("* Stage " + request.getId() + " is initialized");
     LOG.info("* InterQuery: " + interQuery
         + (interQuery ? ", Use " + this.shuffleType + " shuffle":"") +
         ", Fragments (num: " + request.getFragments().size() + ")" +
@@ -734,24 +734,24 @@ public class Task {
     final List<String> types = params.get("type");
     final List<String> qids = params.get("qid");
     final List<String> taskIdList = params.get("ta");
-    final List<String> subQueryIds = params.get("sid");
+    final List<String> stageIds = params.get("sid");
     final List<String> partIds = params.get("p");
     final List<String> offsetList = params.get("offset");
     final List<String> lengthList = params.get("length");
 
-    if (types == null || subQueryIds == null || qids == null || partIds == null) {
-      LOG.error("Invalid URI - Required queryId, type, subquery Id, and part id");
+    if (types == null || stageIds == null || qids == null || partIds == null) {
+      LOG.error("Invalid URI - Required queryId, type, stage Id, and part id");
       return null;
     }
 
-    if (qids.size() != 1 && types.size() != 1 || subQueryIds.size() != 1) {
-      LOG.error("Invalid URI - Required qids, type, taskIds, subquery Id, and part id");
+    if (qids.size() != 1 && types.size() != 1 || stageIds.size() != 1) {
+      LOG.error("Invalid URI - Required qids, type, taskIds, stage Id, and part id");
       return null;
     }
 
     String queryId = qids.get(0);
     String shuffleType = types.get(0);
-    String sid = subQueryIds.get(0);
+    String sid = stageIds.get(0);
     String partId = partIds.get(0);
 
     if (shuffleType.equals("r") && taskIdList == null) {
@@ -767,10 +767,10 @@ public class Task {
     LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId
 	+ ", taskIds=" + taskIdList);
 
-    // The working directory of Tajo worker for each query, including subquery
+    // The working directory of Tajo worker for each query, including stage
     String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/";
 
-    // If the subquery requires a range shuffle
+    // If the stage requires a range shuffle
     if (shuffleType.equals("r")) {
       String ta = taskIds.get(0);
       if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) {
@@ -790,7 +790,7 @@ public class Task {
         return null;
       }
 
-      // If the subquery requires a hash shuffle or a scattered hash shuffle
+      // If the stage requires a hash shuffle or a scattered hash shuffle
     } else if (shuffleType.equals("h") || shuffleType.equals("s")) {
       int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf);
       String partPath = queryBaseDir + "hash-shuffle/" + partParentId + "/" + partId;

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 1556a44..3092c47 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
@@ -51,7 +51,7 @@ import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
 
 
 /**
- * Contains the information about executing subquery.
+ * Contains the information about executing task attempt.
  */
 public class TaskAttemptContext {
   private static final Log LOG = LogFactory.getLog(TaskAttemptContext.class);

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/resources/webapps/admin/querydetail.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/querydetail.jsp b/tajo-core/src/main/resources/webapps/admin/querydetail.jsp
index 41b0e8f..099301e 100644
--- a/tajo-core/src/main/resources/webapps/admin/querydetail.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/querydetail.jsp
@@ -25,7 +25,7 @@
 <%@ page import="java.text.SimpleDateFormat" %>
 <%@ page import="java.util.List" %>
 <%@ page import="org.apache.tajo.util.history.QueryHistory" %>
-<%@ page import="org.apache.tajo.util.history.SubQueryHistory" %>
+<%@ page import="org.apache.tajo.util.history.StageHistory" %>
 <%@ page import="org.apache.tajo.util.history.HistoryReader" %>
 
 <%
@@ -36,8 +36,8 @@
   String startTime = request.getParameter("startTime");
   QueryHistory queryHistory = reader.getQueryHistory(queryId, Long.parseLong(startTime));
 
-  List<SubQueryHistory> subQueryHistories =
-      queryHistory != null ? JSPUtil.sortSubQueryHistory(queryHistory.getSubQueryHistories()) : null;
+  List<StageHistory> stageHistories =
+      queryHistory != null ? JSPUtil.sortStageHistories(queryHistory.getStageHistories()) : null;
 
   SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 %>
@@ -61,34 +61,34 @@ if (queryHistory == null) {
   <div>No Query history data.</div>
 <%
 } else {
-  if (subQueryHistories == null) {
+  if (stageHistories == null) {
 %>
-    <div>No SubQuery history data.</div>
+    <div>No Stage history data.</div>
 <%
   } else {
 %>
   <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>Succeeded/Total</th><th>Failed/Killed</th></tr>
 <%
-    for(SubQueryHistory eachSubQuery: subQueryHistories) {
-        String detailLink = "querytasks.jsp?queryId=" + queryId + "&ebid=" + eachSubQuery.getExecutionBlockId() + "&startTime=" + startTime;
+    for(StageHistory eachStage: stageHistories) {
+        String detailLink = "querytasks.jsp?queryId=" + queryId + "&ebid=" + eachStage.getExecutionBlockId() + "&startTime=" + startTime;
 %>
   <tr>
-    <td><a href='<%=detailLink%>'><%=eachSubQuery.getExecutionBlockId()%></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'><%=eachSubQuery.getSucceededObjectCount()%> / <%=eachSubQuery.getTotalScheduledObjectsCount()%></td>
-    <td align='center'><%=eachSubQuery.getFailedObjectCount()%> / <%=eachSubQuery.getKilledObjectCount()%></td>
+    <td><a href='<%=detailLink%>'><%=eachStage.getExecutionBlockId()%></a></td>
+    <td><%=eachStage.getState()%></td>
+    <td><%=df.format(eachStage.getStartTime())%></td>
+    <td><%=eachStage.getFinishTime() == 0 ? "-" : df.format(eachStage.getFinishTime())%></td>
+    <td><%=JSPUtil.getElapsedTime(eachStage.getStartTime(), eachStage.getFinishTime())%></td>
+    <td align='center'><%=JSPUtil.percentFormat(eachStage.getProgress())%>%</td>
+    <td align='center'><%=eachStage.getSucceededObjectCount()%> / <%=eachStage.getTotalScheduledObjectsCount()%></td>
+    <td align='center'><%=eachStage.getFailedObjectCount()%> / <%=eachStage.getKilledObjectCount()%></td>
   </tr>
   <%
     }  //end of for
   %>
   </table>
 <%
-  }   //end of else [if (subQueryHistories == null)]
+  }   //end of else [if (stageHistories == null)]
 %>
   <p/>
   <h3>Applied Session Variables</h3>

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp
index ed97eff..09d9e2e 100644
--- a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp
@@ -28,7 +28,7 @@
 <%@ page import="org.apache.tajo.master.TajoMaster" %>
 <%@ page import="org.apache.tajo.util.history.HistoryReader" %>
 <%@ page import="org.apache.tajo.util.history.QueryHistory" %>
-<%@ page import="org.apache.tajo.util.history.SubQueryHistory" %>
+<%@ page import="org.apache.tajo.util.history.StageHistory" %>
 <%@ page import="org.apache.tajo.master.rm.Worker" %>
 <%@ page import="java.util.*" %>
 <%@ page import="org.apache.tajo.util.history.TaskHistory" %>
@@ -43,14 +43,14 @@
 
   QueryHistory queryHistory = reader.getQueryHistory(queryId);
 
-  List<SubQueryHistory> subQueryHistories =
-      queryHistory != null ? JSPUtil.sortSubQueryHistory(queryHistory.getSubQueryHistories()) : null;
+  List<StageHistory> stageHistories =
+      queryHistory != null ? JSPUtil.sortStageHistories(queryHistory.getStageHistories()) : null;
 
-  SubQueryHistory subQuery = null;
-  if (subQueryHistories != null) {
-    for (SubQueryHistory eachSubQuery: subQueryHistories) {
-      if (eachSubQuery.getExecutionBlockId().equals(ebId)) {
-        subQuery = eachSubQuery;
+  StageHistory stage = null;
+  if (stageHistories != null) {
+    for (StageHistory eachStage: stageHistories) {
+      if (eachStage.getExecutionBlockId().equals(ebId)) {
+        stage = eachStage;
         break;
       }
     }
@@ -92,12 +92,12 @@
   long totalWriteBytes = 0;
   long totalWriteRows = 0;
 
-  if (subQuery != null) {
-    totalInputBytes = subQuery.getTotalInputBytes();
-    totalReadBytes = subQuery.getTotalReadBytes();
-    totalReadRows = subQuery.getTotalReadRows();
-    totalWriteBytes = subQuery.getTotalWriteBytes();
-    totalWriteRows = subQuery.getTotalWriteRows();
+  if (stage != null) {
+    totalInputBytes = stage.getTotalInputBytes();
+    totalReadBytes = stage.getTotalReadBytes();
+    totalReadRows = stage.getTotalReadRows();
+    totalWriteBytes = stage.getTotalWriteBytes();
+    totalWriteRows = stage.getTotalWriteRows();
   }
 
   List<TaskHistory> allTasks = reader.getTaskHistory(queryId, ebId);
@@ -150,12 +150,12 @@
   <h3><a href='querydetail.jsp?queryId=<%=queryId%>&startTime=<%=startTime%>'><%=ebId.toString()%></a></h3>
   <hr/>
   <p/>
-  <pre style="white-space:pre-wrap;"><%=subQuery.getPlan()%></pre>
+  <pre style="white-space:pre-wrap;"><%=stage.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.getHostLocalAssigned()%>, Rack Local Tasks: <%=subQuery.getRackLocalAssigned()%>)</td></tr>
+    <tr><td align='right' width='180px'>Status:</td><td><%=stage.getState()%></td></tr>
+    <tr><td align='right'>Started:</td><td><%=df.format(stage.getStartTime())%> ~ <%=stage.getFinishTime() == 0 ? "-" : df.format(stage.getFinishTime())%></td></tr>
+    <tr><td align='right'># Tasks:</td><td><%=numTasks%> (Local Tasks: <%=stage.getHostLocalAssigned()%>, Rack Local Tasks: <%=stage.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>

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index ceb1c56..340eb95 100644
--- a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
@@ -31,7 +31,7 @@
 <%@ page import="java.util.Map" %>
 <%@ page import="org.apache.tajo.SessionVars" %>
 <%@ page import="org.apache.tajo.util.history.QueryHistory" %>
-<%@ page import="org.apache.tajo.util.history.SubQueryHistory" %>
+<%@ page import="org.apache.tajo.util.history.StageHistory" %>
 <%@ page import="org.apache.tajo.util.history.HistoryReader" %>
 
 <%
@@ -61,8 +61,8 @@
     return;
   }
 
-  List<SubQueryHistory> subQueryHistories =
-      queryHistory != null ? JSPUtil.sortSubQueryHistory(queryHistory.getSubQueryHistories()) : null;
+  List<StageHistory> stageHistories =
+      queryHistory != null ? JSPUtil.sortStageHistories(queryHistory.getStageHistories()) : null;
 
   SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 %>
@@ -86,26 +86,26 @@ if (runningQuery && query == null) {
   if (errorMessage != null && !errorMessage.isEmpty()) {
     out.write("<p/>Message:<p/><pre>" + errorMessage + "</pre>");
   }
-} else if (subQueryHistories == null) {
-  out.write("<p/>Message:<p/><pre>No SubQueries</pre>");
+} else if (stageHistories == null) {
+  out.write("<p/>Message:<p/><pre>No Stages</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(SubQueryHistory eachSubQuery: subQueryHistories) {
-    eachSubQuery.getSucceededObjectCount();
-    String detailLink = "querytasks.jsp?queryId=" + queryId + "&ebid=" + eachSubQuery.getExecutionBlockId();
+for(StageHistory eachStage: stageHistories) {
+    eachStage.getSucceededObjectCount();
+    String detailLink = "querytasks.jsp?queryId=" + queryId + "&ebid=" + eachStage.getExecutionBlockId();
 %>
   <tr>
-    <td><a href='<%=detailLink%>'><%=eachSubQuery.getExecutionBlockId()%></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>
+    <td><a href='<%=detailLink%>'><%=eachStage.getExecutionBlockId()%></a></td>
+    <td><%=eachStage.getState()%></td>
+    <td><%=df.format(eachStage.getStartTime())%></td>
+    <td><%=eachStage.getFinishTime() == 0 ? "-" : df.format(eachStage.getFinishTime())%></td>
+    <td><%=JSPUtil.getElapsedTime(eachStage.getStartTime(), eachStage.getFinishTime())%></td>
+    <td align='center'><%=JSPUtil.percentFormat(eachStage.getProgress())%>%</td>
+    <td align='center'><a href='<%=detailLink%>&status=SUCCEEDED'><%=eachStage.getSucceededObjectCount()%></a>/<a href='<%=detailLink%>&status=ALL'><%=eachStage.getTotalScheduledObjectsCount()%></a></td>
   </tr>
   <%
 }  //end of for

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index ec860b9..88de97d 100644
--- a/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
@@ -25,7 +25,7 @@
 <%@ 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.master.querymaster.Stage" %>
 <%@ page import="org.apache.tajo.engine.planner.global.ExecutionBlock" %>
 <%@ page import="java.util.*" %>
 <%@ page import="org.apache.tajo.ExecutionBlockId" %>
@@ -46,22 +46,22 @@
 
   Query query = queryMasterTask.getQuery();
 
-  Map<ExecutionBlockId, SubQuery> subQueryMap = new HashMap<ExecutionBlockId, SubQuery>();
+  Map<ExecutionBlockId, Stage> stageMap = new HashMap<ExecutionBlockId, Stage>();
 
-  for(SubQuery eachSubQuery: query.getSubQueries()) {
-    subQueryMap.put(eachSubQuery.getId(), eachSubQuery);
+  for(Stage eachStage : query.getStages()) {
+    stageMap.put(eachStage.getId(), eachStage);
   }
 
-  class SubQueryInfo {
+  class StageInfo {
     ExecutionBlock executionBlock;
-    SubQuery subQuery;
+    Stage stage;
     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) {
+    public StageInfo(ExecutionBlock executionBlock, Stage stage, ExecutionBlockId parentId, int px, int py, int pos) {
       this.executionBlock = executionBlock;
-      this.subQuery = subQuery;
+      this.stage = stage;
       this.parentId = parentId;
       this.px = px;
       this.py = py;
@@ -102,21 +102,21 @@
   String curIdStr = null;
   int x=35, y=1;
   int pos;
-  List<SubQueryInfo> subQueryInfos = new ArrayList<SubQueryInfo>();
+  List<StageInfo> stageInfos = new ArrayList<StageInfo>();
 
-  subQueryInfos.add(new SubQueryInfo(masterPlan.getRoot(), null, null, x, y, 0));
+  stageInfos.add(new StageInfo(masterPlan.getRoot(), null, null, x, y, 0));
 
-  while (!subQueryInfos.isEmpty()) {
-    SubQueryInfo eachSubQueryInfo = subQueryInfos.remove(0);
-    curIdStr = eachSubQueryInfo.executionBlock.getId().toString();
+  while (!stageInfos.isEmpty()) {
+    StageInfo eachStageInfo = stageInfos.remove(0);
+    curIdStr = eachStageInfo.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;
+    y = eachStageInfo.py + 13;
+    if (eachStageInfo.pos == 0) {
+      x = eachStageInfo.px;
+    } else if (eachStageInfo.pos == 1) {
+      x = eachStageInfo.px - 20;
+    } else if (eachStageInfo.pos == 2) {
+      x = eachStageInfo.px + 20;
     }
 %>
   <script type='text/javascript'>
@@ -128,17 +128,17 @@
   </div>
 
 <%
-    if (eachSubQueryInfo.parentId != null) {
+    if (eachStageInfo.parentId != null) {
       String outgoing = "";
       String prefix = "";
-      for (DataChannel channel : masterPlan.getOutgoingChannels(eachSubQueryInfo.executionBlock.getId())) {
+      for (DataChannel channel : masterPlan.getOutgoingChannels(eachStageInfo.executionBlock.getId())) {
         outgoing += prefix + channel.getShuffleType();
         prefix = "; ";
       }
 %>
   <script type="text/javascript">
     var srcId = "<%=curIdStr%>";
-    var destId = "<%=eachSubQueryInfo.parentId.toString()%>";
+    var destId = "<%=eachStageInfo.parentId.toString()%>";
     var src = window.jsPlumb.addEndpoint(srcId, {
         anchor:"AutoDefault",
         paintStyle:{
@@ -187,7 +187,7 @@
 
   <script type='text/javascript'>
     var e = document.getElementById("<%=curIdStr%>");
-    var state = "<%=eachSubQueryInfo.subQuery != null ? eachSubQueryInfo.subQuery.getState().name(): ""%>";
+    var state = "<%=eachStageInfo.stage != null ? eachStageInfo.stage.getState().name(): ""%>";
     switch (state) {
       case 'NEW':
         e.style.borderColor = "black";
@@ -219,7 +219,7 @@
   </script>
 
 <%
-    List<ExecutionBlock> children = masterPlan.getChilds(eachSubQueryInfo.executionBlock.getId());
+    List<ExecutionBlock> children = masterPlan.getChilds(eachStageInfo.executionBlock.getId());
 
     if (children.size() == 1) {
       pos = 0;
@@ -227,7 +227,7 @@
       pos = 1;
     }
     for (ExecutionBlock child : children) {
-      subQueryInfos.add(new SubQueryInfo(child, subQueryMap.get(child.getId()), eachSubQueryInfo.executionBlock.getId(), x, y, pos++));
+      stageInfos.add(new StageInfo(child, stageMap.get(child.getId()), eachStageInfo.executionBlock.getId(), x, y, pos++));
     }
   } //end of while
 %>

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 265937c..3aef49d 100644
--- a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
@@ -78,14 +78,14 @@
   }
 
   Query query = queryMasterTask.getQuery();
-  SubQuery subQuery = query.getSubQuery(ebid);
+  Stage stage = query.getStage(ebid);
 
-  if(subQuery == null) {
+  if(stage == null) {
     out.write("<script type='text/javascript'>alert('no sub-query'); history.back(0); </script>");
     return;
   }
 
-  if(subQuery == null) {
+  if(stage == null) {
 %>
 <script type="text/javascript">
   alert("No Execution Block for" + ebid);
@@ -97,7 +97,7 @@
 
   SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 
-  Task[] allTasks = subQuery.getTasks();
+  Task[] allTasks = stage.getTasks();
 
   long totalInputBytes = 0;
   long totalReadBytes = 0;
@@ -105,8 +105,6 @@
   long totalWriteBytes = 0;
   long totalWriteRows = 0;
   int numTasks = allTasks.length;
-//  int numSucceededTasks = 0;
-//  int localReadTasks = subQuery.;
   int numShuffles = 0;
 
   float totalProgress = 0.0f;
@@ -166,12 +164,12 @@
   <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>
+  <pre style="white-space:pre-wrap;"><%=PlannerUtil.buildExplainString(stage.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' width='180px'>Status:</td><td><%=stage.getState()%></td></tr>
+    <tr><td align='right'>Started:</td><td><%=df.format(stage.getStartTime())%> ~ <%=stage.getFinishTime() == 0 ? "-" : df.format(stage.getFinishTime())%></td></tr>
+    <tr><td align='right'># Tasks:</td><td><%=numTasks%> (Local Tasks: <%=stage.getTaskScheduler().getHostLocalAssigned()%>, Rack Local Tasks: <%=stage.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>

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/resources/webapps/worker/task.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/task.jsp b/tajo-core/src/main/resources/webapps/worker/task.jsp
index 5abbd8c..81b1e6d 100644
--- a/tajo-core/src/main/resources/webapps/worker/task.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/task.jsp
@@ -28,7 +28,7 @@
 <%@ page import="org.apache.tajo.master.querymaster.Query" %>
 <%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
 <%@ page import="org.apache.tajo.master.querymaster.Task" %>
-<%@ page import="org.apache.tajo.master.querymaster.SubQuery" %>
+<%@ page import="org.apache.tajo.master.querymaster.Stage" %>
 <%@ page import="org.apache.tajo.storage.DataLocation" %>
 <%@ page import="org.apache.tajo.storage.fragment.FileFragment" %>
 <%@ page import="org.apache.tajo.storage.fragment.FragmentConvertor" %>
@@ -65,14 +65,14 @@
     }
 
     Query query = queryMasterTask.getQuery();
-    SubQuery subQuery = query.getSubQuery(ebid);
+    Stage stage = query.getStage(ebid);
 
-    if(subQuery == null) {
+    if(stage == null) {
         out.write("<script type='text/javascript'>alert('no sub-query'); history.back(0); </script>");
         return;
     }
 
-    if(subQuery == null) {
+    if(stage == null) {
 %>
 <script type="text/javascript">
     alert("No Execution Block for" + ebid);
@@ -84,7 +84,7 @@
     SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 
     TaskId taskId = new TaskId(ebid, taskSeq);
-    Task task = subQuery.getTask(taskId);
+    Task task = stage.getTask(taskId);
     if(task == null) {
 %>
 <script type="text/javascript">

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 9868297..7dc1089 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -45,8 +45,8 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.master.TajoMaster;
 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.master.querymaster.Stage;
+import org.apache.tajo.master.querymaster.StageState;
 import org.apache.tajo.master.rm.TajoWorkerResourceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
@@ -804,10 +804,10 @@ public class TajoTestingCluster {
     }
   }
 
-  public void waitForSubQueryState(SubQuery subQuery, SubQueryState expected, int delay) throws Exception {
+  public void waitForStageState(Stage stage, StageState expected, int delay) throws Exception {
 
     int i = 0;
-    while (subQuery == null || subQuery.getSynchronizedState() != expected) {
+    while (stage == null || stage.getSynchronizedState() != expected) {
       try {
         Thread.sleep(delay);
       } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index 836dd2f..8dc95de 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TestQueryIdFactory.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TestQueryIdFactory.java
@@ -38,12 +38,12 @@ public class TestQueryIdFactory {
   }
   
   @Test
-  public void testNewSubQueryId() {
+  public void testNewStageId() {
     QueryId qid = LocalTajoTestingUtility.newQueryId();
     MasterPlan plan = new MasterPlan(qid, null, null);
-    ExecutionBlockId subqid1 = plan.newExecutionBlockId();
-    ExecutionBlockId subqid2 = plan.newExecutionBlockId();
-    assertTrue(subqid1.compareTo(subqid2) < 0);
+    ExecutionBlockId stageId1 = plan.newExecutionBlockId();
+    ExecutionBlockId stageId2 = plan.newExecutionBlockId();
+    assertTrue(stageId1.compareTo(stageId2) < 0);
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index d187071..113288a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -38,7 +38,7 @@ import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.QueryHistoryProto;
 import org.apache.tajo.ipc.ClientProtos.QueryInfoProto;
-import org.apache.tajo.ipc.ClientProtos.SubQueryHistoryProto;
+import org.apache.tajo.ipc.ClientProtos.StageHistoryProto;
 import org.apache.tajo.jdbc.TajoResultSet;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.storage.StorageUtil;
@@ -785,13 +785,13 @@ public class TestTajoClient {
     QueryHistoryProto queryHistory = client.getQueryHistory(queryId);
     assertNotNull(queryHistory);
     assertEquals(queryId.toString(), queryHistory.getQueryId());
-    assertEquals(2, queryHistory.getSubQueryHistoriesCount());
+    assertEquals(2, queryHistory.getStageHistoriesCount());
 
-    List<SubQueryHistoryProto> taskHistories =
-        new ArrayList<SubQueryHistoryProto>(queryHistory.getSubQueryHistoriesList());
-    Collections.sort(taskHistories, new Comparator<SubQueryHistoryProto>() {
+    List<ClientProtos.StageHistoryProto> taskHistories =
+        new ArrayList<StageHistoryProto>(queryHistory.getStageHistoriesList());
+    Collections.sort(taskHistories, new Comparator<StageHistoryProto>() {
       @Override
-      public int compare(SubQueryHistoryProto o1, SubQueryHistoryProto o2) {
+      public int compare(ClientProtos.StageHistoryProto o1, StageHistoryProto o2) {
         return o1.getExecutionBlockId().compareTo(o2.getExecutionBlockId());
       }
     });

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index edbe029..bfd1700 100644
--- 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
@@ -27,7 +27,7 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.master.querymaster.Query;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.master.querymaster.Task;
-import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.Stage;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
@@ -367,7 +367,7 @@ public class TestGroupByQuery extends QueryTestCaseBase {
     assertResultSet(res, "testDistinctAggregation_case4.result");
     res.close();
 
-    // two groupby, two distinct, two aggregation with subquery
+    // two groupby, two distinct, two aggregation with stage
     res = executeFile("testDistinctAggregation_case5.sql");
     assertResultSet(res, "testDistinctAggregation_case5.result");
     res.close();
@@ -731,12 +731,12 @@ public class TestGroupByQuery extends QueryTestCaseBase {
       Set<Integer> partitionIds = new HashSet<Integer>();
 
       Query query = qmTasks.get(qmTasks.size() - 1).getQuery();
-      Collection<SubQuery> subQueries = query.getSubQueries();
-      assertNotNull(subQueries);
-      assertTrue(!subQueries.isEmpty());
-      for (SubQuery subQuery: subQueries) {
-        if (subQuery.getId().toStringNoPrefix().endsWith("_000001")) {
-          for (Task.IntermediateEntry eachInterm: subQuery.getHashShuffleIntermediateEntries()) {
+      Collection<Stage> stages = query.getStages();
+      assertNotNull(stages);
+      assertTrue(!stages.isEmpty());
+      for (Stage stage : stages) {
+        if (stage.getId().toStringNoPrefix().endsWith("_000001")) {
+          for (Task.IntermediateEntry eachInterm: stage.getHashShuffleIntermediateEntries()) {
             partitionIds.add(eachInterm.getPartId());
           }
         }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index b1e1bec..3400752 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -287,7 +287,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertResultSet(res, "case12.result");
     res.close();
 
-    // alias partition column in subquery
+    // alias partition column in stage
     res = executeFile("case13.sql");
     assertResultSet(res, "case13.result");
     res.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
index 3d292a4..d46d110 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
@@ -145,7 +145,7 @@ public class TestUnionQuery extends QueryTestCaseBase {
   @Test
   public final void testUnion12() throws Exception {
     // test filter pushdown
-    // with subquery in union query
+    // with stage in union query
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);
@@ -154,7 +154,7 @@ public class TestUnionQuery extends QueryTestCaseBase {
   @Test
   public final void testUnion13() throws Exception {
     // test filter pushdown
-    // with subquery in union query
+    // with stage in union query
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);
@@ -163,7 +163,7 @@ public class TestUnionQuery extends QueryTestCaseBase {
   @Test
   public final void testUnion14() throws Exception {
     // test filter pushdown
-    // with group by subquery in union query
+    // with group by stage in union query
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java
index 37ee402..c1f4178 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java
@@ -85,13 +85,13 @@ public class TestKillQuery {
       assertEquals(TajoProtos.QueryState.QUERY_RUNNING, queryMasterTask.getQuery().getSynchronizedState());
     }
 
-    SubQuery subQuery = queryMasterTask.getQuery().getSubQueries().iterator().next();
-    assertNotNull(subQuery);
+    Stage stage = queryMasterTask.getQuery().getStages().iterator().next();
+    assertNotNull(stage);
 
     try{
-      cluster.waitForSubQueryState(subQuery, SubQueryState.INITED, 2);
+      cluster.waitForStageState(stage, StageState.INITED, 2);
     } finally {
-      assertEquals(SubQueryState.INITED, subQuery.getSynchronizedState());
+      assertEquals(StageState.INITED, stage.getSynchronizedState());
     }
 
     // fire kill event

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java
index 1c2a1a8..3a54478 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java
@@ -103,7 +103,7 @@ public class TestTaskStatusUpdate extends QueryTestCaseBase {
       String actualResult = resultSetToString(res);
       System.out.println(actualResult);
 
-      // in/out * subquery(4)
+      // in/out * stage(4)
       long[] expectedNumRows = new long[]{2, 2, 5, 5, 7, 2, 2, 2};
       long[] expectedNumBytes = new long[]{8, 34, 20, 75, 109, 34, 34, 18};
       long[] expectedReadBytes = new long[]{8, 8, 20, 20, 109, 0, 34, 0};
@@ -131,7 +131,7 @@ public class TestTaskStatusUpdate extends QueryTestCaseBase {
     res.close();
   }
 
-  private void assertStatus(int numSubQueries,
+  private void assertStatus(int numStages,
                             long[] expectedNumRows,
                             long[] expectedNumBytes,
                             long[] expectedReadBytes) throws Exception {
@@ -160,20 +160,20 @@ public class TestTaskStatusUpdate extends QueryTestCaseBase {
 
       assertNotNull(query);
 
-      List<SubQuery> subQueries = new ArrayList<SubQuery>(query.getSubQueries());
-      assertEquals(numSubQueries, subQueries.size());
+      List<Stage> stages = new ArrayList<Stage>(query.getStages());
+      assertEquals(numStages, stages.size());
 
-      Collections.sort(subQueries, new Comparator<SubQuery>() {
+      Collections.sort(stages, new Comparator<Stage>() {
         @Override
-        public int compare(SubQuery o1, SubQuery o2) {
+        public int compare(Stage o1, Stage o2) {
           return o1.getId().compareTo(o2.getId());
         }
       });
 
       int index = 0;
-      for (SubQuery eachSubQuery: subQueries) {
-        TableStats inputStats = eachSubQuery.getInputStats();
-        TableStats resultStats = eachSubQuery.getResultStats();
+      for (Stage eachStage : stages) {
+        TableStats inputStats = eachStage.getInputStats();
+        TableStats resultStats = eachStage.getResultStats();
 
         assertNotNull(inputStats);
         assertEquals(expectedNumRows[index], inputStats.getNumRows().longValue());

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
index 8d6a94d..632e9c2 100644
--- a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
+++ b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
@@ -126,12 +126,12 @@ public class TestHistoryWriterReader extends QueryTestCaseBase {
       QueryId queryId = QueryIdFactory.newQueryId(startTime, 1);
       queryHistory.setQueryId(queryId.toString());
       queryHistory.setLogicalPlan("LogicalPlan");
-      List<SubQueryHistory> subQueries = new ArrayList<SubQueryHistory>();
+      List<StageHistory> stages = new ArrayList<StageHistory>();
       for (int i = 0; i < 3; i++) {
         ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(queryId, i);
-        SubQueryHistory subQueryHistory = new SubQueryHistory();
-        subQueryHistory.setExecutionBlockId(ebId.toString());
-        subQueryHistory.setStartTime(startTime + i);
+        StageHistory stageHistory = new StageHistory();
+        stageHistory.setExecutionBlockId(ebId.toString());
+        stageHistory.setStartTime(startTime + i);
 
         List<TaskHistory> taskHistories = new ArrayList<TaskHistory>();
         for (int j = 0; j < 5; j++) {
@@ -139,10 +139,10 @@ public class TestHistoryWriterReader extends QueryTestCaseBase {
           taskHistory.setId(QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(ebId), 1).toString());
           taskHistories.add(taskHistory);
         }
-        subQueryHistory.setTasks(taskHistories);
-        subQueries.add(subQueryHistory);
+        stageHistory.setTasks(taskHistories);
+        stages.add(stageHistory);
       }
-      queryHistory.setSubQueryHistories(subQueries);
+      queryHistory.setStageHistories(stages);
 
       writer.appendHistory(queryHistory);
 
@@ -166,16 +166,16 @@ public class TestHistoryWriterReader extends QueryTestCaseBase {
       QueryHistory foundQueryHistory = reader.getQueryHistory(queryId.toString());
       assertNotNull(foundQueryHistory);
       assertEquals(queryId.toString(), foundQueryHistory.getQueryId());
-      assertEquals(3, foundQueryHistory.getSubQueryHistories().size());
+      assertEquals(3, foundQueryHistory.getStageHistories().size());
 
       for (int i = 0; i < 3; i++) {
         String ebId = QueryIdFactory.newExecutionBlockId(queryId, i).toString();
-        SubQueryHistory subQueryHistory = foundQueryHistory.getSubQueryHistories().get(i);
-        assertEquals(ebId, subQueryHistory.getExecutionBlockId());
-        assertEquals(startTime + i, subQueryHistory.getStartTime());
+        StageHistory stageHistory = foundQueryHistory.getStageHistories().get(i);
+        assertEquals(ebId, stageHistory.getExecutionBlockId());
+        assertEquals(startTime + i, stageHistory.getStartTime());
 
         // TaskHistory is stored in the other file.
-        assertNull(subQueryHistory.getTasks());
+        assertNull(stageHistory.getTasks());
 
         List<TaskHistory> tasks = reader.getTaskHistory(queryId.toString(), ebId);
         assertNotNull(tasks);
@@ -183,7 +183,7 @@ public class TestHistoryWriterReader extends QueryTestCaseBase {
 
         for (int j = 0; j < 5; j++) {
           TaskHistory taskHistory = tasks.get(j);
-          assertEquals(subQueries.get(i).getTasks().get(j).getId(), taskHistory.getId());
+          assertEquals(stages.get(i).getTasks().get(j).getId(), taskHistory.getId());
         }
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index c408b16..69b91f9 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -224,7 +224,7 @@
                   <arguments>
                     <argument>Tajo</argument>
                     <argument>org.apache.tajo.master.querymaster.Query,
-                      org.apache.tajo.master.querymaster.SubQuery,
+                      org.apache.tajo.master.querymaster.Stage,
                       org.apache.tajo.master.querymaster.Task,
                       org.apache.tajo.master.querymaster.TaskAttempt
                     </argument>

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
index 1c63c8a..d633058 100644
--- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
@@ -363,18 +363,18 @@ public class PullServerAuxService extends AuxiliaryService {
           new QueryStringDecoder(request.getUri()).getParameters();
       final List<String> types = params.get("type");
       final List<String> taskIdList = params.get("ta");
-      final List<String> subQueryIds = params.get("sid");
+      final List<String> stageIds = params.get("sid");
       final List<String> partitionIds = params.get("p");
 
-      if (types == null || taskIdList == null || subQueryIds == null
+      if (types == null || taskIdList == null || stageIds == null
           || partitionIds == null) {
-        sendError(ctx, "Required type, taskIds, subquery Id, and partition id",
+        sendError(ctx, "Required type, taskIds, stage Id, and partition id",
             BAD_REQUEST);
         return;
       }
 
-      if (types.size() != 1 || subQueryIds.size() != 1) {
-        sendError(ctx, "Required type, taskIds, subquery Id, and partition id",
+      if (types.size() != 1 || stageIds.size() != 1) {
+        sendError(ctx, "Required type, taskIds, stage Id, and partition id",
             BAD_REQUEST);
         return;
       }
@@ -382,7 +382,7 @@ public class PullServerAuxService extends AuxiliaryService {
       final List<FileChunk> chunks = Lists.newArrayList();
 
       String repartitionType = types.get(0);
-      String sid = subQueryIds.get(0);
+      String sid = stageIds.get(0);
       String partitionId = partitionIds.get(0);
       List<String> taskIds = splitMaps(taskIdList);
 
@@ -399,7 +399,7 @@ public class PullServerAuxService extends AuxiliaryService {
       }
       LOG.info("PullServer baseDir: " + taskLocalDir + "/" + queryBaseDir);
 
-      // if a subquery requires a range partitioning
+      // if a stage requires a range partitioning
       if (repartitionType.equals("r")) {
         String ta = taskIds.get(0);
         Path path = localFS.makeQualified(
@@ -422,7 +422,7 @@ public class PullServerAuxService extends AuxiliaryService {
           chunks.add(chunk);
         }
 
-        // if a subquery requires a hash repartition  or a scattered hash repartition
+        // if a stage requires a hash repartition  or a scattered hash repartition
       } else if (repartitionType.equals("h") || repartitionType.equals("s")) {
         for (String ta : taskIds) {
           Path path = localFS.makeQualified(

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
index 5a4e69f..860bc8e 100644
--- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
@@ -493,19 +493,19 @@ public class TajoPullServerService extends AbstractService {
       final List<String> types = params.get("type");
       final List<String> qids = params.get("qid");
       final List<String> taskIdList = params.get("ta");
-      final List<String> subQueryIds = params.get("sid");
+      final List<String> stageIds = params.get("sid");
       final List<String> partIds = params.get("p");
       final List<String> offsetList = params.get("offset");
       final List<String> lengthList = params.get("length");
 
-      if (types == null || subQueryIds == null || qids == null || partIds == null) {
-        sendError(ctx, "Required queryId, type, subquery Id, and part id",
+      if (types == null || stageIds == null || qids == null || partIds == null) {
+        sendError(ctx, "Required queryId, type, stage Id, and part id",
             BAD_REQUEST);
         return;
       }
 
-      if (qids.size() != 1 && types.size() != 1 || subQueryIds.size() != 1) {
-        sendError(ctx, "Required qids, type, taskIds, subquery Id, and part id",
+      if (qids.size() != 1 && types.size() != 1 || stageIds.size() != 1) {
+        sendError(ctx, "Required qids, type, taskIds, stage Id, and part id",
             BAD_REQUEST);
         return;
       }
@@ -513,7 +513,7 @@ public class TajoPullServerService extends AbstractService {
       String partId = partIds.get(0);
       String queryId = qids.get(0);
       String shuffleType = types.get(0);
-      String sid = subQueryIds.get(0);
+      String sid = stageIds.get(0);
 
       long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L;
       long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L;
@@ -536,7 +536,7 @@ public class TajoPullServerService extends AbstractService {
 
       final List<FileChunk> chunks = Lists.newArrayList();
 
-      // if a subquery requires a range shuffle
+      // if a stage requires a range shuffle
       if (shuffleType.equals("r")) {
         String ta = taskIds.get(0);
         if(!lDirAlloc.ifExists(queryBaseDir + "/" + sid + "/" + ta + "/output/", conf)){
@@ -562,7 +562,7 @@ public class TajoPullServerService extends AbstractService {
           chunks.add(chunk);
         }
 
-        // if a subquery requires a hash shuffle or a scattered hash shuffle
+        // if a stage requires a hash shuffle or a scattered hash shuffle
       } else if (shuffleType.equals("h") || shuffleType.equals("s")) {
         int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf);
         String partPath = queryBaseDir + "/" + sid + "/hash-shuffle/" + partParentId + "/" + partId;

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
index 3d4f7d5..c427940 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
@@ -49,7 +49,7 @@ public class FileStorageManager extends StorageManager {
   private final Log LOG = LogFactory.getLog(FileStorageManager.class);
 
   static final String OUTPUT_FILE_PREFIX="part-";
-  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SUBQUERY =
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_STAGE =
       new ThreadLocal<NumberFormat>() {
         @Override
         public NumberFormat initialValue() {
@@ -274,10 +274,10 @@ public class FileStorageManager extends StorageManager {
       return workDir;
     }
     // 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.
+    // where ss is the stage id associated with this task, and nnnnnn is the task id.
     Path outFilePath = StorageUtil.concatPath(workDir, TajoConstants.RESULT_DIR_NAME,
         OUTPUT_FILE_PREFIX +
-            OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskAttemptId.getTaskId().getExecutionBlockId().getId()) + "-" +
+            OUTPUT_FILE_FORMAT_STAGE.get().format(taskAttemptId.getTaskId().getExecutionBlockId().getId()) + "-" +
             OUTPUT_FILE_FORMAT_TASK.get().format(taskAttemptId.getTaskId().getId()) + "-" +
             OUTPUT_FILE_FORMAT_SEQ.get().format(0));
     LOG.info("Output File Path: " + outFilePath);


[9/9] tajo git commit: Merge branch 'index_support' of http://git-wip-us.apache.org/repos/asf/tajo into index_support

Posted by ji...@apache.org.
Merge branch 'index_support' of http://git-wip-us.apache.org/repos/asf/tajo into index_support


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

Branch: refs/heads/index_support
Commit: 3b5c28711dcb372a75bf06d0d075defbd63fd743
Parents: 829fcd9 94faf6e
Author: Jihoon Son <ji...@apache.org>
Authored: Mon Dec 22 17:11:06 2014 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Mon Dec 22 17:11:06 2014 +0900

----------------------------------------------------------------------

----------------------------------------------------------------------



[6/9] tajo git commit: TAJO-1262: Rename the prefix 'SubQuery' to 'Stage'.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/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
index cf6b917..4cf6ce2 100644
--- 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
@@ -81,11 +81,11 @@ public class Repartitioner {
   private final static int HTTP_REQUEST_MAXIMUM_LENGTH = 1900;
   private final static String UNKNOWN_HOST = "unknown";
 
-  public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerContext, SubQuery subQuery)
+  public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerContext, Stage stage)
       throws IOException {
-    MasterPlan masterPlan = subQuery.getMasterPlan();
-    ExecutionBlock execBlock = subQuery.getBlock();
-    QueryMasterTask.QueryMasterTaskContext masterContext = subQuery.getContext();
+    MasterPlan masterPlan = stage.getMasterPlan();
+    ExecutionBlock execBlock = stage.getBlock();
+    QueryMasterTask.QueryMasterTaskContext masterContext = stage.getContext();
 
     ScanNode[] scans = execBlock.getScanNodes();
 
@@ -98,17 +98,17 @@ public class Repartitioner {
       TableDesc tableDesc = masterContext.getTableDescMap().get(scans[i].getCanonicalName());
       if (tableDesc == null) { // if it is a real table stored on storage
         FileStorageManager storageManager =
-            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+            (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
 
         tablePath = storageManager.getTablePath(scans[i].getTableName());
         if (execBlock.getUnionScanMap() != null && !execBlock.getUnionScanMap().isEmpty()) {
           for (Map.Entry<ExecutionBlockId, ExecutionBlockId> unionScanEntry: execBlock.getUnionScanMap().entrySet()) {
             ExecutionBlockId originScanEbId = unionScanEntry.getKey();
-            stats[i] += masterContext.getSubQuery(originScanEbId).getResultStats().getNumBytes();
+            stats[i] += masterContext.getStage(originScanEbId).getResultStats().getNumBytes();
           }
         } else {
           ExecutionBlockId scanEBId = TajoIdUtils.createExecutionBlockId(scans[i].getTableName());
-          stats[i] = masterContext.getSubQuery(scanEBId).getResultStats().getNumBytes();
+          stats[i] = masterContext.getStage(scanEBId).getResultStats().getNumBytes();
         }
         fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
       } else {
@@ -119,7 +119,7 @@ public class Repartitioner {
         }
 
         StorageManager storageManager =
-            StorageManager.getStorageManager(subQuery.getContext().getConf(), tableDesc.getMeta().getStoreType());
+            StorageManager.getStorageManager(stage.getContext().getConf(), tableDesc.getMeta().getStoreType());
 
         // if table has no data, storageManager will return empty FileFragment.
         // So, we need to handle FileFragment by its size.
@@ -223,7 +223,7 @@ public class Repartitioner {
       execBlock.removeBroadcastTable(scans[baseScanIdx].getCanonicalName());
       LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d",
           scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx]));
-      scheduleLeafTasksWithBroadcastTable(schedulerContext, subQuery, baseScanIdx, fragments);
+      scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments);
     } else if (!execBlock.getBroadcastTables().isEmpty()) { // If some relations of this EB are broadcasted
       boolean hasNonLeafNode = false;
       List<Integer> largeScanIndexList = new ArrayList<Integer>();
@@ -266,7 +266,7 @@ public class Repartitioner {
         int baseScanIdx = largeScanIndexList.isEmpty() ? maxStatsScanIdx : largeScanIndexList.get(0);
         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);
+        scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments);
       } else {
         if (largeScanIndexList.size() > 2) {
           throw new IOException("Symmetric Repartition Join should have two scan node, but " + nonLeafScanNames);
@@ -292,12 +292,12 @@ public class Repartitioner {
           index++;
         }
         LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join, join_node=%s", nonLeafScanNames));
-        scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, subQuery,
+        scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage,
             intermediateScans, intermediateScanStats, intermediateFragments, broadcastScans, broadcastFragments);
       }
     } else {
       LOG.info("[Distributed Join Strategy] : Symmetric Repartition Join");
-      scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, subQuery, scans, stats, fragments, null, null);
+      scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage, scans, stats, fragments, null, null);
     }
   }
 
@@ -305,7 +305,7 @@ public class Repartitioner {
    * Scheduling in tech case of Symmetric Repartition Join
    * @param masterContext
    * @param schedulerContext
-   * @param subQuery
+   * @param stage
    * @param scans
    * @param stats
    * @param fragments
@@ -313,21 +313,21 @@ public class Repartitioner {
    */
   private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMasterTaskContext masterContext,
                                                        TaskSchedulerContext schedulerContext,
-                                                       SubQuery subQuery,
+                                                       Stage stage,
                                                        ScanNode[] scans,
                                                        long[] stats,
                                                        Fragment[] fragments,
                                                        ScanNode[] broadcastScans,
                                                        Fragment[] broadcastFragments) throws IOException {
-    MasterPlan masterPlan = subQuery.getMasterPlan();
-    ExecutionBlock execBlock = subQuery.getBlock();
+    MasterPlan masterPlan = stage.getMasterPlan();
+    ExecutionBlock execBlock = stage.getBlock();
     // The hash map is modeling as follows:
     // <Part Id, <EbId, List<Intermediate Data>>>
     Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> hashEntries =
         new HashMap<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>>();
 
     // Grouping IntermediateData by a partition key and a table name
-    List<ExecutionBlock> childBlocks = masterPlan.getChilds(subQuery.getId());
+    List<ExecutionBlock> childBlocks = masterPlan.getChilds(stage.getId());
 
     // In the case of join with union, there is one ScanNode for union.
     Map<ExecutionBlockId, ExecutionBlockId> unionScanMap = execBlock.getUnionScanMap();
@@ -336,7 +336,7 @@ public class Repartitioner {
       if (scanEbId == null) {
         scanEbId = childBlock.getId();
       }
-      SubQuery childExecSM = subQuery.getContext().getSubQuery(childBlock.getId());
+      Stage childExecSM = stage.getContext().getStage(childBlock.getId());
 
       if (childExecSM.getHashShuffleIntermediateEntries() != null &&
           !childExecSM.getHashShuffleIntermediateEntries().isEmpty()) {
@@ -387,7 +387,7 @@ public class Repartitioner {
     // 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.getMasterPlan().getContext().getInt(SessionVars.JOIN_TASK_INPUT_SIZE);
+    int desireJoinTaskVolumn = stage.getMasterPlan().getContext().getInt(SessionVars.JOIN_TASK_INPUT_SIZE);
 
     // calculate the number of tasks according to the data size
     int mb = (int) Math.ceil((double) stats[largerIdx] / 1048576);
@@ -412,7 +412,7 @@ public class Repartitioner {
         TableDesc tableDesc = masterContext.getTableDescMap().get(eachScan.getCanonicalName());
         if (eachScan.getType() == NodeType.PARTITIONS_SCAN) {
           FileStorageManager storageManager =
-              (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+              (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
 
           PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)eachScan;
           partitionScanPaths = partitionScan.getInputPaths();
@@ -420,7 +420,7 @@ public class Repartitioner {
           getFragmentsFromPartitionedTable(storageManager, eachScan, tableDesc);
           partitionScan.setInputPaths(partitionScanPaths);
         } else {
-          StorageManager storageManager = StorageManager.getStorageManager(subQuery.getContext().getConf(),
+          StorageManager storageManager = StorageManager.getStorageManager(stage.getContext().getConf(),
               tableDesc.getMeta().getStoreType());
           Collection<Fragment> scanFragments = storageManager.getSplits(eachScan.getCanonicalName(),
               tableDesc, eachScan);
@@ -430,12 +430,12 @@ public class Repartitioner {
         }
       }
     }
-    SubQuery.scheduleFragment(subQuery, fragments[0], rightFragments);
+    Stage.scheduleFragment(stage, fragments[0], rightFragments);
 
     // Assign partitions to tasks in a round robin manner.
     for (Entry<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> entry
         : hashEntries.entrySet()) {
-      addJoinShuffle(subQuery, entry.getKey(), entry.getValue());
+      addJoinShuffle(stage, entry.getKey(), entry.getValue());
     }
 
     schedulerContext.setTaskSize((int) Math.ceil((double) bothFetchSize / joinTaskNum));
@@ -503,9 +503,9 @@ public class Repartitioner {
     return fragments;
   }
 
-  private static void scheduleLeafTasksWithBroadcastTable(TaskSchedulerContext schedulerContext, SubQuery subQuery,
+  private static void scheduleLeafTasksWithBroadcastTable(TaskSchedulerContext schedulerContext, Stage stage,
                                                           int baseScanId, Fragment[] fragments) throws IOException {
-    ExecutionBlock execBlock = subQuery.getBlock();
+    ExecutionBlock execBlock = stage.getBlock();
     ScanNode[] scans = execBlock.getScanNodes();
 
     for (int i = 0; i < scans.length; i++) {
@@ -527,7 +527,7 @@ public class Repartitioner {
     List<Fragment> broadcastFragments = new ArrayList<Fragment>();
     for (int i = 0; i < scans.length; i++) {
       ScanNode scan = scans[i];
-      TableDesc desc = subQuery.getContext().getTableDescMap().get(scan.getCanonicalName());
+      TableDesc desc = stage.getContext().getTableDescMap().get(scan.getCanonicalName());
       TableMeta meta = desc.getMeta();
 
       Collection<Fragment> scanFragments;
@@ -537,11 +537,11 @@ public class Repartitioner {
         partitionScanPaths = partitionScan.getInputPaths();
         // set null to inputPaths in getFragmentsFromPartitionedTable()
         FileStorageManager storageManager =
-            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+            (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
         scanFragments = getFragmentsFromPartitionedTable(storageManager, scan, desc);
       } else {
         StorageManager storageManager =
-            StorageManager.getStorageManager(subQuery.getContext().getConf(), desc.getMeta().getStoreType());
+            StorageManager.getStorageManager(stage.getContext().getConf(), desc.getMeta().getStoreType());
 
         scanFragments = storageManager.getSplits(scan.getCanonicalName(), desc, scan);
       }
@@ -565,14 +565,14 @@ public class Repartitioner {
       throw new IOException("No fragments for " + scans[baseScanId].getTableName());
     }
 
-    SubQuery.scheduleFragments(subQuery, baseFragments, broadcastFragments);
+    Stage.scheduleFragments(stage, baseFragments, broadcastFragments);
     schedulerContext.setEstimatedTaskNum(baseFragments.size());
   }
 
-  private static void addJoinShuffle(SubQuery subQuery, int partitionId,
+  private static void addJoinShuffle(Stage stage, int partitionId,
                                      Map<ExecutionBlockId, List<IntermediateEntry>> grouppedPartitions) {
     Map<String, List<FetchImpl>> fetches = new HashMap<String, List<FetchImpl>>();
-    for (ExecutionBlock execBlock : subQuery.getMasterPlan().getChilds(subQuery.getId())) {
+    for (ExecutionBlock execBlock : stage.getMasterPlan().getChilds(stage.getId())) {
       if (grouppedPartitions.containsKey(execBlock.getId())) {
         Collection<FetchImpl> requests = mergeShuffleRequest(partitionId, HASH_SHUFFLE,
             grouppedPartitions.get(execBlock.getId()));
@@ -581,10 +581,10 @@ public class Repartitioner {
     }
 
     if (fetches.isEmpty()) {
-      LOG.info(subQuery.getId() + "'s " + partitionId + " partition has empty result.");
+      LOG.info(stage.getId() + "'s " + partitionId + " partition has empty result.");
       return;
     }
-    SubQuery.scheduleFetches(subQuery, fetches);
+    Stage.scheduleFetches(stage, fetches);
   }
 
   /**
@@ -616,14 +616,14 @@ public class Repartitioner {
   }
 
   public static void scheduleFragmentsForNonLeafTasks(TaskSchedulerContext schedulerContext,
-                                                      MasterPlan masterPlan, SubQuery subQuery, int maxNum)
+                                                      MasterPlan masterPlan, Stage stage, int maxNum)
       throws IOException {
-    DataChannel channel = masterPlan.getIncomingChannels(subQuery.getBlock().getId()).get(0);
+    DataChannel channel = masterPlan.getIncomingChannels(stage.getBlock().getId()).get(0);
     if (channel.getShuffleType() == HASH_SHUFFLE
         || channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
-      scheduleHashShuffledFetches(schedulerContext, masterPlan, subQuery, channel, maxNum);
+      scheduleHashShuffledFetches(schedulerContext, masterPlan, stage, channel, maxNum);
     } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
-      scheduleRangeShuffledFetches(schedulerContext, masterPlan, subQuery, channel, maxNum);
+      scheduleRangeShuffledFetches(schedulerContext, masterPlan, stage, channel, maxNum);
     } else {
       throw new InternalException("Cannot support partition type");
     }
@@ -634,22 +634,22 @@ public class Repartitioner {
     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());
+      Stage childStage = context.getStage(childBlock.getId());
+      tableStatses.add(childStage.getResultStats());
     }
     return StatisticsUtil.aggregateTableStat(tableStatses);
   }
 
   public static void scheduleRangeShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
-                                                  SubQuery subQuery, DataChannel channel, int maxNum)
+                                                  Stage stage, DataChannel channel, int maxNum)
       throws IOException {
-    ExecutionBlock execBlock = subQuery.getBlock();
+    ExecutionBlock execBlock = stage.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
     Path tablePath;
-    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf()))
+    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf()))
         .getTablePath(scan.getTableName());
 
-    ExecutionBlock sampleChildBlock = masterPlan.getChild(subQuery.getId(), 0);
+    ExecutionBlock sampleChildBlock = masterPlan.getChild(stage.getId(), 0);
     SortNode sortNode = PlannerUtil.findTopNode(sampleChildBlock.getPlan(), NodeType.SORT);
     SortSpec [] sortSpecs = sortNode.getSortKeys();
     Schema sortSchema = new Schema(channel.getShuffleKeys());
@@ -658,7 +658,7 @@ public class Repartitioner {
     int determinedTaskNum;
 
     // calculate the number of maximum query ranges
-    TableStats totalStat = computeChildBlocksStats(subQuery.getContext(), masterPlan, subQuery.getId());
+    TableStats totalStat = computeChildBlocksStats(stage.getContext(), masterPlan, stage.getId());
 
     // If there is an empty table in inner join, it should return zero rows.
     if (totalStat.getNumBytes() == 0 && totalStat.getColumnStats().size() == 0) {
@@ -668,15 +668,15 @@ public class Repartitioner {
 
     if (sortNode.getSortPurpose() == SortPurpose.STORAGE_SPECIFIED) {
       StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
-      CatalogService catalog = subQuery.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
+      CatalogService catalog = stage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
       LogicalRootNode rootNode = masterPlan.getLogicalPlan().getRootBlock().getRoot();
       TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
       if (tableDesc == null) {
         throw new IOException("Can't get table meta data from catalog: " +
             PlannerUtil.getStoreTableName(masterPlan.getLogicalPlan()));
       }
-      ranges = StorageManager.getStorageManager(subQuery.getContext().getConf(), storeType)
-          .getInsertSortRanges(subQuery.getContext().getQueryContext(), tableDesc,
+      ranges = StorageManager.getStorageManager(stage.getContext().getConf(), storeType)
+          .getInsertSortRanges(stage.getContext().getQueryContext(), tableDesc,
               sortNode.getInSchema(), sortSpecs,
               mergedRange);
       determinedTaskNum = ranges.length;
@@ -687,36 +687,36 @@ public class Repartitioner {
       // 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.
       if (card.compareTo(BigInteger.valueOf(maxNum)) < 0) {
-        LOG.info(subQuery.getId() + ", The range cardinality (" + card
+        LOG.info(stage.getId() + ", The range cardinality (" + card
             + ") is less then the desired number of tasks (" + maxNum + ")");
         determinedTaskNum = card.intValue();
       } else {
         determinedTaskNum = maxNum;
       }
 
-      LOG.info(subQuery.getId() + ", Try to divide " + mergedRange + " into " + determinedTaskNum +
+      LOG.info(stage.getId() + ", Try to divide " + mergedRange + " into " + determinedTaskNum +
           " sub ranges (total units: " + determinedTaskNum + ")");
       ranges = partitioner.partition(determinedTaskNum);
       if (ranges == null || ranges.length == 0) {
-        LOG.warn(subQuery.getId() + " no range infos.");
+        LOG.warn(stage.getId() + " no range infos.");
       }
       TupleUtil.setMaxRangeIfNull(sortSpecs, sortSchema, totalStat.getColumnStats(), ranges);
       if (LOG.isDebugEnabled()) {
         if (ranges != null) {
           for (TupleRange eachRange : ranges) {
-            LOG.debug(subQuery.getId() + " range: " + eachRange.getStart() + " ~ " + eachRange.getEnd());
+            LOG.debug(stage.getId() + " range: " + eachRange.getStart() + " ~ " + eachRange.getEnd());
           }
         }
       }
     }
 
     FileFragment dummyFragment = new FileFragment(scan.getTableName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
-    SubQuery.scheduleFragment(subQuery, dummyFragment);
+    Stage.scheduleFragment(stage, dummyFragment);
 
     List<FetchImpl> fetches = new ArrayList<FetchImpl>();
-    List<ExecutionBlock> childBlocks = masterPlan.getChilds(subQuery.getId());
+    List<ExecutionBlock> childBlocks = masterPlan.getChilds(stage.getId());
     for (ExecutionBlock childBlock : childBlocks) {
-      SubQuery childExecSM = subQuery.getContext().getSubQuery(childBlock.getId());
+      Stage childExecSM = stage.getContext().getStage(childBlock.getId());
       for (Task qu : childExecSM.getTasks()) {
         for (IntermediateEntry p : qu.getIntermediateData()) {
           FetchImpl fetch = new FetchImpl(p.getPullHost(), RANGE_SHUFFLE, childBlock.getId(), 0);
@@ -758,12 +758,12 @@ public class Repartitioner {
       LOG.error(e);
     }
 
-    scheduleFetchesByRoundRobin(subQuery, map, scan.getTableName(), determinedTaskNum);
+    scheduleFetchesByRoundRobin(stage, map, scan.getTableName(), determinedTaskNum);
 
     schedulerContext.setEstimatedTaskNum(determinedTaskNum);
   }
 
-  public static void scheduleFetchesByRoundRobin(SubQuery subQuery, Map<?, Collection<FetchImpl>> partitions,
+  public static void scheduleFetchesByRoundRobin(Stage stage, Map<?, Collection<FetchImpl>> partitions,
                                                    String tableName, int num) {
     int i;
     Map<String, List<FetchImpl>>[] fetchesArray = new Map[num];
@@ -777,7 +777,7 @@ public class Repartitioner {
       if (i == num) i = 0;
     }
     for (Map<String, List<FetchImpl>> eachFetches : fetchesArray) {
-      SubQuery.scheduleFetches(subQuery, eachFetches);
+      Stage.scheduleFetches(stage, eachFetches);
     }
   }
 
@@ -807,18 +807,18 @@ public class Repartitioner {
   }
 
   public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
-                                                 SubQuery subQuery, DataChannel channel,
+                                                 Stage stage, DataChannel channel,
                                                  int maxNum) throws IOException {
-    ExecutionBlock execBlock = subQuery.getBlock();
+    ExecutionBlock execBlock = stage.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
     Path tablePath;
-    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf()))
+    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf()))
         .getTablePath(scan.getTableName());
 
     Fragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
     List<Fragment> fragments = new ArrayList<Fragment>();
     fragments.add(frag);
-    SubQuery.scheduleFragments(subQuery, fragments);
+    Stage.scheduleFragments(stage, fragments);
 
     Map<Integer, FetchGroupMeta> finalFetches = new HashMap<Integer, FetchGroupMeta>();
     Map<ExecutionBlockId, List<IntermediateEntry>> intermediates = new HashMap<ExecutionBlockId,
@@ -826,7 +826,7 @@ public class Repartitioner {
 
     for (ExecutionBlock block : masterPlan.getChilds(execBlock)) {
       List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
-      partitions.addAll(subQuery.getContext().getSubQuery(block.getId()).getHashShuffleIntermediateEntries());
+      partitions.addAll(stage.getContext().getStage(block.getId()).getHashShuffleIntermediateEntries());
 
       // In scattered hash shuffle, Collecting each IntermediateEntry
       if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
@@ -861,16 +861,16 @@ public class Repartitioner {
     }
 
     int groupingColumns = 0;
-    LogicalNode[] groupbyNodes = PlannerUtil.findAllNodes(subQuery.getBlock().getPlan(),
+    LogicalNode[] groupbyNodes = PlannerUtil.findAllNodes(stage.getBlock().getPlan(),
         new NodeType[]{NodeType.GROUP_BY, NodeType.DISTINCT_GROUP_BY});
     if (groupbyNodes != null && groupbyNodes.length > 0) {
       LogicalNode bottomNode = groupbyNodes[0];
       if (bottomNode.getType() == NodeType.GROUP_BY) {
         groupingColumns = ((GroupbyNode)bottomNode).getGroupingColumns().length;
       } else if (bottomNode.getType() == NodeType.DISTINCT_GROUP_BY) {
-        DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(subQuery.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
+        DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(stage.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
         if (distinctNode == null) {
-          LOG.warn(subQuery.getId() + ", Can't find current DistinctGroupbyNode");
+          LOG.warn(stage.getId() + ", Can't find current DistinctGroupbyNode");
           distinctNode = (DistinctGroupbyNode)bottomNode;
         }
         groupingColumns = distinctNode.getGroupingColumns().length;
@@ -879,8 +879,8 @@ public class Repartitioner {
         EnforceProperty property = PhysicalPlannerImpl.getAlgorithmEnforceProperty(enforcer, distinctNode);
         if (property != null) {
           if (property.getDistinct().getIsMultipleAggregation()) {
-            MultipleAggregationStage stage = property.getDistinct().getMultipleAggregationStage();
-            if (stage != MultipleAggregationStage.THRID_STAGE) {
+            MultipleAggregationStage mulAggStage = property.getDistinct().getMultipleAggregationStage();
+            if (mulAggStage != MultipleAggregationStage.THRID_STAGE) {
               groupingColumns = distinctNode.getOutSchema().size();
             }
           }
@@ -889,13 +889,13 @@ public class Repartitioner {
     }
     // get a proper number of tasks
     int determinedTaskNum = Math.min(maxNum, finalFetches.size());
-    LOG.info(subQuery.getId() + ", ScheduleHashShuffledFetches - Max num=" + maxNum + ", finalFetchURI=" + finalFetches.size());
+    LOG.info(stage.getId() + ", ScheduleHashShuffledFetches - Max num=" + maxNum + ", finalFetchURI=" + finalFetches.size());
 
     if (groupingColumns == 0) {
       determinedTaskNum = 1;
-      LOG.info(subQuery.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
+      LOG.info(stage.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
     } else {
-      TableStats totalStat = computeChildBlocksStats(subQuery.getContext(), masterPlan, subQuery.getId());
+      TableStats totalStat = computeChildBlocksStats(stage.getContext(), masterPlan, stage.getId());
       if (totalStat.getNumRows() == 0) {
         determinedTaskNum = 1;
       }
@@ -903,13 +903,13 @@ public class Repartitioner {
 
     // set the proper number of tasks to the estimated task num
     if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
-      scheduleScatteredHashShuffleFetches(schedulerContext, subQuery, intermediates,
+      scheduleScatteredHashShuffleFetches(schedulerContext, stage, intermediates,
           scan.getTableName());
     } else {
       schedulerContext.setEstimatedTaskNum(determinedTaskNum);
       // divide fetch uris into the the proper number of tasks according to volumes
-      scheduleFetchesByEvenDistributedVolumes(subQuery, finalFetches, scan.getTableName(), determinedTaskNum);
-      LOG.info(subQuery.getId() + ", DeterminedTaskNum : " + determinedTaskNum);
+      scheduleFetchesByEvenDistributedVolumes(stage, finalFetches, scan.getTableName(), determinedTaskNum);
+      LOG.info(stage.getId() + ", DeterminedTaskNum : " + determinedTaskNum);
     }
   }
 
@@ -970,12 +970,12 @@ public class Repartitioner {
     return new Pair<Long[], Map<String, List<FetchImpl>>[]>(assignedVolumes, fetchesArray);
   }
 
-  public static void scheduleFetchesByEvenDistributedVolumes(SubQuery subQuery, Map<Integer, FetchGroupMeta> partitions,
+  public static void scheduleFetchesByEvenDistributedVolumes(Stage stage, Map<Integer, FetchGroupMeta> partitions,
                                                              String tableName, int num) {
     Map<String, List<FetchImpl>>[] fetchsArray = makeEvenDistributedFetchImpl(partitions, tableName, num).getSecond();
     // Schedule FetchImpls
     for (Map<String, List<FetchImpl>> eachFetches : fetchsArray) {
-      SubQuery.scheduleFetches(subQuery, eachFetches);
+      Stage.scheduleFetches(stage, eachFetches);
     }
   }
 
@@ -987,12 +987,12 @@ public class Repartitioner {
   // to $DIST_QUERY_TABLE_PARTITION_VOLUME. Finally, each subgroup is assigned to a query unit.
   // It is usually used for writing partitioned tables.
   public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext schedulerContext,
-       SubQuery subQuery, Map<ExecutionBlockId, List<IntermediateEntry>> intermediates,
+       Stage stage, Map<ExecutionBlockId, List<IntermediateEntry>> intermediates,
        String tableName) {
     long splitVolume = StorageUnit.MB *
-        subQuery.getMasterPlan().getContext().getLong(SessionVars.TABLE_PARTITION_PER_SHUFFLE_SIZE);
+        stage.getMasterPlan().getContext().getLong(SessionVars.TABLE_PARTITION_PER_SHUFFLE_SIZE);
     long pageSize = StorageUnit.MB * 
-        subQuery.getContext().getConf().getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME); // in bytes
+        stage.getContext().getConf().getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME); // in bytes
     if (pageSize >= splitVolume) {
       throw new RuntimeException("tajo.dist-query.table-partition.task-volume-mb should be great than " +
           "tajo.shuffle.hash.appender.page.volumn-mb");
@@ -1033,11 +1033,11 @@ public class Repartitioner {
       fetchesArray[i] = new HashMap<String, List<FetchImpl>>();
       fetchesArray[i].put(tableName, entry);
 
-      SubQuery.scheduleFetches(subQuery, fetchesArray[i]);
+      Stage.scheduleFetches(stage, fetchesArray[i]);
       i++;
     }
 
-    LOG.info(subQuery.getId()
+    LOG.info(stage.getId()
         + ", ShuffleType:" + SCATTERED_HASH_SHUFFLE.name()
         + ", Intermediate Size: " + totalIntermediateSize
         + ", splitSize: " + splitVolume
@@ -1207,16 +1207,16 @@ public class Repartitioner {
     return hashed;
   }
 
-  public static SubQuery setShuffleOutputNumForTwoPhase(SubQuery subQuery, final int desiredNum, DataChannel channel) {
-    ExecutionBlock execBlock = subQuery.getBlock();
+  public static Stage setShuffleOutputNumForTwoPhase(Stage stage, final int desiredNum, DataChannel channel) {
+    ExecutionBlock execBlock = stage.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();
+    MasterPlan masterPlan = stage.getMasterPlan();
     keys = channel.getShuffleKeys();
-    if (!masterPlan.isRoot(subQuery.getBlock()) ) {
-      ExecutionBlock parentBlock = masterPlan.getParent(subQuery.getBlock());
+    if (!masterPlan.isRoot(stage.getBlock()) ) {
+      ExecutionBlock parentBlock = masterPlan.getParent(stage.getBlock());
       if (parentBlock.getPlan().getType() == NodeType.JOIN) {
         channel.setShuffleOutputNum(desiredNum);
       }
@@ -1246,6 +1246,6 @@ public class Repartitioner {
         channel.setShuffleOutputNum(desiredNum);
       }
     }
-    return subQuery;
+    return stage;
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java
new file mode 100644
index 0000000..e421417
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java
@@ -0,0 +1,1342 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.*;
+import org.apache.tajo.catalog.CatalogUtil;
+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.StoreType;
+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.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+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.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage;
+import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
+import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto;
+import org.apache.tajo.master.*;
+import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
+import org.apache.tajo.master.querymaster.Task.IntermediateEntry;
+import org.apache.tajo.master.container.TajoContainer;
+import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.storage.FileStorageManager;
+import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.unit.StorageUnit;
+import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.util.history.TaskHistory;
+import org.apache.tajo.util.history.StageHistory;
+import org.apache.tajo.worker.FetchImpl;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+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.plan.serder.PlanProto.ShuffleType;
+
+
+/**
+ * Stage plays a role in controlling an ExecutionBlock and is a finite state machine.
+ */
+public class Stage implements EventHandler<StageEvent> {
+
+  private static final Log LOG = LogFactory.getLog(Stage.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 AbstractTaskScheduler taskScheduler;
+  private QueryMasterTask.QueryMasterTaskContext context;
+  private final List<String> diagnostics = new ArrayList<String>();
+  private StageState stageState;
+
+  private long startTime;
+  private long finishTime;
+
+  volatile Map<TaskId, Task> tasks = new ConcurrentHashMap<TaskId, Task>();
+  volatile Map<TajoContainerId, TajoContainer> containers = new ConcurrentHashMap<TajoContainerId,
+    TajoContainer>();
+
+  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 StageCompleteTransition STAGE_COMPLETED_TRANSITION = new StageCompleteTransition();
+  private StateMachine<StageState, StageEventType, StageEvent> stateMachine;
+
+  protected static final StateMachineFactory<Stage, StageState,
+      StageEventType, StageEvent> stateMachineFactory =
+      new StateMachineFactory <Stage, StageState,
+          StageEventType, StageEvent> (StageState.NEW)
+
+          // Transitions from NEW state
+          .addTransition(StageState.NEW,
+              EnumSet.of(StageState.INITED, StageState.ERROR, StageState.SUCCEEDED),
+              StageEventType.SQ_INIT,
+              new InitAndRequestContainer())
+          .addTransition(StageState.NEW, StageState.NEW,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(StageState.NEW, StageState.KILLED,
+              StageEventType.SQ_KILL)
+          .addTransition(StageState.NEW, StageState.ERROR,
+              StageEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from INITED state
+          .addTransition(StageState.INITED, StageState.RUNNING,
+              StageEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINER_LAUNCH_TRANSITION)
+          .addTransition(StageState.INITED, StageState.INITED,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(StageState.INITED, StageState.KILL_WAIT,
+              StageEventType.SQ_KILL, new KillTasksTransition())
+          .addTransition(StageState.INITED, StageState.ERROR,
+              StageEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from RUNNING state
+          .addTransition(StageState.RUNNING, StageState.RUNNING,
+              StageEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINER_LAUNCH_TRANSITION)
+          .addTransition(StageState.RUNNING, StageState.RUNNING,
+              StageEventType.SQ_TASK_COMPLETED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(StageState.RUNNING,
+              EnumSet.of(StageState.SUCCEEDED, StageState.FAILED),
+              StageEventType.SQ_STAGE_COMPLETED,
+              STAGE_COMPLETED_TRANSITION)
+          .addTransition(StageState.RUNNING, StageState.RUNNING,
+              StageEventType.SQ_FAILED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(StageState.RUNNING, StageState.RUNNING,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(StageState.RUNNING, StageState.KILL_WAIT,
+              StageEventType.SQ_KILL,
+              new KillTasksTransition())
+          .addTransition(StageState.RUNNING, StageState.ERROR,
+              StageEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+          // Ignore-able Transition
+          .addTransition(StageState.RUNNING, StageState.RUNNING,
+              StageEventType.SQ_START)
+
+          // Transitions from KILL_WAIT state
+          .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+              StageEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+              EnumSet.of(StageEventType.SQ_KILL), new KillTasksTransition())
+          .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+              StageEventType.SQ_TASK_COMPLETED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(StageState.KILL_WAIT,
+              EnumSet.of(StageState.SUCCEEDED, StageState.FAILED, StageState.KILLED),
+              StageEventType.SQ_STAGE_COMPLETED,
+              STAGE_COMPLETED_TRANSITION)
+          .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+              StageEventType.SQ_FAILED,
+              TASK_COMPLETED_TRANSITION)
+          .addTransition(StageState.KILL_WAIT, StageState.ERROR,
+              StageEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+              // Transitions from SUCCEEDED state
+          .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
+              StageEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(StageState.SUCCEEDED, StageState.ERROR,
+              StageEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+              // Ignore-able events
+          .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
+              EnumSet.of(
+                  StageEventType.SQ_START,
+                  StageEventType.SQ_KILL,
+                  StageEventType.SQ_CONTAINER_ALLOCATED))
+
+          // Transitions from KILLED state
+          .addTransition(StageState.KILLED, StageState.KILLED,
+              StageEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(StageState.KILLED, StageState.KILLED,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(StageState.KILLED, StageState.ERROR,
+              StageEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+              // Ignore-able transitions
+          .addTransition(StageState.KILLED, StageState.KILLED,
+              EnumSet.of(
+                  StageEventType.SQ_START,
+                  StageEventType.SQ_KILL,
+                  StageEventType.SQ_CONTAINER_ALLOCATED,
+                  StageEventType.SQ_FAILED))
+
+          // Transitions from FAILED state
+          .addTransition(StageState.FAILED, StageState.FAILED,
+              StageEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(StageState.FAILED, StageState.FAILED,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(StageState.FAILED, StageState.ERROR,
+              StageEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+          // Ignore-able transitions
+          .addTransition(StageState.FAILED, StageState.FAILED,
+              EnumSet.of(
+                  StageEventType.SQ_START,
+                  StageEventType.SQ_KILL,
+                  StageEventType.SQ_CONTAINER_ALLOCATED,
+                  StageEventType.SQ_FAILED))
+
+          // Transitions from ERROR state
+          .addTransition(StageState.ERROR, StageState.ERROR,
+              StageEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINERS_CANCEL_TRANSITION)
+          .addTransition(StageState.ERROR, StageState.ERROR,
+              StageEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          // Ignore-able transitions
+          .addTransition(StageState.ERROR, StageState.ERROR,
+              EnumSet.of(
+                  StageEventType.SQ_START,
+                  StageEventType.SQ_KILL,
+                  StageEventType.SQ_FAILED,
+                  StageEventType.SQ_INTERNAL_ERROR,
+                  StageEventType.SQ_STAGE_COMPLETED))
+
+          .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;
+  private List<IntermediateEntry> hashShuffleIntermediateEntries = new ArrayList<IntermediateEntry>();
+  private AtomicInteger completeReportReceived = new AtomicInteger(0);
+  private StageHistory finalStageHistory;
+
+  public Stage(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block) {
+    this.context = context;
+    this.masterPlan = masterPlan;
+    this.block = block;
+    this.eventHandler = context.getEventHandler();
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+    stateMachine = stateMachineFactory.make(this);
+    stageState = stateMachine.getCurrentState();
+  }
+
+  public static boolean isRunningState(StageState state) {
+    return state == StageState.INITED || state == StageState.NEW || state == StageState.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() == StageState.NEW) {
+        return 0;
+      } else {
+        return (float)(succeededObjectCount) / (float)totalScheduledObjectsCount;
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public float getProgress() {
+    List<Task> tempTasks = null;
+    readLock.lock();
+    try {
+      if (getState() == StageState.NEW) {
+        return 0.0f;
+      } else {
+        tempTasks = new ArrayList<Task>(tasks.values());
+      }
+    } finally {
+      readLock.unlock();
+    }
+
+    float totalProgress = 0.0f;
+    for (Task eachTask : tempTasks) {
+      if (eachTask.getLastAttempt() != null) {
+        totalProgress += eachTask.getLastAttempt().getProgress();
+      }
+    }
+
+    if (totalProgress > 0.0f) {
+      return (float) Math.floor((totalProgress / (float) Math.max(tempTasks.size(), 1)) * 1000.0f) / 1000.0f;
+    } else {
+      return 0.0f;
+    }
+  }
+
+  public int getSucceededObjectCount() {
+    return succeededObjectCount;
+  }
+
+  public int getTotalScheduledObjectsCount() {
+    return totalScheduledObjectsCount;
+  }
+
+  public ExecutionBlock getBlock() {
+    return block;
+  }
+
+  public void addTask(Task task) {
+    tasks.put(task.getId(), task);
+  }
+
+  public StageHistory getStageHistory() {
+    if (finalStageHistory != null) {
+      if (finalStageHistory.getFinishTime() == 0) {
+        finalStageHistory = makeStageHistory();
+        finalStageHistory.setTasks(makeTaskHistories());
+      }
+      return finalStageHistory;
+    } else {
+      return makeStageHistory();
+    }
+  }
+
+  private List<TaskHistory> makeTaskHistories() {
+    List<TaskHistory> taskHistories = new ArrayList<TaskHistory>();
+
+    for(Task eachTask : getTasks()) {
+      taskHistories.add(eachTask.getTaskHistory());
+    }
+
+    return taskHistories;
+  }
+
+  private StageHistory makeStageHistory() {
+    StageHistory stageHistory = new StageHistory();
+
+    stageHistory.setExecutionBlockId(getId().toString());
+    stageHistory.setPlan(PlannerUtil.buildExplainString(block.getPlan()));
+    stageHistory.setState(getState().toString());
+    stageHistory.setStartTime(startTime);
+    stageHistory.setFinishTime(finishTime);
+    stageHistory.setSucceededObjectCount(succeededObjectCount);
+    stageHistory.setKilledObjectCount(killedObjectCount);
+    stageHistory.setFailedObjectCount(failedObjectCount);
+    stageHistory.setTotalScheduledObjectsCount(totalScheduledObjectsCount);
+    stageHistory.setHostLocalAssigned(getTaskScheduler().getHostLocalAssigned());
+    stageHistory.setRackLocalAssigned(getTaskScheduler().getRackLocalAssigned());
+
+    long totalInputBytes = 0;
+    long totalReadBytes = 0;
+    long totalReadRows = 0;
+    long totalWriteBytes = 0;
+    long totalWriteRows = 0;
+    int numShuffles = 0;
+    for(Task eachTask : getTasks()) {
+      numShuffles = eachTask.getShuffleOutpuNum();
+      if (eachTask.getLastAttempt() != null) {
+        TableStats inputStats = eachTask.getLastAttempt().getInputStats();
+        if (inputStats != null) {
+          totalInputBytes += inputStats.getNumBytes();
+          totalReadBytes += inputStats.getReadBytes();
+          totalReadRows += inputStats.getNumRows();
+        }
+        TableStats outputStats = eachTask.getLastAttempt().getResultStats();
+        if (outputStats != null) {
+          totalWriteBytes += outputStats.getNumBytes();
+          totalWriteRows += outputStats.getNumRows();
+        }
+      }
+    }
+
+    stageHistory.setTotalInputBytes(totalInputBytes);
+    stageHistory.setTotalReadBytes(totalReadBytes);
+    stageHistory.setTotalReadRows(totalReadRows);
+    stageHistory.setTotalWriteBytes(totalWriteBytes);
+    stageHistory.setTotalWriteRows(totalWriteRows);
+    stageHistory.setNumShuffles(numShuffles);
+    stageHistory.setProgress(getProgress());
+    return stageHistory;
+  }
+
+  /**
+   * It finalizes this stage. It is only invoked when the stage is succeeded.
+   */
+  public void complete() {
+    cleanup();
+    finalizeStats();
+    setFinishTime();
+    eventHandler.handle(new StageCompletedEvent(getId(), StageState.SUCCEEDED));
+  }
+
+  /**
+   * It finalizes this stage. Unlike {@link Stage#complete()},
+   * it is invoked when a stage is abnormally finished.
+   *
+   * @param finalState The final stage state
+   */
+  public void abort(StageState finalState) {
+    // TODO -
+    // - committer.abortStage(...)
+    // - record Stage Finish Time
+    // - CleanUp Tasks
+    // - Record History
+    cleanup();
+    setFinishTime();
+    eventHandler.handle(new StageCompletedEvent(getId(), finalState));
+  }
+
+  public StateMachine<StageState, StageEventType, StageEvent> getStateMachine() {
+    return this.stateMachine;
+  }
+
+  public void setPriority(int priority) {
+    this.priority = priority;
+  }
+
+
+  public int getPriority() {
+    return this.priority;
+  }
+
+  public ExecutionBlockId getId() {
+    return block.getId();
+  }
+  
+  public Task[] getTasks() {
+    return tasks.values().toArray(new Task[tasks.size()]);
+  }
+  
+  public Task getTask(TaskId 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 Stage) {
+      Stage other = (Stage)o;
+      return getId().equals(other.getId());
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    return getId().hashCode();
+  }
+  
+  public int compareTo(Stage other) {
+    return getId().compareTo(other.getId());
+  }
+
+  public StageState getSynchronizedState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /* non-blocking call for client API */
+  public StageState getState() {
+    return stageState;
+  }
+
+  public static TableStats[] computeStatFromUnionBlock(Stage stage) {
+    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 = stage.getMasterPlan();
+    Iterator<ExecutionBlock> it = masterPlan.getChilds(stage.getBlock()).iterator();
+    while (it.hasNext()) {
+      ExecutionBlock block = it.next();
+      Stage childStage = stage.context.getStage(block.getId());
+      TableStats[] childStatArray = new TableStats[]{
+          childStage.getInputStats(), childStage.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 (Task unit : getTasks()) {
+      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()));
+  }
+
+  /**
+   * 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);
+
+    // if store plan (i.e., CREATE or INSERT OVERWRITE)
+    StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
+    if (storeType == null) {
+      // get default or store type
+      storeType = StoreType.CSV;
+    }
+
+    schema = channel.getSchema();
+    meta = CatalogUtil.newTableMeta(storeType, new KeyValueSet());
+    inputStatistics = statsArray[0];
+    resultStatistics = statsArray[1];
+  }
+
+  @Override
+  public void handle(StageEvent event) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getStageId() + " of type " + event.getType() + ", preState="
+          + getSynchronizedState());
+    }
+
+    try {
+      writeLock.lock();
+      StageState oldState = getSynchronizedState();
+      try {
+        getStateMachine().doTransition(event.getType(), event);
+        stageState = getSynchronizedState();
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state"
+            + ", eventType:" + event.getType().name()
+            + ", oldState:" + oldState.name()
+            + ", nextState:" + getSynchronizedState().name()
+            , e);
+        eventHandler.handle(new StageEvent(getId(),
+            StageEventType.SQ_INTERNAL_ERROR));
+      }
+
+      // notify the eventhandler of state change
+      if (LOG.isDebugEnabled()) {
+        if (oldState != getSynchronizedState()) {
+          LOG.debug(getId() + " Stage Transitioned from " + oldState + " to "
+              + getSynchronizedState());
+        }
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+    taskScheduler.handleTaskRequestEvent(event);
+  }
+
+  private static class InitAndRequestContainer implements MultipleArcTransition<Stage,
+      StageEvent, StageState> {
+
+    @Override
+    public StageState transition(final Stage stage, StageEvent stageEvent) {
+      stage.setStartTime();
+      ExecutionBlock execBlock = stage.getBlock();
+      StageState state;
+
+      try {
+        // Union operator does not require actual query processing. It is performed logically.
+        if (execBlock.hasUnion()) {
+          stage.finalizeStats();
+          state = StageState.SUCCEEDED;
+        } else {
+          // execute pre-processing asyncronously
+          stage.getContext().getQueryMasterContext().getEventExecutor()
+              .submit(new Runnable() {
+                        @Override
+                        public void run() {
+                          try {
+                            ExecutionBlock parent = stage.getMasterPlan().getParent(stage.getBlock());
+                            DataChannel channel = stage.getMasterPlan().getChannel(stage.getId(), parent.getId());
+                            setShuffleIfNecessary(stage, channel);
+                            initTaskScheduler(stage);
+                            schedule(stage);
+                            stage.totalScheduledObjectsCount = stage.getTaskScheduler().remainingScheduledObjectNum();
+                            LOG.info(stage.totalScheduledObjectsCount + " objects are scheduled");
+
+                            if (stage.getTaskScheduler().remainingScheduledObjectNum() == 0) { // if there is no tasks
+                              stage.complete();
+                            } else {
+                              if(stage.getSynchronizedState() == StageState.INITED) {
+                                stage.taskScheduler.start();
+                                allocateContainers(stage);
+                              } else {
+                                stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
+                              }
+                            }
+                          } catch (Throwable e) {
+                            LOG.error("Stage (" + stage.getId() + ") ERROR: ", e);
+                            stage.setFinishTime();
+                            stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), e.getMessage()));
+                            stage.eventHandler.handle(new StageCompletedEvent(stage.getId(), StageState.ERROR));
+                          }
+                        }
+                      }
+              );
+          state = StageState.INITED;
+        }
+      } catch (Throwable e) {
+        LOG.error("Stage (" + stage.getId() + ") ERROR: ", e);
+        stage.setFinishTime();
+        stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), e.getMessage()));
+        stage.eventHandler.handle(new StageCompletedEvent(stage.getId(), StageState.ERROR));
+        return StageState.ERROR;
+      }
+
+      return state;
+    }
+
+    private void initTaskScheduler(Stage stage) throws IOException {
+      TajoConf conf = stage.context.getConf();
+      stage.schedulerContext = new TaskSchedulerContext(stage.context,
+          stage.getMasterPlan().isLeaf(stage.getId()), stage.getId());
+      stage.taskScheduler = TaskSchedulerFactory.get(conf, stage.schedulerContext, stage);
+      stage.taskScheduler.init(conf);
+      LOG.info(stage.taskScheduler.getName() + " is chosen for the task scheduling for " + stage.getId());
+    }
+
+    /**
+     * If a parent block requires a repartition operation, the method sets proper repartition
+     * methods and the number of partitions to a given Stage.
+     */
+    private static void setShuffleIfNecessary(Stage stage, DataChannel channel) {
+      if (channel.getShuffleType() != ShuffleType.NONE_SHUFFLE) {
+        int numTasks = calculateShuffleOutputNum(stage, channel);
+        Repartitioner.setShuffleOutputNumForTwoPhase(stage, numTasks, channel);
+      }
+    }
+
+    /**
+     * Getting the total memory of cluster
+     *
+     * @param stage
+     * @return mega bytes
+     */
+    private static int getClusterTotalMemory(Stage stage) {
+      List<TajoMasterProtocol.WorkerResourceProto> workers =
+          stage.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 stage
+     * @return
+     */
+    public static int calculateShuffleOutputNum(Stage stage, DataChannel channel) {
+      TajoConf conf = stage.context.getConf();
+      MasterPlan masterPlan = stage.getMasterPlan();
+      ExecutionBlock parent = masterPlan.getParent(stage.getBlock());
+
+      LogicalNode grpNode = null;
+      if (parent != null) {
+        grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.GROUP_BY);
+        if (grpNode == null) {
+          grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.DISTINCT_GROUP_BY);
+        }
+      }
+
+      // We assume this execution block the first stage of join if two or more tables are included in this block,
+      if (parent != null && parent.getScanNodes().length >= 2) {
+        List<ExecutionBlock> childs = masterPlan.getChilds(parent);
+
+        // for outer
+        ExecutionBlock outer = childs.get(0);
+        long outerVolume = getInputVolume(stage.masterPlan, stage.context, outer);
+
+        // for inner
+        ExecutionBlock inner = childs.get(1);
+        long innerVolume = getInputVolume(stage.masterPlan, stage.context, inner);
+        LOG.info(stage.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(stage.getId() + ", Bigger Table's volume is approximately " + mb + " MB");
+
+        int taskNum = (int) Math.ceil((double) mb / masterPlan.getContext().getInt(SessionVars.JOIN_PER_SHUFFLE_SIZE));
+
+        if (masterPlan.getContext().containsKey(SessionVars.TEST_MIN_TASK_NUM)) {
+          taskNum = masterPlan.getContext().getInt(SessionVars.TEST_MIN_TASK_NUM);
+          LOG.warn("!!!!! TESTCASE MODE !!!!!");
+        }
+
+        // The shuffle output numbers of join may be inconsistent by execution block order.
+        // Thus, we need to compare the number with DataChannel output numbers.
+        // If the number is right, the number and DataChannel output numbers will be consistent.
+        int outerShuffleOutputNum = 0, innerShuffleOutputNum = 0;
+        for (DataChannel eachChannel : masterPlan.getOutgoingChannels(outer.getId())) {
+          outerShuffleOutputNum = Math.max(outerShuffleOutputNum, eachChannel.getShuffleOutputNum());
+        }
+        for (DataChannel eachChannel : masterPlan.getOutgoingChannels(inner.getId())) {
+          innerShuffleOutputNum = Math.max(innerShuffleOutputNum, eachChannel.getShuffleOutputNum());
+        }
+        if (outerShuffleOutputNum != innerShuffleOutputNum
+            && taskNum != outerShuffleOutputNum
+            && taskNum != innerShuffleOutputNum) {
+          LOG.info(stage.getId() + ", Change determined number of join partitions cause difference of outputNum" +
+                  ", originTaskNum=" + taskNum + ", changedTaskNum=" + Math.max(outerShuffleOutputNum, innerShuffleOutputNum) +
+                  ", outerShuffleOutptNum=" + outerShuffleOutputNum +
+                  ", innerShuffleOutputNum=" + innerShuffleOutputNum);
+          taskNum = Math.max(outerShuffleOutputNum, innerShuffleOutputNum);
+        }
+
+        LOG.info(stage.getId() + ", The determined number of join partitions is " + taskNum);
+
+        return taskNum;
+        // Is this stage the first step of group-by?
+      } else if (grpNode != null) {
+        boolean hasGroupColumns = true;
+        if (grpNode.getType() == NodeType.GROUP_BY) {
+          hasGroupColumns = ((GroupbyNode)grpNode).getGroupingColumns().length > 0;
+        } else if (grpNode.getType() == NodeType.DISTINCT_GROUP_BY) {
+          // Find current distinct stage node.
+          DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(stage.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
+          if (distinctNode == null) {
+            LOG.warn(stage.getId() + ", Can't find current DistinctGroupbyNode");
+            distinctNode = (DistinctGroupbyNode)grpNode;
+          }
+          hasGroupColumns = distinctNode.getGroupingColumns().length > 0;
+
+          Enforcer enforcer = stage.getBlock().getEnforcer();
+          if (enforcer == null) {
+            LOG.warn(stage.getId() + ", DistinctGroupbyNode's enforcer is null.");
+          }
+          EnforceProperty property = PhysicalPlannerImpl.getAlgorithmEnforceProperty(enforcer, distinctNode);
+          if (property != null) {
+            if (property.getDistinct().getIsMultipleAggregation()) {
+              MultipleAggregationStage multiAggStage = property.getDistinct().getMultipleAggregationStage();
+              if (multiAggStage != MultipleAggregationStage.THRID_STAGE) {
+                hasGroupColumns = true;
+              }
+            }
+          }
+        }
+        if (!hasGroupColumns) {
+          LOG.info(stage.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
+          return 1;
+        } else {
+          long volume = getInputVolume(stage.masterPlan, stage.context, stage.block);
+
+          int volumeByMB = (int) Math.ceil((double) volume / StorageUnit.MB);
+          LOG.info(stage.getId() + ", Table's volume is approximately " + volumeByMB + " MB");
+          // determine the number of task
+          int taskNum = (int) Math.ceil((double) volumeByMB /
+              masterPlan.getContext().getInt(SessionVars.GROUPBY_PER_SHUFFLE_SIZE));
+          LOG.info(stage.getId() + ", The determined number of aggregation partitions is " + taskNum);
+          return taskNum;
+        }
+      } else {
+        LOG.info("============>>>>> Unexpected Case! <<<<<================");
+        long volume = getInputVolume(stage.masterPlan, stage.context, stage.block);
+
+        int mb = (int) Math.ceil((double)volume / 1048576);
+        LOG.info(stage.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(stage.getId() + ", The determined number of partitions is " + taskNum);
+        return taskNum;
+      }
+    }
+
+    private static void schedule(Stage stage) throws IOException {
+      MasterPlan masterPlan = stage.getMasterPlan();
+      ExecutionBlock execBlock = stage.getBlock();
+      if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
+        scheduleFragmentsForLeafQuery(stage);
+      } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join
+        Repartitioner.scheduleFragmentsForJoinQuery(stage.schedulerContext, stage);
+      } else { // Case 3: Others (Sort or Aggregation)
+        int numTasks = getNonLeafTaskNum(stage);
+        Repartitioner.scheduleFragmentsForNonLeafTasks(stage.schedulerContext, masterPlan, stage, numTasks);
+      }
+    }
+
+    /**
+     * Getting the desire number of tasks according to the volume of input data
+     *
+     * @param stage
+     * @return
+     */
+    public static int getNonLeafTaskNum(Stage stage) {
+      // Getting intermediate data size
+      long volume = getInputVolume(stage.getMasterPlan(), stage.context, stage.getBlock());
+
+      int mb = (int) Math.ceil((double)volume / 1048576);
+      LOG.info(stage.getId() + ", 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(stage.getId() + ", 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)) {
+          Stage stage = context.getStage(childBlock.getId());
+          if (stage == null || stage.getSynchronizedState() != StageState.SUCCEEDED) {
+            aggregatedVolume += getInputVolume(masterPlan, context, childBlock);
+          } else {
+            aggregatedVolume += stage.getResultStats().getNumBytes();
+          }
+        }
+
+        return aggregatedVolume;
+      }
+    }
+
+    public static void allocateContainers(Stage stage) {
+      ExecutionBlock execBlock = stage.getBlock();
+
+      //TODO consider disk slot
+      int requiredMemoryMBPerTask = 512;
+
+      int numRequest = stage.getContext().getResourceAllocator().calculateNumRequestContainers(
+          stage.getContext().getQueryMasterContext().getWorkerContext(),
+          stage.schedulerContext.getEstimatedTaskNum(),
+          requiredMemoryMBPerTask
+      );
+
+      final Resource resource = Records.newRecord(Resource.class);
+
+      resource.setMemory(requiredMemoryMBPerTask);
+
+      LOG.info("Request Container for " + stage.getId() + " containers=" + numRequest);
+
+      Priority priority = Records.newRecord(Priority.class);
+      priority.setPriority(stage.getPriority());
+      ContainerAllocationEvent event =
+          new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ,
+              stage.getId(), priority, resource, numRequest,
+              stage.masterPlan.isLeaf(execBlock), 0.0f);
+      stage.eventHandler.handle(event);
+    }
+
+    private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException {
+      ExecutionBlock execBlock = stage.getBlock();
+      ScanNode[] scans = execBlock.getScanNodes();
+      Preconditions.checkArgument(scans.length == 1, "Must be Scan Query");
+      ScanNode scan = scans[0];
+      TableDesc table = stage.context.getTableDescMap().get(scan.getCanonicalName());
+
+      Collection<Fragment> 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) {
+        // After calling this method, partition paths are removed from the physical plan.
+        FileStorageManager storageManager =
+            (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
+        fragments = Repartitioner.getFragmentsFromPartitionedTable(storageManager, scan, table);
+      } else {
+        StorageManager storageManager =
+            StorageManager.getStorageManager(stage.getContext().getConf(), meta.getStoreType());
+        fragments = storageManager.getSplits(scan.getCanonicalName(), table, scan);
+      }
+
+      Stage.scheduleFragments(stage, fragments);
+      if (stage.getTaskScheduler() instanceof DefaultTaskScheduler) {
+        //Leaf task of DefaultTaskScheduler should be fragment size
+        // EstimatedTaskNum determined number of initial container
+        stage.schedulerContext.setEstimatedTaskNum(fragments.size());
+      } else {
+        TajoConf conf = stage.context.getConf();
+        stage.schedulerContext.setTaskSize(conf.getIntVar(ConfVars.TASK_DEFAULT_SIZE) * 1024 * 1024);
+        int estimatedTaskNum = (int) Math.ceil((double) table.getStats().getNumBytes() /
+            (double) stage.schedulerContext.getTaskSize());
+        stage.schedulerContext.setEstimatedTaskNum(estimatedTaskNum);
+      }
+    }
+  }
+
+  public static void scheduleFragment(Stage stage, Fragment fragment) {
+    stage.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+        stage.getId(), fragment));
+  }
+
+
+  public static void scheduleFragments(Stage stage, Collection<Fragment> fragments) {
+    for (Fragment eachFragment : fragments) {
+      scheduleFragment(stage, eachFragment);
+    }
+  }
+
+  public static void scheduleFragments(Stage stage, Collection<Fragment> leftFragments,
+                                       Collection<Fragment> broadcastFragments) {
+    for (Fragment eachLeafFragment : leftFragments) {
+      scheduleFragment(stage, eachLeafFragment, broadcastFragments);
+    }
+  }
+
+  public static void scheduleFragment(Stage stage,
+                                      Fragment leftFragment, Collection<Fragment> rightFragments) {
+    stage.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+        stage.getId(), leftFragment, rightFragments));
+  }
+
+  public static void scheduleFetches(Stage stage, Map<String, List<FetchImpl>> fetches) {
+    stage.taskScheduler.handle(new FetchScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+        stage.getId(), fetches));
+  }
+
+  public static Task newEmptyTask(TaskSchedulerContext schedulerContext,
+                                  TaskAttemptScheduleContext taskContext,
+                                  Stage stage, int taskId) {
+    ExecutionBlock execBlock = stage.getBlock();
+    Task unit = new Task(schedulerContext.getMasterContext().getConf(),
+        taskContext,
+        QueryIdFactory.newTaskId(schedulerContext.getBlockId(), taskId),
+        schedulerContext.isLeafQuery(), stage.eventHandler);
+    unit.setLogicalPlan(execBlock.getPlan());
+    stage.addTask(unit);
+    return unit;
+  }
+
+  private static class ContainerLaunchTransition
+      implements SingleArcTransition<Stage, StageEvent> {
+
+    @Override
+    public void transition(Stage stage, StageEvent event) {
+      try {
+        StageContainerAllocationEvent allocationEvent =
+            (StageContainerAllocationEvent) event;
+        for (TajoContainer container : allocationEvent.getAllocatedContainer()) {
+          TajoContainerId cId = container.getId();
+          if (stage.containers.containsKey(cId)) {
+            stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
+                "Duplicated containers are allocated: " + cId.toString()));
+            stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
+          }
+          stage.containers.put(cId, container);
+        }
+        LOG.info("Stage (" + stage.getId() + ") has " + stage.containers.size() + " containers!");
+        stage.eventHandler.handle(
+            new LaunchTaskRunnersEvent(stage.getId(), allocationEvent.getAllocatedContainer(),
+                stage.getContext().getQueryContext(),
+                CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class))
+        );
+
+        stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_START));
+      } catch (Throwable t) {
+        stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
+            ExceptionUtils.getStackTrace(t)));
+        stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.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<Stage, StageEvent> {
+    @Override
+    public void transition(Stage stage, StageEvent event) {
+      try {
+        StageContainerAllocationEvent allocationEvent =
+            (StageContainerAllocationEvent) event;
+        stage.eventHandler.handle(
+            new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP,
+                stage.getId(), allocationEvent.getAllocatedContainer()));
+        LOG.info(String.format("[%s] %d allocated containers are canceled",
+            stage.getId().toString(),
+            allocationEvent.getAllocatedContainer().size()));
+      } catch (Throwable t) {
+        stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
+            ExceptionUtils.getStackTrace(t)));
+        stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
+      }
+    }
+  }
+
+  private static class TaskCompletedTransition implements SingleArcTransition<Stage, StageEvent> {
+
+    @Override
+    public void transition(Stage stage,
+                           StageEvent event) {
+      StageTaskEvent taskEvent = (StageTaskEvent) event;
+      Task task = stage.getTask(taskEvent.getTaskId());
+
+      if (task == null) { // task failed
+        LOG.error(String.format("Task %s is absent", taskEvent.getTaskId()));
+        stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_FAILED));
+      } else {
+        stage.completedTaskCount++;
+
+        if (taskEvent.getState() == TaskState.SUCCEEDED) {
+          stage.succeededObjectCount++;
+        } else if (task.getState() == TaskState.KILLED) {
+          stage.killedObjectCount++;
+        } else if (task.getState() == TaskState.FAILED) {
+          stage.failedObjectCount++;
+          // if at least one task is failed, try to kill all tasks.
+          stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
+        }
+
+        LOG.info(String.format("[%s] Task Completion Event (Total: %d, Success: %d, Killed: %d, Failed: %d)",
+            stage.getId(),
+            stage.getTotalScheduledObjectsCount(),
+            stage.succeededObjectCount,
+            stage.killedObjectCount,
+            stage.failedObjectCount));
+
+        if (stage.totalScheduledObjectsCount ==
+            stage.succeededObjectCount + stage.killedObjectCount + stage.failedObjectCount) {
+          stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_STAGE_COMPLETED));
+        }
+      }
+    }
+  }
+
+  private static class KillTasksTransition implements SingleArcTransition<Stage, StageEvent> {
+
+    @Override
+    public void transition(Stage stage, StageEvent stageEvent) {
+      if(stage.getTaskScheduler() != null){
+        stage.getTaskScheduler().stop();
+      }
+
+      for (Task task : stage.getTasks()) {
+        stage.eventHandler.handle(new TaskEvent(task.getId(), TaskEventType.T_KILL));
+      }
+    }
+  }
+
+  private void cleanup() {
+    stopScheduler();
+    releaseContainers();
+
+    if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) {
+      List<ExecutionBlock> childs = getMasterPlan().getChilds(getId());
+      List<TajoIdProtos.ExecutionBlockIdProto> ebIds = Lists.newArrayList();
+
+      for (ExecutionBlock executionBlock : childs) {
+        ebIds.add(executionBlock.getId().getProto());
+      }
+
+      getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds);
+    }
+
+    this.finalStageHistory = makeStageHistory();
+    this.finalStageHistory.setTasks(makeTaskHistories());
+  }
+
+  public List<IntermediateEntry> getHashShuffleIntermediateEntries() {
+    return hashShuffleIntermediateEntries;
+  }
+
+  protected void waitingIntermediateReport() {
+    LOG.info(getId() + ", waiting IntermediateReport: expectedTaskNum=" + completeReportReceived.get());
+    synchronized(completeReportReceived) {
+      long startTime = System.currentTimeMillis();
+      while (true) {
+        if (completeReportReceived.get() >= tasks.size()) {
+          LOG.info(getId() + ", completed waiting IntermediateReport");
+          return;
+        } else {
+          try {
+            completeReportReceived.wait(10 * 1000);
+          } catch (InterruptedException e) {
+          }
+          long elapsedTime = System.currentTimeMillis() - startTime;
+          if (elapsedTime >= 120 * 1000) {
+            LOG.error(getId() + ", Timeout while receiving intermediate reports: " + elapsedTime + " ms");
+            abort(StageState.FAILED);
+            return;
+          }
+        }
+      }
+    }
+  }
+
+  public void receiveExecutionBlockReport(TajoWorkerProtocol.ExecutionBlockReport report) {
+    LOG.info(getId() + ", receiveExecutionBlockReport:" +  report.getSucceededTasks());
+    if (!report.getReportSuccess()) {
+      LOG.error(getId() + ", ExecutionBlock final report fail cause:" + report.getReportErrorMessage());
+      abort(StageState.FAILED);
+      return;
+    }
+    if (report.getIntermediateEntriesCount() > 0) {
+      synchronized (hashShuffleIntermediateEntries) {
+        for (IntermediateEntryProto eachInterm: report.getIntermediateEntriesList()) {
+          hashShuffleIntermediateEntries.add(new IntermediateEntry(eachInterm));
+        }
+      }
+    }
+    synchronized(completeReportReceived) {
+      completeReportReceived.addAndGet(report.getSucceededTasks());
+      completeReportReceived.notifyAll();
+    }
+  }
+
+  private static class StageCompleteTransition implements MultipleArcTransition<Stage, StageEvent, StageState> {
+
+    @Override
+    public StageState transition(Stage stage, StageEvent stageEvent) {
+      // TODO - Commit Stage
+      // TODO - records succeeded, failed, killed completed task
+      // TODO - records metrics
+      try {
+        LOG.info(String.format("Stage completed - %s (total=%d, success=%d, killed=%d)",
+            stage.getId().toString(),
+            stage.getTotalScheduledObjectsCount(),
+            stage.getSucceededObjectCount(),
+            stage.killedObjectCount));
+
+        if (stage.killedObjectCount > 0 || stage.failedObjectCount > 0) {
+          if (stage.failedObjectCount > 0) {
+            stage.abort(StageState.FAILED);
+            return StageState.FAILED;
+          } else if (stage.killedObjectCount > 0) {
+            stage.abort(StageState.KILLED);
+            return StageState.KILLED;
+          } else {
+            LOG.error("Invalid State " + stage.getSynchronizedState() + " State");
+            stage.abort(StageState.ERROR);
+            return StageState.ERROR;
+          }
+        } else {
+          stage.complete();
+          return StageState.SUCCEEDED;
+        }
+      } catch (Throwable t) {
+        LOG.error(t.getMessage(), t);
+        stage.abort(StageState.ERROR);
+        return StageState.ERROR;
+      }
+    }
+  }
+
+  private static class DiagnosticsUpdateTransition implements SingleArcTransition<Stage, StageEvent> {
+    @Override
+    public void transition(Stage stage, StageEvent event) {
+      stage.addDiagnostic(((StageDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
+    }
+  }
+
+  private static class InternalErrorTransition implements SingleArcTransition<Stage, StageEvent> {
+    @Override
+    public void transition(Stage stage, StageEvent stageEvent) {
+      stage.abort(StageState.ERROR);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3c833e2a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.java
new file mode 100644
index 0000000..82a06fe
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.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 StageState {
+  NEW,
+  INITED,
+  RUNNING,
+  SUCCEEDED,
+  FAILED,
+  KILL_WAIT,
+  KILLED,
+  ERROR
+}


[3/9] tajo git commit: TAJO-1250: RawFileAppender occasionally causes BufferOverflowException. (jinho)

Posted by ji...@apache.org.
TAJO-1250: RawFileAppender occasionally causes BufferOverflowException. (jinho)

Closes #303


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

Branch: refs/heads/index_support
Commit: cf66a390060c79ba757097886703e30f93d31401
Parents: a4c3484
Author: jhkim <jh...@apache.org>
Authored: Mon Dec 22 16:28:41 2014 +0900
Committer: jhkim <jh...@apache.org>
Committed: Mon Dec 22 16:28:41 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  3 +
 .../org/apache/tajo/storage/StorageManager.java | 13 ++++
 .../src/main/resources/storage-default.xml      | 22 ++++++
 .../src/test/resources/storage-default.xml      | 22 ++++++
 .../java/org/apache/tajo/storage/RawFile.java   | 20 +++--
 .../tajo/storage/text/DelimitedLineReader.java  | 14 +++-
 .../tajo/storage/text/DelimitedTextFile.java    | 16 ++--
 .../org/apache/tajo/storage/TestStorages.java   | 77 ++++++++++++++++++++
 .../src/test/resources/storage-default.xml      | 22 ++++++
 9 files changed, 195 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 36cff8a..245918e 100644
--- a/CHANGES
+++ b/CHANGES
@@ -118,6 +118,9 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1250: RawFileAppender occasionally causes BufferOverflowException. 
+    (jinho)
+
     TAJO-1259: A title in catalog configuration document is different from others. 
     (Jongyoung Park via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
index 07a51ba..609ca20 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.storage;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -173,6 +174,18 @@ public abstract class StorageManager {
    */
   public abstract void closeStorageManager();
 
+
+  /**
+   * Clear all class cache
+   */
+  @VisibleForTesting
+  protected synchronized static void clearCache() {
+    CONSTRUCTOR_CACHE.clear();
+    SCANNER_HANDLER_CACHE.clear();
+    APPENDER_HANDLER_CACHE.clear();
+    storageManagers.clear();
+  }
+
   /**
    * It is called by a Repartitioner for range shuffling when the SortRangeType of SortNode is USING_STORAGE_MANAGER.
    * In general Repartitioner determines the partition range using previous output statistics data.

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
index 67033ed..abea9de 100644
--- a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
@@ -195,4 +195,26 @@
     <name>tajo.storage.appender-handler.hfile.class</name>
     <value>org.apache.tajo.storage.hbase.HFileAppender</value>
   </property>
+
+  <!--- Storage buffer -->
+  <property>
+    <name>tajo.storage.text.io.read-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB read buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.text.io.write-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB write buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.raw.io.read-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB read buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.raw.io.write-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB write buffer</description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
index d1c561b..712f664 100644
--- a/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
@@ -161,4 +161,26 @@
     <name>tajo.storage.appender-handler.avro.class</name>
     <value>org.apache.tajo.storage.avro.AvroAppender</value>
   </property>
+
+  <!--- Storage buffer -->
+  <property>
+    <name>tajo.storage.text.io.read-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB read buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.text.io.write-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB write buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.raw.io.read-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB read buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.raw.io.write-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB write buffer</description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
index 45e07d3..5213ba0 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
@@ -46,6 +46,9 @@ import java.nio.channels.FileChannel;
 
 public class RawFile {
   private static final Log LOG = LogFactory.getLog(RawFile.class);
+  public static final String READ_BUFFER_SIZE = "tajo.storage.raw.io.read-buffer.bytes";
+  public static final String WRITE_BUFFER_SIZE = "tajo.storage.raw.io.write-buffer.bytes";
+  public static final int DEFAULT_BUFFER_SIZE = 128 * StorageUnit.KB;
 
   public static class RawFileScanner extends FileScanner implements SeekableScanner {
     private FileChannel channel;
@@ -92,7 +95,7 @@ public class RawFile {
             + ", fragment length :" + fragment.getLength());
       }
 
-      buf = BufferPool.directBuffer(64 * StorageUnit.KB);
+      buf = BufferPool.directBuffer(conf.getInt(READ_BUFFER_SIZE, DEFAULT_BUFFER_SIZE));
       buffer = buf.nioBuffer(0, buf.capacity());
 
       columnTypes = new DataType[schema.size()];
@@ -382,7 +385,7 @@ public class RawFile {
       if (buffer.capacity() - buffer.remaining()  <  writableBytes) {
         buf.setIndex(buffer.position(), buffer.limit());
         buf.markReaderIndex();
-        buf.discardSomeReadBytes();
+        buf.discardReadBytes();
         buf.ensureWritable(writableBytes);
         buffer = buf.nioBuffer(0, buf.capacity());
         buffer.limit(buf.writerIndex());
@@ -491,7 +494,7 @@ public class RawFile {
         columnTypes[i] = schema.getColumn(i).getDataType();
       }
 
-      buf = BufferPool.directBuffer(64 * StorageUnit.KB);
+      buf = BufferPool.directBuffer(conf.getInt(WRITE_BUFFER_SIZE, DEFAULT_BUFFER_SIZE));
       buffer = buf.nioBuffer(0, buf.capacity());
 
       // comput the number of bytes, representing the null flags
@@ -532,6 +535,13 @@ public class RawFile {
         buffer.limit(limit);
         buffer.compact();
 
+        //increase the write-buffer
+        if(buffer.remaining() < sizeToBeWritten) {
+          buf.setIndex(buffer.position(), buffer.limit());
+          buf.ensureWritable(sizeToBeWritten);
+          buffer = buf.nioBuffer(0, buf.capacity());
+          buffer.position(buf.readerIndex());
+        }
         return true;
       } else {
         return false;
@@ -632,8 +642,8 @@ public class RawFile {
           continue;
         }
 
-        // 8 is the maximum bytes size of all types
-        if (flushBufferAndReplace(recordOffset, 8)) {
+        // 10 is the maximum bytes size of all types
+        if (flushBufferAndReplace(recordOffset, 10)) {
           recordOffset = 0;
         }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
index 1b433b5..8b33858 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
@@ -36,6 +36,7 @@ import org.apache.tajo.storage.ByteBufInputChannel;
 import org.apache.tajo.storage.FileScanner;
 import org.apache.tajo.storage.compress.CodecPool;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.unit.StorageUnit;
 
 import java.io.Closeable;
 import java.io.DataInputStream;
@@ -45,7 +46,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 public class DelimitedLineReader implements Closeable {
   private static final Log LOG = LogFactory.getLog(DelimitedLineReader.class);
-  private final static int DEFAULT_PAGE_SIZE = 128 * 1024;
 
   private FileSystem fs;
   private FSDataInputStream fis;
@@ -60,12 +60,18 @@ public class DelimitedLineReader implements Closeable {
   private AtomicInteger lineReadBytes = new AtomicInteger();
   private FileFragment fragment;
   private Configuration conf;
+  private int bufferSize;
 
   public DelimitedLineReader(Configuration conf, final FileFragment fragment) throws IOException {
+    this(conf, fragment, 128 * StorageUnit.KB);
+  }
+
+  public DelimitedLineReader(Configuration conf, final FileFragment fragment, int bufferSize) throws IOException {
     this.fragment = fragment;
     this.conf = conf;
     this.factory = new CompressionCodecFactory(conf);
     this.codec = factory.getCodec(fragment.getPath());
+    this.bufferSize = bufferSize;
     if (this.codec instanceof SplittableCompressionCodec) {
       throw new NotImplementedException(); // bzip2 does not support multi-thread model
     }
@@ -83,14 +89,16 @@ public class DelimitedLineReader implements Closeable {
       decompressor = CodecPool.getDecompressor(codec);
       is = new DataInputStream(codec.createInputStream(fis, decompressor));
       ByteBufInputChannel channel = new ByteBufInputChannel(is);
-      lineReader = new ByteBufLineReader(channel, BufferPool.directBuffer(DEFAULT_PAGE_SIZE));
+
+      ByteBuf buf = BufferPool.directBuffer(bufferSize);
+      lineReader = new ByteBufLineReader(channel, buf);
     } else {
       fis.seek(startOffset);
       is = fis;
 
       ByteBufInputChannel channel = new ByteBufInputChannel(is);
       lineReader = new ByteBufLineReader(channel,
-          BufferPool.directBuffer((int) Math.min(DEFAULT_PAGE_SIZE, end)));
+          BufferPool.directBuffer((int) Math.min(bufferSize, end)));
     }
     eof = false;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index 59129d1..15db4c3 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -39,6 +39,7 @@ import org.apache.tajo.storage.compress.CodecPool;
 import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.ReflectionUtil;
 
 import java.io.BufferedOutputStream;
@@ -55,6 +56,9 @@ import static org.apache.tajo.storage.StorageConstants.TEXT_ERROR_TOLERANCE_MAXN
 public class DelimitedTextFile {
 
   public static final byte LF = '\n';
+  public static final String READ_BUFFER_SIZE = "tajo.storage.text.io.read-buffer.bytes";
+  public static final String WRITE_BUFFER_SIZE = "tajo.storage.text.io.write-buffer.bytes";
+  public static final int DEFAULT_BUFFER_SIZE = 128 * StorageUnit.KB;
 
   private static final Log LOG = LogFactory.getLog(DelimitedTextFile.class);
 
@@ -105,8 +109,7 @@ public class DelimitedTextFile {
     private CompressionCodecFactory codecFactory;
     private CompressionCodec codec;
     private Path compressedPath;
-    private byte[] nullChars;
-    private int BUFFER_SIZE = 128 * 1024;
+    private int bufferSize;
     private int bufferedBytes = 0;
     private long pos = 0;
 
@@ -165,8 +168,9 @@ public class DelimitedTextFile {
       serializer = getLineSerde().createSerializer(schema, meta);
       serializer.init();
 
+      bufferSize = conf.getInt(WRITE_BUFFER_SIZE, DEFAULT_BUFFER_SIZE);
       if (os == null) {
-        os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
+        os = new NonSyncByteArrayOutputStream(bufferSize);
       }
 
       os.reset();
@@ -189,7 +193,7 @@ public class DelimitedTextFile {
       bufferedBytes += rowBytes;
 
       // refill buffer if necessary
-      if (bufferedBytes > BUFFER_SIZE) {
+      if (bufferedBytes > bufferSize) {
         flushBuffer();
       }
       // Statistical section
@@ -288,7 +292,7 @@ public class DelimitedTextFile {
                                     final Fragment fragment)
         throws IOException {
       super(conf, schema, meta, fragment);
-      reader = new DelimitedLineReader(conf, this.fragment);
+      reader = new DelimitedLineReader(conf, this.fragment, conf.getInt(READ_BUFFER_SIZE, 128 * StorageUnit.KB));
       if (!reader.isCompressed()) {
         splittable = true;
       }
@@ -307,7 +311,7 @@ public class DelimitedTextFile {
         reader.close();
       }
 
-      reader = new DelimitedLineReader(conf, fragment);
+      reader = new DelimitedLineReader(conf, fragment, conf.getInt(READ_BUFFER_SIZE, 128 * StorageUnit.KB));
       reader.init();
       recordCount = 0;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
index 15998f2..9577e3d 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -94,6 +94,20 @@ public class TestStorages {
       "  ]\n" +
       "}\n";
 
+  private static String TEST_MAX_VALUE_AVRO_SCHEMA =
+      "{\n" +
+          "  \"type\": \"record\",\n" +
+          "  \"namespace\": \"org.apache.tajo\",\n" +
+          "  \"name\": \"testMaxValue\",\n" +
+          "  \"fields\": [\n" +
+          "    { \"name\": \"col4\", \"type\": \"float\" },\n" +
+          "    { \"name\": \"col5\", \"type\": \"double\" },\n" +
+          "    { \"name\": \"col1\", \"type\": \"int\" },\n" +
+          "    { \"name\": \"col2\", \"type\": \"int\" },\n" +
+          "    { \"name\": \"col3\", \"type\": \"long\" }\n" +
+          "  ]\n" +
+          "}\n";
+
   private StoreType storeType;
   private boolean splitable;
   private boolean statsable;
@@ -875,4 +889,67 @@ public class TestStorages {
       assertEquals(appender.getStats().getNumRows().longValue(), readRows);
     }
   }
+
+  @Test
+  public void testMaxValue() throws IOException {
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.FLOAT4);
+    schema.addColumn("col2", Type.FLOAT8);
+    schema.addColumn("col3", Type.INT2);
+    schema.addColumn("col4", Type.INT4);
+    schema.addColumn("col5", Type.INT8);
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    if (storeType == StoreType.AVRO) {
+      meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL, TEST_MAX_VALUE_AVRO_SCHEMA);
+    }
+
+    if (storeType == StoreType.RAW) {
+      StorageManager.clearCache();
+      /* TAJO-1250 reproduce BufferOverflow of RAWFile */
+      int headerSize = 4 + 2 + 1; //Integer record length + Short null-flag length + 1 byte null flags
+      /* max varint32: 5 bytes, max varint64: 10 bytes */
+      int record = 4 + 8 + 2 + 5 + 8; // required size is 27
+      conf.setInt(RawFile.WRITE_BUFFER_SIZE, record + headerSize);
+    }
+
+    FileStorageManager sm = (FileStorageManager) StorageManager.getFileStorageManager(conf);
+    Path tablePath = new Path(testDir, "testMaxValue.data");
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+
+    appender.init();
+
+    Tuple tuple = new VTuple(5);
+    tuple.put(new Datum[]{
+        DatumFactory.createFloat4(Float.MAX_VALUE),
+        DatumFactory.createFloat8(Double.MAX_VALUE),
+        DatumFactory.createInt2(Short.MAX_VALUE),
+        DatumFactory.createInt4(Integer.MAX_VALUE),
+        DatumFactory.createInt8(Long.MAX_VALUE)
+    });
+
+    appender.addTuple(tuple);
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner = sm.getScanner(meta, schema, fragment);
+    scanner.init();
+
+    Tuple retrieved;
+    while ((retrieved = scanner.next()) != null) {
+      for (int i = 0; i < tuple.size(); i++) {
+        assertEquals(tuple.get(i), retrieved.get(i));
+      }
+    }
+    scanner.close();
+
+
+    if (storeType == StoreType.RAW){
+      StorageManager.clearCache();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/cf66a390/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
index 737284b..adddf66 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
@@ -175,4 +175,26 @@
     <name>tajo.storage.appender-handler.avro.class</name>
     <value>org.apache.tajo.storage.avro.AvroAppender</value>
   </property>
+
+  <!--- Storage buffer -->
+  <property>
+    <name>tajo.storage.text.io.read-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB read buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.text.io.write-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB write buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.raw.io.read-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB read buffer</description>
+  </property>
+  <property>
+    <name>tajo.storage.raw.io.write-buffer.bytes</name>
+    <value>131072</value>
+    <description>128KB write buffer</description>
+  </property>
 </configuration>