tajo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hyun...@apache.org
Subject git commit: TAJO-387: Query is hanging when errors occurs in Query or SubQuery class. (hyunsik)
Date Fri, 06 Dec 2013 07:24:54 GMT
Updated Branches:
  refs/heads/master c18a3f862 -> da8cd2984


TAJO-387: Query is hanging when errors occurs in Query or SubQuery class. (hyunsik)


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

Branch: refs/heads/master
Commit: da8cd2984317b61273ccd0d415401873c4eb4d7c
Parents: c18a3f8
Author: Hyunsik Choi <hyunsik@apache.org>
Authored: Fri Dec 6 16:24:35 2013 +0900
Committer: Hyunsik Choi <hyunsik@apache.org>
Committed: Fri Dec 6 16:24:35 2013 +0900

----------------------------------------------------------------------
 .../tajo/catalog/statistics/StatisticsUtil.java |   5 +-
 .../main/java/org/apache/tajo/cli/TajoCli.java  |   7 +-
 .../org/apache/tajo/client/QueryStatus.java     |   6 -
 .../org/apache/tajo/master/GlobalEngine.java    |  18 +-
 .../apache/tajo/master/TajoContainerProxy.java  |   4 -
 .../tajo/master/TajoMasterClientService.java    |   7 +-
 .../apache/tajo/master/YarnContainerProxy.java  |   4 -
 .../event/QueryDiagnosticsUpdateEvent.java      |   2 +-
 .../tajo/master/event/QueryEventType.java       |   1 -
 .../event/SubQueryDiagnosticsUpdateEvent.java   |  34 ++++
 .../tajo/master/event/SubQueryEventType.java    |   2 +
 .../apache/tajo/master/querymaster/Query.java   |  93 +++++----
 .../master/querymaster/QueryMasterTask.java     |   3 +-
 .../tajo/master/querymaster/SubQuery.java       | 196 ++++++++++++++-----
 .../tajo/master/querymaster/SubQueryState.java  |   3 +-
 .../tajo/worker/TajoWorkerClientService.java    |   4 +-
 .../src/main/proto/ClientProtos.proto           |   1 -
 .../tajo/storage/AbstractStorageManager.java    |   8 +-
 18 files changed, 270 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
index 014b67c..fd72a9d 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
@@ -45,10 +45,11 @@ public class StatisticsUtil {
   }
 
   public static TableStats aggregateTableStat(List<TableStats> tableStatses) {
-    if(tableStatses == null || tableStatses.size() == 0 || tableStatses.get(0) == null)
-      return null;
     TableStats aggregated = new TableStats();
 
+    if(tableStatses == null || tableStatses.size() == 0 || tableStatses.get(0) == null)
+      return aggregated;
+
     ColumnStats[] css = null;
     if (tableStatses.size() > 0) {
       for (TableStats ts : tableStatses) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
index 9f09a39..df5ae2b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
@@ -354,14 +354,15 @@ public class TajoCli {
         }
       }
 
-      if (isFailed(status.getState())) {
-        sout.println(status.getErrorMessage());
+      if (status.getState() == QueryState.QUERY_ERROR) {
+        sout.println("Internal error!");
+      } else if (status.getState() == QueryState.QUERY_FAILED) {
+        sout.println("Query failed!");
       } else if (status.getState() == QueryState.QUERY_KILLED) {
         sout.println(queryId + " is killed.");
       } else {
         if (status.getState() == QueryState.QUERY_SUCCEEDED) {
           sout.println("final state: " + status.getState()
-              + ", init time: " + (((float)(status.getInitTime() - status.getSubmitTime())
/ 1000.0) + " sec")
               + ", response time: " + (((float)(status.getFinishTime() - status.getSubmitTime())
/ 1000.0)
               + " sec"));
           if (status.hasResult()) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
index c7122b3..203f9aa 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
@@ -27,7 +27,6 @@ public class QueryStatus {
   private QueryState state;
   private float progress;
   private long submitTime;
-  private long initTime;
   private long finishTime;
   private boolean hasResult;
   private String errorText;
@@ -39,7 +38,6 @@ public class QueryStatus {
     state = proto.getState();
     progress = proto.getProgress();
     submitTime = proto.getSubmitTime();
-    initTime = proto.getInitTime();
     finishTime = proto.getFinishTime();
     hasResult = proto.getHasResult();
     if (proto.hasErrorMessage()) {
@@ -74,10 +72,6 @@ public class QueryStatus {
     return this.submitTime;
   }
 
-  public long getInitTime() {
-    return this.initTime;
-  }
-
   public long getFinishTime() {
     return this.finishTime;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 4f18b11..29febff 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -246,19 +246,19 @@ public class GlobalEngine extends AbstractService {
       meta = CatalogUtil.newTableMeta(createTable.getStorageType());
     }
 
-    if(!createTable.isExternal()){
+    if(createTable.isExternal()){
+      Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
+    } else {
       Path tablePath = new Path(sm.getWarehouseDir(), createTable.getTableName().toLowerCase());
       createTable.setPath(tablePath);
-    } else {
-      Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
     }
 
-    return createTableOnDirectory(createTable.getTableName(), createTable.getSchema(), meta,
-        createTable.getPath(), true, createTable.getPartitions());
+    return createTableOnPath(createTable.getTableName(), createTable.getSchema(), meta,
+        createTable.getPath(), !createTable.isExternal(), createTable.getPartitions());
   }
 
-  public TableDesc createTableOnDirectory(String tableName, Schema schema, TableMeta meta,
-                                          Path path, boolean isCreated, Partitions partitions)
+  public TableDesc createTableOnPath(String tableName, Schema schema, TableMeta meta,
+                                     Path path, boolean isCreated, Partitions partitions)
       throws IOException {
     if (catalog.existsTable(tableName)) {
       throw new AlreadyExistsTableException(tableName);
@@ -270,8 +270,8 @@ public class GlobalEngine extends AbstractService {
       fs.mkdirs(path);
     }
 
-    if(fs.exists(path) && fs.isFile(path)) {
-      throw new IOException("ERROR: LOCATION must be a directory.");
+    if(!fs.exists(path)) {
+      throw new IOException("ERROR: " + path.toUri() + " does not exist");
     }
 
     long totalSize = 0;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
index ce5f401..bdedd98 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.event.QueryEvent;
-import org.apache.tajo.master.event.QueryEventType;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.master.rm.TajoWorkerContainer;
 import org.apache.tajo.master.rm.TajoWorkerContainerId;
@@ -61,8 +59,6 @@ public class TajoContainerProxy extends ContainerProxy {
         container.getId() + "," + container.getNodeId() + ", pullServer=" + port);
 
     assignExecutionBlock(executionBlockId, container);
-
-    context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
   }
 
   private void assignExecutionBlock(ExecutionBlockId executionBlockId, Container container)
{

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index ed05e44..4231aef 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -31,7 +31,10 @@ import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.exception.NoSuchTableException;
 import org.apache.tajo.catalog.partition.Partitions;
 import org.apache.tajo.conf.TajoConf;
@@ -317,7 +320,7 @@ public class TajoMasterClientService extends AbstractService {
 
         TableDesc desc;
         try {
-          desc = context.getGlobalEngine().createTableOnDirectory(request.getName(), schema,
+          desc = context.getGlobalEngine().createTableOnPath(request.getName(), schema,
               meta, path, false, partitions);
         } catch (Exception e) {
           return TableResponse.newBuilder()

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
index 5117700..4dea476 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
@@ -42,8 +42,6 @@ import org.apache.hadoop.yarn.util.Records;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.event.QueryEvent;
-import org.apache.tajo.master.event.QueryEventType;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.pullserver.PullServerAuxService;
 import org.apache.tajo.worker.TajoWorker;
@@ -140,8 +138,6 @@ public class YarnContainerProxy extends ContainerProxy {
             + port + " returned for " + containerID);
       }
 
-      context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
-
       this.state = ContainerState.RUNNING;
       this.hostName = containerMgrAddress.split(":")[0];
       context.getResourceAllocator().addContainer(containerID, this);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
index a9191ac..eebf5c0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryDiagnosticsUpdateEvent.java
@@ -24,7 +24,7 @@ public class QueryDiagnosticsUpdateEvent extends QueryEvent {
   private final String msg;
 
   public QueryDiagnosticsUpdateEvent(final QueryId id, String diagnostic) {
-    super(id, QueryEventType.SUBQUERY_COMPLETED);
+    super(id, QueryEventType.DIAGNOSTIC_UPDATE);
     this.msg = diagnostic;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryEventType.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryEventType.java
index c767df1..d5f7e38 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryEventType.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryEventType.java
@@ -20,7 +20,6 @@ package org.apache.tajo.master.event;
 
 public enum QueryEventType {
   START,
-  INIT_COMPLETED,
   INTERNAL_ERROR,
   SUBQUERY_COMPLETED,
   KILL,

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
index bddd5ba..2e56c79 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEventType.java
@@ -35,5 +35,7 @@ public enum SubQueryEventType {
   // Producer: Completed
   SQ_SUBQUERY_COMPLETED,
 
+  // Producer: Any component
+  SQ_DIAGNOSTIC_UPDATE,
   SQ_INTERNAL_ERROR
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 49e670b..c17b726 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -83,27 +83,56 @@ public class Query implements EventHandler<QueryEvent> {
   // State Machine
   private final StateMachine<QueryState, QueryEventType, QueryEvent> stateMachine;
 
+  // Transition Handler
+  private static final SingleArcTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+  private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
+
   protected static final StateMachineFactory
       <Query,QueryState,QueryEventType,QueryEvent> stateMachineFactory =
       new StateMachineFactory<Query, QueryState, QueryEventType, QueryEvent>
           (QueryState.QUERY_NEW)
 
-      .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_RUNNING,
-          QueryEventType.START, new StartTransition())
-
-      .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
-          QueryEventType.INIT_COMPLETED, new InitCompleteTransition())
-      .addTransition(QueryState.QUERY_RUNNING,
-          EnumSet.of(QueryState.QUERY_RUNNING, QueryState.QUERY_SUCCEEDED,
-              QueryState.QUERY_FAILED),
-          QueryEventType.SUBQUERY_COMPLETED,
-          new SubQueryCompletedTransition())
-      .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
-          QueryEventType.INTERNAL_ERROR, new InternalErrorTransition())
-       .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
-          QueryEventType.INTERNAL_ERROR)
-
-      .installTopology();
+          // Transitions from NEW state
+          .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_RUNNING,
+              QueryEventType.START,
+              new StartTransition())
+          .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_NEW,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from RUNNING state
+          .addTransition(QueryState.QUERY_RUNNING,
+              EnumSet.of(QueryState.QUERY_RUNNING, QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED,
+                  QueryState.QUERY_ERROR),
+              QueryEventType.SUBQUERY_COMPLETED,
+              new SubQueryCompletedTransition())
+          .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from FAILED state
+          .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from ERROR state
+          .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+              QueryEventType.DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+              QueryEventType.INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          .installTopology();
 
   public Query(final QueryMasterTask.QueryMasterTaskContext context, final QueryId id,
                final long appSubmitTime,
@@ -180,15 +209,6 @@ public class Query implements EventHandler<QueryEvent> {
     startTime = clock.getTime();
   }
 
-  public long getInitializationTime() {
-    return initializationTime;
-  }
-
-  public void setInitializationTime() {
-    initializationTime = clock.getTime();
-  }
-
-
   public long getFinishTime() {
     return finishTime;
   }
@@ -265,10 +285,9 @@ public class Query implements EventHandler<QueryEvent> {
           query.getExecutionBlockCursor().nextBlock(), query.sm);
       subQuery.setPriority(query.priority--);
       query.addSubQuery(subQuery);
-      LOG.debug("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
 
-      subQuery.handle(new SubQueryEvent(subQuery.getId(),
-          SubQueryEventType.SQ_INIT));
+      subQuery.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_INIT));
+      LOG.debug("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
     }
   }
 
@@ -321,8 +340,12 @@ public class Query implements EventHandler<QueryEvent> {
 
           return QueryState.QUERY_SUCCEEDED;
         }
+      } else if (castEvent.getFinalState() == SubQueryState.ERROR) {
+        query.setFinishTime();
+        return QueryState.QUERY_ERROR;
       } else {
         // if at least one subquery is failed, the query is also failed.
+        query.setFinishTime();
         return QueryState.QUERY_FAILED;
       }
     }
@@ -392,21 +415,18 @@ public class Query implements EventHandler<QueryEvent> {
     }
   }
 
-  private static class InitCompleteTransition implements
-      SingleArcTransition<Query, QueryEvent> {
+  private static class DiagnosticsUpdateTransition implements SingleArcTransition<Query,
QueryEvent> {
     @Override
     public void transition(Query query, QueryEvent event) {
-      if (query.initializationTime == 0) {
-        query.setInitializationTime();
-      }
+      query.addDiagnostic(((QueryDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
     }
   }
 
-  private static class InternalErrorTransition
-      implements SingleArcTransition<Query, QueryEvent> {
+  private static class InternalErrorTransition implements SingleArcTransition<Query, QueryEvent>
{
 
     @Override
     public void transition(Query query, QueryEvent event) {
+      query.setFinishTime();
       query.finished(QueryState.QUERY_ERROR);
     }
   }
@@ -438,8 +458,7 @@ public class Query implements EventHandler<QueryEvent> {
         getStateMachine().doTransition(event.getType(), event);
       } catch (InvalidStateTransitonException e) {
         LOG.error("Can't handle this event at current state", e);
-        eventHandler.handle(new QueryEvent(this.id,
-            QueryEventType.INTERNAL_ERROR));
+        eventHandler.handle(new QueryEvent(this.id, QueryEventType.INTERNAL_ERROR));
       }
 
       //notify the eventhandler of state change

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index 5b23fb2..8cd7d45 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -299,8 +299,7 @@ public class QueryMasterTask extends CompositeService {
 
       dispatcher.register(QueryEventType.class, query);
 
-      queryTaskContext.getEventHandler().handle(new QueryEvent(queryId,
-          QueryEventType.START));
+      queryTaskContext.getEventHandler().handle(new QueryEvent(queryId, QueryEventType.START));
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
       //TODO how set query failed(???)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index a75c200..edaa463 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -84,6 +84,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
   private final AbstractStorageManager sm;
   private AbstractTaskScheduler taskScheduler;
   private QueryMasterTask.QueryMasterTaskContext context;
+  private final List<String> diagnostics = new ArrayList<String>();
 
   private long startTime;
   private long finishTime;
@@ -91,52 +92,112 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
   volatile Map<QueryUnitId, QueryUnit> tasks = new ConcurrentHashMap<QueryUnitId,
QueryUnit>();
   volatile Map<ContainerId, Container> containers = new ConcurrentHashMap<ContainerId,
Container>();
 
-  private static ContainerLaunchTransition CONTAINER_LAUNCH_TRANSITION = new ContainerLaunchTransition();
-  private StateMachine<SubQueryState, SubQueryEventType, SubQueryEvent>
-      stateMachine;
+  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 FailedTransition FAILED_TRANSITION = new FailedTransition();
+  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.INIT, SubQueryState.FAILED, SubQueryState.SUCCEEDED),
+              EnumSet.of(SubQueryState.INIT, 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.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from INIT state
           .addTransition(SubQueryState.INIT, SubQueryState.CONTAINER_ALLOCATED,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
-
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINER_LAUNCH_TRANSITION)
+          .addTransition(SubQueryState.INIT, SubQueryState.INIT,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.NEW, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from CONTAINER_ALLOCATED state
           .addTransition(SubQueryState.CONTAINER_ALLOCATED,
-              EnumSet.of(SubQueryState.RUNNING, SubQueryState.FAILED,
-                  SubQueryState.SUCCEEDED), SubQueryEventType.SQ_START, new StartTransition())
+              EnumSet.of(SubQueryState.RUNNING, SubQueryState.FAILED, SubQueryState.SUCCEEDED),
+              SubQueryEventType.SQ_START,
+              new StartTransition())
+          .addTransition(SubQueryState.CONTAINER_ALLOCATED, SubQueryState.CONTAINER_ALLOCATED,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINER_LAUNCH_TRANSITION)
           .addTransition(SubQueryState.CONTAINER_ALLOCATED, SubQueryState.CONTAINER_ALLOCATED,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.CONTAINER_ALLOCATED, 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_START)
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED,
+              CONTAINER_LAUNCH_TRANSITION)
           .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
-              SubQueryEventType.SQ_TASK_COMPLETED, new TaskCompletedTransition())
+              SubQueryEventType.SQ_START)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_TASK_COMPLETED,
+              new TaskCompletedTransition())
           .addTransition(SubQueryState.RUNNING, SubQueryState.SUCCEEDED,
-              SubQueryEventType.SQ_SUBQUERY_COMPLETED, new SubQueryCompleteTransition())
+              SubQueryEventType.SQ_SUBQUERY_COMPLETED,
+              new SubQueryCompleteTransition())
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
           .addTransition(SubQueryState.RUNNING, SubQueryState.FAILED,
-              SubQueryEventType.SQ_FAILED, new InternalErrorTransition())
+              SubQueryEventType.SQ_FAILED,
+              FAILED_TRANSITION)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
 
+          // Transitions from SUCCEEDED state
           .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
               SubQueryEventType.SQ_START)
           .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
               SubQueryEventType.SQ_CONTAINER_ALLOCATED)
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
 
+          // Transitions from FAILED state
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
           .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
               SubQueryEventType.SQ_START)
           .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
               SubQueryEventType.SQ_CONTAINER_ALLOCATED)
           .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
-                 SubQueryEventType.SQ_FAILED)
-          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              SubQueryEventType.SQ_FAILED)
+          .addTransition(SubQueryState.FAILED, SubQueryState.ERROR,
+              SubQueryEventType.SQ_INTERNAL_ERROR,
+              INTERNAL_ERROR_TRANSITION)
+
+          // Transitions from FAILED state
+          .addTransition(SubQueryState.ERROR, SubQueryState.ERROR,
+              SubQueryEventType.SQ_DIAGNOSTIC_UPDATE,
+              DIAGNOSTIC_UPDATE_TRANSITION)
+          .addTransition(SubQueryState.ERROR, SubQueryState.ERROR,
+              SubQueryEventType.SQ_FAILED)
+          .addTransition(SubQueryState.ERROR, SubQueryState.ERROR,
               SubQueryEventType.SQ_INTERNAL_ERROR)
-      .installTopology();
+
+          .installTopology();
 
 
   private final Lock readLock;
@@ -232,6 +293,9 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
     // - CleanUp Tasks
     // - Record History
 
+    stopScheduler();
+    releaseContainers();
+    setFinishTime();
     eventHandler.handle(new SubQueryCompletedEvent(getId(), finalState));
   }
 
@@ -276,6 +340,19 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
     return statistics;
   }
 
+  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());
@@ -358,8 +435,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
 
   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()));
+    eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP ,getId(),
containers.values()));
   }
 
   private void finish() {
@@ -450,12 +526,11 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
           }
         }
       } catch (Exception e) {
-        LOG.warn("SubQuery (" + subQuery.getId() + ") failed", e);
-        subQuery.eventHandler.handle(
-            new QueryDiagnosticsUpdateEvent(subQuery.getId().getQueryId(), e.getMessage()));
-        subQuery.eventHandler.handle(
-            new SubQueryCompletedEvent(subQuery.getId(), SubQueryState.FAILED));
-        return SubQueryState.FAILED;
+        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;
@@ -710,23 +785,27 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
 
     @Override
     public void transition(SubQuery subQuery, SubQueryEvent event) {
-      SubQueryContainerAllocationEvent allocationEvent =
-          (SubQueryContainerAllocationEvent) event;
-      for (Container container : allocationEvent.getAllocatedContainer()) {
-        ContainerId cId = container.getId();
-        if (subQuery.containers.containsKey(cId)) {
-          LOG.info(">>>>>>>>>>>> Duplicate Container!
<<<<<<<<<<<");
+      try {
+        SubQueryContainerAllocationEvent allocationEvent =
+            (SubQueryContainerAllocationEvent) event;
+        for (Container container : allocationEvent.getAllocatedContainer()) {
+          ContainerId cId = container.getId();
+          if (subQuery.containers.containsKey(cId)) {
+            LOG.info(">>>>>>>>>>>> Duplicate Container!
<<<<<<<<<<<");
+          }
+          subQuery.containers.put(cId, container);
+          // TODO - This is debugging message. Should be removed
+          subQuery.i++;
         }
-        subQuery.containers.put(cId, container);
-        // TODO - This is debugging message. Should be removed
-        subQuery.i++;
-      }
-      LOG.info("SubQuery (" + subQuery.getId() + ") has " + subQuery.i + " containers!");
-      subQuery.eventHandler.handle(
-          new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_LAUNCH,
-              subQuery.getId(), allocationEvent.getAllocatedContainer()));
+        LOG.info("SubQuery (" + subQuery.getId() + ") has " + subQuery.i + " containers!");
+        subQuery.eventHandler.handle(
+            new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_LAUNCH,
+                subQuery.getId(), allocationEvent.getAllocatedContainer()));
 
-      subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_START));
+        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_START));
+      } catch (Throwable t) {
+
+      }
     }
   }
 
@@ -755,16 +834,20 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
 
     @Override
     public void transition(SubQuery subQuery,
-                                     SubQueryEvent event) {
+                           SubQueryEvent event) {
       subQuery.completedTaskCount++;
       SubQueryTaskEvent taskEvent = (SubQueryTaskEvent) event;
       QueryUnitAttempt task = subQuery.getQueryUnit(taskEvent.getTaskId()).getSuccessfulAttempt();
 
-      LOG.info(subQuery.getId() + " SubQuery Succeeded " + subQuery.completedTaskCount +
"/"
-          + subQuery.tasks.size() + " on " + task.getHost() + ":" + task.getPort());
-      if (subQuery.completedTaskCount == subQuery.tasks.size()) {
-        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
-            SubQueryEventType.SQ_SUBQUERY_COMPLETED));
+      if (task == null) { // task failed
+        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_FAILED));
+      } else {
+        LOG.info(subQuery.getId() + " SubQuery Succeeded " + subQuery.completedTaskCount
+ "/"
+            + subQuery.tasks.size() + " on " + task.getHost() + ":" + task.getPort());
+        if (subQuery.completedTaskCount == subQuery.tasks.size()) {
+          subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
+              SubQueryEventType.SQ_SUBQUERY_COMPLETED));
+        }
       }
     }
   }
@@ -784,13 +867,24 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
     }
   }
 
-  private static class InternalErrorTransition
-      implements SingleArcTransition<SubQuery, SubQueryEvent> {
+  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) {
+    public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      subQuery.abortSubQuery(SubQueryState.ERROR);
+    }
+  }
 
+  private static class FailedTransition implements SingleArcTransition<SubQuery, SubQueryEvent>
{
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      subQuery.abortSubQuery(SubQueryState.FAILED);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
index c8256ec..ce4d209 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
@@ -24,5 +24,6 @@ public enum SubQueryState {
   INIT,
   RUNNING,
   SUCCEEDED,
-  FAILED
+  FAILED,
+  ERROR
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
index 4afea20..55bc746 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
@@ -29,7 +29,6 @@ import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.QueryMasterClientProtocol;
 import org.apache.tajo.master.querymaster.Query;
@@ -141,7 +140,7 @@ public class TajoWorkerClientService extends AbstractService {
       } else {
         switch (query.getState()) {
           case QUERY_SUCCEEDED:
-            builder.setTableDesc((CatalogProtos.TableDescProto)query.getResultDesc().getProto());
+            builder.setTableDesc(query.getResultDesc().getProto());
             break;
           case QUERY_FAILED:
           case QUERY_ERROR:
@@ -179,7 +178,6 @@ public class TajoWorkerClientService extends AbstractService {
           builder.setState(query.getState());
           builder.setProgress(query.getProgress());
           builder.setSubmitTime(query.getAppSubmitTime());
-          builder.setInitTime(query.getInitializationTime());
           builder.setHasResult(
               !(queryMasterTask.getQueryTaskContext().getQueryContext().isCreateTable() ||
                   queryMasterTask.getQueryTaskContext().getQueryContext().isInsert())

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
index 40811a0..abadfeb 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
@@ -91,7 +91,6 @@ message GetQueryStatusResponse {
   optional QueryState state = 3;
   optional float progress = 4;
   optional int64 submitTime = 5;
-  optional int64 initTime = 6;
   optional int64 finishTime = 7;
   optional bool hasResult = 8;
   optional string errorMessage = 9;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/da8cd298/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
index 314eff0..74e704f 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.storage;
 
 
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.net.util.Base64;
@@ -559,8 +560,13 @@ public abstract class AbstractStorageManager {
     // generate splits'
 
     List<FileFragment> splits = new ArrayList<FileFragment>();
-    List<FileStatus> files = listStatus(inputPath);
     FileSystem fs = inputPath.getFileSystem(conf);
+    List<FileStatus> files;
+    if (fs.isFile(inputPath)) {
+      files = Lists.newArrayList(fs.getFileStatus(inputPath));
+    } else {
+      files = listStatus(inputPath);
+    }
     for (FileStatus file : files) {
       Path path = file.getPath();
       long length = file.getLen();


Mime
View raw message