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>