From d58f48fddf8485621d914bcb45cf64cca0285eb7 Mon Sep 17 00:00:00 2001 From: "clark.kang" Date: Fri, 5 Jun 2015 22:56:42 +0900 Subject: [PATCH 1/7] remove-offset --- .../ws/rs/resources/QueryResultResource.java | 35 +++------ .../rs/resources/TestQueryResultResource.java | 75 +++---------------- 2 files changed, 21 insertions(+), 89 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java index 92b95d24f0..4a78921a2d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java @@ -72,6 +72,9 @@ public class QueryResultResource { private static final String countKeyName = "count"; private static final String tajoDigestHeaderName = "X-Tajo-Digest"; + private static final String tajoOffsetHeaderName = "X-Tajo-Offset"; + private static final String tajoCountHeaderName = "X-Tajo-Count"; + private static final String tajoEOSHeaderName = "X-Tajo-EOS"; public UriInfo getUriInfo() { return uriInfo; @@ -259,7 +262,6 @@ public Response run(JerseyResourceDelegateContext context) { @Produces(MediaType.APPLICATION_OCTET_STREAM) public Response getQueryResultSet(@HeaderParam(QueryResource.tajoSessionIdHeaderName) String sessionId, @PathParam("cacheId") String cacheId, - @DefaultValue("-1") @QueryParam("offset") int offset, @DefaultValue("100") @QueryParam("count") int count) { if (LOG.isDebugEnabled()) { LOG.debug("Client sent a get query result set request."); @@ -274,9 +276,6 @@ public Response getQueryResultSet(@HeaderParam(QueryResource.tajoSessionIdHeader JerseyResourceDelegateContextKey cacheIdKey = JerseyResourceDelegateContextKey.valueOf(cacheIdKeyName, Long.class); context.put(cacheIdKey, Long.valueOf(cacheId)); - JerseyResourceDelegateContextKey offsetKey = - JerseyResourceDelegateContextKey.valueOf(offsetKeyName, Integer.class); - context.put(offsetKey, offset); JerseyResourceDelegateContextKey countKey = JerseyResourceDelegateContextKey.valueOf(countKeyName, Integer.class); context.put(countKey, count); @@ -311,9 +310,6 @@ public Response run(JerseyResourceDelegateContext context) { JerseyResourceDelegateContextKey clientApplicationKey = JerseyResourceDelegateContextKey.valueOf(JerseyResourceDelegateUtil.ClientApplicationKey, ClientApplication.class); ClientApplication clientApplication = context.get(clientApplicationKey); - JerseyResourceDelegateContextKey offsetKey = - JerseyResourceDelegateContextKey.valueOf(offsetKeyName, Integer.class); - int offset = context.get(offsetKey); JerseyResourceDelegateContextKey countKey = JerseyResourceDelegateContextKey.valueOf(countKeyName, Integer.class); int count = context.get(countKey); @@ -346,14 +342,17 @@ public Response run(JerseyResourceDelegateContext context) { clientApplication.getCachedNonForwardResultScanner(queryIdObj, cacheId.longValue()); try { - skipOffsetRow(cachedQueryResultScanner, offset); - + int start_offset = cachedQueryResultScanner.getCurrentRowNumber(); List output = cachedQueryResultScanner.getNextRows(count); String digestString = getEncodedBase64DigestString(output); + boolean eos = count != output.size(); return Response.ok(new QueryResultStreamingOutput(output)) - .header(tajoDigestHeaderName, digestString) - .build(); + .header(tajoDigestHeaderName, digestString) + .header(tajoOffsetHeaderName, start_offset) + .header(tajoCountHeaderName, output.size()) + .header(tajoEOSHeaderName, eos) + .build(); } catch (IOException e) { LOG.error(e.getMessage(), e); @@ -365,20 +364,6 @@ public Response run(JerseyResourceDelegateContext context) { } } - private void skipOffsetRow(NonForwardQueryResultScanner queryResultScanner, int offset) throws IOException { - if (offset < 0) { - return; - } - - int currentRow = queryResultScanner.getCurrentRowNumber(); - - if (offset < (currentRow+1)) { - throw new RuntimeException("Offset must be over the current row number"); - } - - queryResultScanner.getNextRows(offset - currentRow - 1); - } - private String getEncodedBase64DigestString(List outputList) throws NoSuchAlgorithmException { MessageDigest messageDigest = MessageDigest.getInstance("SHA-1"); diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java index d83eb6064c..bbb080543c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java @@ -63,6 +63,9 @@ public class TestQueryResultResource extends QueryTestCaseBase { private static final String tajoSessionIdHeaderName = "X-Tajo-Session"; private static final String tajoDigestHeaderName = "X-Tajo-Digest"; + private static final String tajoOffsetHeaderName = "X-Tajo-Offset"; + private static final String tajoCountHeaderName = "X-Tajo-Count"; + private static final String tajoEOSHeaderName = "X-Tajo-EOS"; public TestQueryResultResource() { super(TajoConstants.DEFAULT_DATABASE_NAME); @@ -219,69 +222,6 @@ public void testGetQueryResultSet() throws Exception { } } - @Test - public void testGetQueryResultSetWithOffset() throws Exception { - String sessionId = generateNewSessionAndGetId(); - URI queryIdURI = sendNewQueryResquest(sessionId, "select * from lineitem"); - URI queryResultURI = new URI(queryIdURI + "/result"); - - GetQueryResultDataResponse response = restClient.target(queryResultURI) - .request().header(tajoSessionIdHeaderName, sessionId) - .get(new GenericType(GetQueryResultDataResponse.class)); - - assertNotNull(response); - assertNotNull(response.getResultCode()); - assertEquals(ResultCode.OK, response.getResultCode()); - assertNotNull(response.getSchema()); - assertEquals(16, response.getSchema().getRootColumns().size()); - assertNotNull(response.getResultset()); - assertTrue(response.getResultset().getId() != 0); - assertNotNull(response.getResultset().getLink()); - - URI queryResultSetURI = response.getResultset().getLink(); - - Response queryResultSetResponse = restClient.target(queryResultSetURI) - .queryParam("count", 100) - .queryParam("offset", 3) - .request().header(tajoSessionIdHeaderName, sessionId) - .get(); - - assertNotNull(queryResultSetResponse); - String tajoDigest = queryResultSetResponse.getHeaderString(tajoDigestHeaderName); - assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); - - DataInputStream queryResultSetInputStream = - new DataInputStream(new BufferedInputStream(queryResultSetResponse.readEntity(InputStream.class))); - - assertNotNull(queryResultSetInputStream); - - boolean isFinished = false; - List tupleList = TUtil.newList(); - RowStoreUtil.RowStoreDecoder decoder = RowStoreUtil.createDecoder(response.getSchema()); - MessageDigest messageDigest = MessageDigest.getInstance("SHA-1"); - while (!isFinished) { - try { - int length = queryResultSetInputStream.readInt(); - byte[] dataByteArray = new byte[length]; - int readBytes = queryResultSetInputStream.read(dataByteArray); - - assertEquals(length, readBytes); - - tupleList.add(decoder.toTuple(dataByteArray)); - messageDigest.update(dataByteArray); - } catch (EOFException eof) { - isFinished = true; - } - } - - assertEquals(3, tupleList.size()); - assertEquals(tajoDigest, Base64.encodeBase64String(messageDigest.digest())); - - for (Tuple aTuple: tupleList) { - assertTrue(aTuple.getInt4(response.getSchema().getColumnId("l_orderkey")) > 0); - } - } - @Test public void testGetQueryResultSetWithDefaultCount() throws Exception { String sessionId = generateNewSessionAndGetId(); @@ -309,7 +249,14 @@ public void testGetQueryResultSetWithDefaultCount() throws Exception { assertNotNull(queryResultSetResponse); String tajoDigest = queryResultSetResponse.getHeaderString(tajoDigestHeaderName); - assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); + int offset = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoOffsetHeaderName)); + int count = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoCountHeaderName)); + boolean eos = Boolean.valueOf(queryResultSetResponse.getHeaderString(tajoEOSHeaderName)); + + assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); + assertTrue(eos); + assertEquals(0, offset); + assertEquals(5, count); DataInputStream queryResultSetInputStream = new DataInputStream(new BufferedInputStream(queryResultSetResponse.readEntity(InputStream.class))); From 36da0dac7d90fa12b3d9cf11fa5b561a586e60ff Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 8 Jun 2015 16:52:05 +0900 Subject: [PATCH 2/7] TAJO-1615: Implement TaskManager. (jinho) Closes #595 --- CHANGES | 2 + .../physical/HashShuffleFileWriteExec.java | 2 +- .../engine/planner/physical/PhysicalExec.java | 2 +- .../tajo/master/rm/TajoResourceTracker.java | 4 +- .../util/TajoUncaughtExceptionHandler.java | 70 ++ .../tajo/util/history/HistoryWriter.java | 2 +- .../tajo/worker/ExecutionBlockContext.java | 83 +- .../apache/tajo/worker/LegacyTaskImpl.java | 844 ++++++++++++++++++ .../tajo/worker/NodeResourceManager.java | 45 +- .../apache/tajo/worker/NodeStatusUpdater.java | 34 +- .../org/apache/tajo/worker/TajoWorker.java | 47 +- .../tajo/worker/TajoWorkerManagerService.java | 9 +- .../java/org/apache/tajo/worker/Task.java | 834 +---------------- .../tajo/worker/TaskAttemptContext.java | 61 +- .../org/apache/tajo/worker/TaskContainer.java | 85 ++ .../org/apache/tajo/worker/TaskExecutor.java | 194 ++++ .../java/org/apache/tajo/worker/TaskImpl.java | 838 +++++++++++++++++ .../org/apache/tajo/worker/TaskManager.java | 180 ++++ .../org/apache/tajo/worker/TaskRunner.java | 10 +- .../apache/tajo/worker/TaskRunnerHistory.java | 1 + .../apache/tajo/worker/TaskRunnerManager.java | 12 +- .../event/ExecutionBlockStartEvent.java | 35 + .../worker/event/ExecutionBlockStopEvent.java | 37 + .../event/NodeResourceAllocateEvent.java | 2 +- .../event/NodeResourceDeallocateEvent.java | 2 +- ...nagerEvent.java => NodeResourceEvent.java} | 9 +- .../tajo/worker/event/NodeStatusEvent.java | 11 +- .../tajo/worker/event/TaskExecutorEvent.java | 44 + .../tajo/worker/event/TaskManagerEvent.java | 43 + .../tajo/worker/event/TaskRunnerEvent.java | 1 + .../worker/event/TaskRunnerStartEvent.java | 44 +- .../worker/event/TaskRunnerStopEvent.java | 1 + .../tajo/worker/event/TaskStartEvent.java | 44 + .../src/main/proto/TajoWorkerProtocol.proto | 1 + .../tajo/querymaster/TestKillQuery.java | 135 ++- .../tajo/worker/MockExecutionBlock.java | 42 + .../tajo/worker/MockNodeResourceManager.java | 96 ++ .../tajo/worker/MockNodeStatusUpdater.java | 4 +- .../apache/tajo/worker/MockTaskExecutor.java | 141 +++ .../apache/tajo/worker/MockTaskManager.java | 59 ++ .../apache/tajo/worker/MockWorkerContext.java | 129 +++ .../org/apache/tajo/worker/TestFetcher.java | 14 +- .../tajo/worker/TestNodeResourceManager.java | 135 ++- .../tajo/worker/TestNodeStatusUpdater.java | 54 +- .../apache/tajo/worker/TestTaskExecutor.java | 330 +++++++ .../apache/tajo/worker/TestTaskManager.java | 185 ++++ 46 files changed, 3874 insertions(+), 1083 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java rename tajo-core/src/main/java/org/apache/tajo/worker/event/{NodeResourceManagerEvent.java => NodeResourceEvent.java} (80%) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java diff --git a/CHANGES b/CHANGES index 44a8939504..066e08612b 100644 --- a/CHANGES +++ b/CHANGES @@ -333,6 +333,8 @@ Release 0.11.0 - unreleased SUB TASKS + TAJO-1615: Implement TaskManager. (jinho) + TAJO-1599: Implement NodeResourceManager and Status updater. (jinho) TAJO-1613: Rename StorageManager to Tablespace. (hyunsik) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java index f1e2fe5060..1a92a7a5f6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java @@ -86,7 +86,7 @@ private HashShuffleAppender getAppender(int partId) throws IOException { HashShuffleAppender appender = appenderMap.get(partId); if (appender == null) { appender = hashShuffleAppenderManager.getAppender(context.getConf(), - context.getQueryId().getTaskId().getExecutionBlockId(), partId, meta, outSchema); + context.getTaskId().getTaskId().getExecutionBlockId(), partId, meta, outSchema); appenderMap.put(partId, appender); } return appender; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java index de14c9a9d7..87a19a9f1d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java @@ -79,7 +79,7 @@ protected void fatal(Log log, String message) { } protected Path getExecutorTmpDir() { - return new Path(context.getQueryId().getTaskId().getExecutionBlockId().getQueryId().toString(), + return new Path(context.getTaskId().getTaskId().getExecutionBlockId().getQueryId().toString(), UUID.randomUUID().toString()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index af28886446..2a18de7ecb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -188,7 +188,9 @@ public void heartbeat( public void nodeHeartbeat(RpcController controller, TajoResourceTrackerProtocol.NodeHeartbeatRequestProto request, RpcCallback done) { //TODO implement with ResourceManager for scheduler - throw new RuntimeException(new ServiceException(new NotImplementedException().getMessage())); + TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.Builder + response = TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.newBuilder(); + done.run(response.setCommand(TajoResourceTrackerProtocol.ResponseCommand.NORMAL).build()); } private Worker createWorkerResource(NodeHeartbeat request) { diff --git a/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java b/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java new file mode 100644 index 0000000000..c424154ae8 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java @@ -0,0 +1,70 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.tajo.util; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.ShutdownHookManager; + +import java.lang.Thread.UncaughtExceptionHandler; + +/** + * This class is intended to be installed by calling + * {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)} + * In the main entry point. It is intended to try and cleanly shut down + * programs using the Yarn Event framework. + * + * Note: Right now it only will shut down the program if a Error is caught, but + * not any other exception. Anything else is just logged. + * + * this is an implementation copied from YarnUncaughtExceptionHandler + */ +public class TajoUncaughtExceptionHandler implements UncaughtExceptionHandler { + private static final Log LOG = LogFactory.getLog(TajoUncaughtExceptionHandler.class); + + @Override + public void uncaughtException(Thread t, Throwable e) { + if(ShutdownHookManager.get().isShutdownInProgress()) { + LOG.error("Thread " + t + " threw an Throwable, but we are shutting " + + "down, so ignoring this", e); + } else if(e instanceof Error) { + try { + LOG.fatal("Thread " + t + " threw an Error.", e); + } catch (Throwable err) { + //We don't want to not exit because of an issue with logging + } + + if(e instanceof OutOfMemoryError) { + //After catching an OOM java says it is undefined behavior, so don't + //even try to clean up or we can get stuck on shutdown. + try { + System.err.println("Halting due to Out Of Memory Error..."); + } catch (Throwable err) { + //Again we done want to exit because of logging issues. + } + ExitUtil.halt(-1); + } else { + //ExitUtil.terminate(-1); + } + } else { + LOG.error("Thread " + t + " threw an Exception.", e); + } + } +} 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 e8ba3046b9..daced3e90c 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 @@ -217,7 +217,7 @@ class WriterThread extends Thread { public void run() { LOG.info("HistoryWriter_" + processName + " started."); SimpleDateFormat df = new SimpleDateFormat("yyyyMMddHH"); - while (!stopped.get()) { + while (!stopped.get() && !Thread.interrupted()) { List> histories = Lists.newArrayList(); try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 0cc330468f..9e4a60f758 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -30,6 +30,7 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.TaskId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.QueryMasterProtocol; @@ -45,7 +46,6 @@ import java.net.ConnectException; import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; @@ -61,10 +61,10 @@ public class ExecutionBlockContext { private static final Log LOG = LogFactory.getLog(ExecutionBlockContext.class); private TaskRunnerManager manager; - public AtomicInteger completedTasksNum = new AtomicInteger(); - public AtomicInteger succeededTasksNum = new AtomicInteger(); - public AtomicInteger killedTasksNum = new AtomicInteger(); - public AtomicInteger failedTasksNum = new AtomicInteger(); + protected AtomicInteger completedTasksNum = new AtomicInteger(); + protected AtomicInteger succeededTasksNum = new AtomicInteger(); + protected AtomicInteger killedTasksNum = new AtomicInteger(); + protected AtomicInteger failedTasksNum = new AtomicInteger(); private FileSystem localFS; // for input files @@ -95,17 +95,18 @@ public class ExecutionBlockContext { // It keeps all of the query unit attempts while a TaskRunner is running. private final ConcurrentMap tasks = Maps.newConcurrentMap(); + @Deprecated private final ConcurrentMap histories = Maps.newConcurrentMap(); - public ExecutionBlockContext(TajoConf conf, TajoWorker.WorkerContext workerContext, - TaskRunnerManager manager, QueryContext queryContext, String plan, - ExecutionBlockId executionBlockId, WorkerConnectionInfo queryMaster, - PlanProto.ShuffleType shuffleType) throws Throwable { + private final Map taskHistories = Maps.newTreeMap(); + + public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, + TaskRunnerManager manager, RunExecutionBlockRequestProto request) throws IOException { this.manager = manager; - this.executionBlockId = executionBlockId; + this.executionBlockId = new ExecutionBlockId(request.getExecutionBlockId()); this.connManager = RpcClientManager.getInstance(); - this.queryMaster = queryMaster; - this.systemConf = conf; + this.queryMaster = new WorkerConnectionInfo(request.getQueryMaster()); + this.systemConf = workerContext.getConf(); this.reporter = new Reporter(); this.defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf); this.localFS = FileSystem.getLocal(systemConf); @@ -113,11 +114,11 @@ public ExecutionBlockContext(TajoConf conf, TajoWorker.WorkerContext workerConte // Setup QueryEngine according to the query plan // Here, we can setup row-based query engine or columnar query engine. this.queryEngine = new TajoQueryEngine(systemConf); - this.queryContext = queryContext; - this.plan = plan; + this.queryContext = new QueryContext(workerContext.getConf(), request.getQueryContext()); + this.plan = request.getPlanJson(); this.resource = new ExecutionBlockSharedResource(); this.workerContext = workerContext; - this.shuffleType = shuffleType; + this.shuffleType = request.getShuffleType(); } public void init() throws Throwable { @@ -131,7 +132,8 @@ public void init() throws Throwable { UserGroupInformation.setConfiguration(systemConf); // TODO - 'load credential' should be implemented // Getting taskOwner - UserGroupInformation taskOwner = UserGroupInformation.createRemoteUser(systemConf.getVar(TajoConf.ConfVars.USERNAME)); + UserGroupInformation + taskOwner = UserGroupInformation.createRemoteUser(systemConf.getVar(TajoConf.ConfVars.USERNAME)); // initialize DFS and LocalFileSystems this.taskOwner = taskOwner; @@ -144,7 +146,7 @@ public void init() throws Throwable { try { getStub().killQuery(null, executionBlockId.getQueryId().getProto(), NullCallback.get()); } catch (Throwable t) { - //ignore + LOG.error(t); } throw e; } @@ -183,9 +185,9 @@ public void stop(){ // If ExecutionBlock is stopped, all running or pending tasks will be marked as failed. for (Task task : tasks.values()) { - if (task.getStatus() == TajoProtos.TaskAttemptState.TA_PENDING || - task.getStatus() == TajoProtos.TaskAttemptState.TA_RUNNING) { - task.setState(TajoProtos.TaskAttemptState.TA_FAILED); + if (task.getTaskContext().getState() == TajoProtos.TaskAttemptState.TA_PENDING || + task.getTaskContext().getState() == TajoProtos.TaskAttemptState.TA_RUNNING) { + try{ task.abort(); } catch (Throwable e){ @@ -194,7 +196,7 @@ public void stop(){ } } tasks.clear(); - + taskHistories.clear(); resource.release(); RpcClientManager.cleanup(client); } @@ -253,18 +255,40 @@ public Task getTask(TaskAttemptId taskAttemptId){ return tasks.get(taskAttemptId); } + @Deprecated public void stopTaskRunner(String id){ manager.stopTaskRunner(id); } + @Deprecated public TaskRunner getTaskRunner(String taskRunnerId){ return manager.getTaskRunner(taskRunnerId); } + @Deprecated public void addTaskHistory(String taskRunnerId, TaskAttemptId quAttemptId, TaskHistory taskHistory) { histories.get(taskRunnerId).addTaskHistory(quAttemptId, taskHistory); } + public void addTaskHistory(TaskId taskId, TaskHistory taskHistory) { + taskHistories.put(taskId, taskHistory); + } + + public Map getTaskHistories() { + return taskHistories; + } + + public void fatalError(TaskAttemptId taskAttemptId, String message) { + if (message == null) { + message = "No error message"; + } + TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder() + .setId(taskAttemptId.getProto()) + .setErrorMessage(message); + + getStub().fatalError(null, builder.build(), NullCallback.get()); + } + public TaskRunnerHistory createTaskRunnerHistory(TaskRunner runner){ histories.putIfAbsent(runner.getId(), new TaskRunnerHistory(runner.getContainerId(), executionBlockId)); return histories.get(runner.getId()); @@ -355,7 +379,6 @@ private void sendHashShuffleReport(ExecutionBlockId ebId) throws Exception { protected class Reporter { private Thread reporterThread; - private AtomicBoolean reporterStop = new AtomicBoolean(); private static final int PROGRESS_INTERVAL = 1000; private static final int MAX_RETRIES = 10; @@ -374,7 +397,7 @@ Runnable createReporterThread() { int remainingRetries = MAX_RETRIES; @Override public void run() { - while (!reporterStop.get() && !Thread.interrupted()) { + while (!isStopped() && !Thread.interrupted()) { try { Interface masterStub = getStub(); @@ -384,13 +407,11 @@ public void run() { } else { for (Task task : new ArrayList(tasks.values())){ - if (task.isRunning() && task.isProgressChanged()) { - task.updateProgress(); + if (task.getTaskContext().getState() == + TajoProtos.TaskAttemptState.TA_RUNNING && task.isProgressChanged()) { masterStub.statusUpdate(null, task.getReport(), NullCallback.get()); - task.getContext().setProgressChanged(false); - } else { - task.updateProgress(); } + task.updateProgress(); } } } catch (Throwable t) { @@ -402,7 +423,7 @@ public void run() { throw new RuntimeException(t); } } finally { - if (remainingRetries > 0 && !reporterStop.get()) { + if (remainingRetries > 0 && !isStopped()) { synchronized (reporterThread) { try { reporterThread.wait(PROGRESS_INTERVAL); @@ -417,10 +438,6 @@ public void run() { } public void stop() throws InterruptedException { - if (reporterStop.getAndSet(true)) { - return; - } - if (reporterThread != null) { // Intent of the lock is to not send an interupt in the middle of an // umbilical.ping or umbilical.statusUpdate diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java new file mode 100644 index 0000000000..0721ef162e --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java @@ -0,0 +1,844 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.netty.handler.codec.http.QueryStringDecoder; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.TaskAttemptState; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.planner.physical.PhysicalExec; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.ipc.QueryMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.*; +import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.TajoScriptEngine; +import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.serder.LogicalNodeDeserializer; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.rpc.NullCallback; +import org.apache.tajo.storage.*; +import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.util.NetUtils; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.URI; +import java.util.*; +import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; + +import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; + +@Deprecated +public class LegacyTaskImpl implements Task { + private static final Log LOG = LogFactory.getLog(LegacyTaskImpl.class); + private static final float FETCHER_PROGRESS = 0.5f; + + private final TajoConf systemConf; + private final QueryContext queryContext; + private final ExecutionBlockContext executionBlockContext; + private final String taskRunnerId; + + private final Path taskDir; + private final TaskRequest request; + private TaskAttemptContext context; + private List fetcherRunners; + private LogicalNode plan; + private final Map descs = Maps.newHashMap(); + private PhysicalExec executor; + private boolean interQuery; + private Path inputTableBaseDir; + + private long startTime; + private long finishTime; + + private final TableStats inputStats; + private List localChunks; + + // TODO - to be refactored + private ShuffleType shuffleType = null; + private Schema finalSchema = null; + private TupleComparator sortComp = null; + + public LegacyTaskImpl(String taskRunnerId, + Path baseDir, + TaskAttemptId taskId, + final ExecutionBlockContext executionBlockContext, + final TaskRequest request) throws IOException { + this(taskRunnerId, baseDir, taskId, executionBlockContext.getConf(), executionBlockContext, request); + } + + public LegacyTaskImpl(String taskRunnerId, + Path baseDir, + TaskAttemptId taskId, + TajoConf conf, + final ExecutionBlockContext executionBlockContext, + final TaskRequest request) throws IOException { + this.taskRunnerId = taskRunnerId; + this.request = request; + + this.systemConf = conf; + this.queryContext = request.getQueryContext(systemConf); + this.executionBlockContext = executionBlockContext; + this.taskDir = StorageUtil.concatPath(baseDir, + taskId.getTaskId().getId() + "_" + taskId.getId()); + + this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskId, + request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); + this.context.setDataChannel(request.getDataChannel()); + this.context.setEnforcer(request.getEnforcer()); + this.context.setState(TaskAttemptState.TA_PENDING); + this.inputStats = new TableStats(); + this.fetcherRunners = Lists.newArrayList(); + } + + public void initPlan() throws IOException { + plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); + LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); + if (scanNode != null) { + for (LogicalNode node : scanNode) { + ScanNode scan = (ScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); + if (partitionScanNode != null) { + for (LogicalNode node : partitionScanNode) { + PartitionedTableScanNode scan = (PartitionedTableScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + interQuery = request.getProto().getInterQuery(); + if (interQuery) { + context.setInterQuery(); + this.shuffleType = context.getDataChannel().getShuffleType(); + + if (shuffleType == ShuffleType.RANGE_SHUFFLE) { + SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); + this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); + this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); + } + } else { + Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) + .getAppenderFilePath(getId(), queryContext.getStagingDir()); + LOG.info("Output File Path: " + outFilePath); + context.setOutputPath(outFilePath); + } + + this.localChunks = Collections.synchronizedList(new ArrayList()); + LOG.info("=================================="); + LOG.info("* Stage " + request.getId() + " is initialized"); + LOG.info("* InterQuery: " + interQuery + + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + + ", Fragments (num: " + request.getFragments().size() + ")" + + ", Fetches (total:" + request.getFetches().size() + ") :"); + + if(LOG.isDebugEnabled()) { + for (FetchImpl f : request.getFetches()) { + LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); + } + } + LOG.info("* Local task dir: " + taskDir); + if(LOG.isDebugEnabled()) { + LOG.debug("* plan:\n"); + LOG.debug(plan.toString()); + } + LOG.info("=================================="); + } + + private void startScriptExecutors() throws IOException { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.start(systemConf); + } + } + + private void stopScriptExecutors() { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.shutdown(); + } + } + + @Override + public void init() throws IOException { + initPlan(); + startScriptExecutors(); + + if (context.getState() == TaskAttemptState.TA_PENDING) { + // initialize a task temporal dir + FileSystem localFS = executionBlockContext.getLocalFS(); + localFS.mkdirs(taskDir); + + if (request.getFetches().size() > 0) { + inputTableBaseDir = localFS.makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( + getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); + localFS.mkdirs(inputTableBaseDir); + Path tableDir; + for (String inputTable : context.getInputTables()) { + tableDir = new Path(inputTableBaseDir, inputTable); + if (!localFS.exists(tableDir)) { + LOG.info("the directory is created " + tableDir.toUri()); + localFS.mkdirs(tableDir); + } + } + } + // for localizing the intermediate data + fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); + } + } + + private TaskAttemptId getId() { + return context.getTaskId(); + } + + public String toString() { + return "queryId: " + this.getId() + " status: " + context.getState(); + } + + @Override + public boolean isStopped() { + return context.isStopped(); + } + + @Override + public TaskAttemptContext getTaskContext() { + return context; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return executionBlockContext; + } + + @Override + public boolean hasFetchPhase() { + return fetcherRunners.size() > 0; + } + + @Override + public void fetch() { + ExecutorService executorService = executionBlockContext.getTaskRunner(taskRunnerId).getFetchLauncher(); + for (Fetcher f : fetcherRunners) { + executorService.submit(new FetchRunner(context, f)); + } + } + + @Override + public void kill() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_KILLED); + context.stop(); + } + + @Override + public void abort() { + stopScriptExecutors(); + context.setState(TajoProtos.TaskAttemptState.TA_FAILED); + context.stop(); + } + + @Override + public TaskStatusProto getReport() { + TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); + builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); + builder.setId(context.getTaskId().getProto()) + .setProgress(context.getProgress()) + .setState(context.getState()); + + builder.setInputStats(reloadInputStats()); + + if (context.getResultStats() != null) { + builder.setResultStats(context.getResultStats().getProto()); + } + return builder.build(); + } + + @Override + public boolean isProgressChanged() { + return context.isProgressChanged(); + } + + @Override + public void updateProgress() { + if(context != null && context.isStopped()){ + return; + } + + if (executor != null && context.getProgress() < 1.0f) { + context.setExecutorProgress(executor.getProgress()); + } + } + + private CatalogProtos.TableStatsProto reloadInputStats() { + synchronized(inputStats) { + if (this.executor == null) { + return inputStats.getProto(); + } + + TableStats executorInputStats = this.executor.getInputStats(); + + if (executorInputStats != null) { + inputStats.setValues(executorInputStats); + } + return inputStats.getProto(); + } + } + + private TaskCompletionReport getTaskCompletionReport() { + TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); + builder.setId(context.getTaskId().getProto()); + + builder.setInputStats(reloadInputStats()); + + if (context.hasResultStats()) { + builder.setResultStats(context.getResultStats().getProto()); + } else { + builder.setResultStats(new TableStats().getProto()); + } + + Iterator> it = context.getShuffleFileOutputs(); + if (it.hasNext()) { + do { + Entry entry = it.next(); + ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); + part.setPartId(entry.getKey()); + + // Set output volume + if (context.getPartitionOutputVolume() != null) { + for (Entry e : context.getPartitionOutputVolume().entrySet()) { + if (entry.getKey().equals(e.getKey())) { + part.setVolume(e.getValue().longValue()); + break; + } + } + } + + builder.addShuffleFileOutputs(part.build()); + } while (it.hasNext()); + } + + return builder.build(); + } + + private void waitForFetch() throws InterruptedException, IOException { + context.getFetchLatch().await(); + LOG.info(context.getTaskId() + " All fetches are done!"); + Collection inputs = Lists.newArrayList(context.getInputTables()); + + // Get all broadcasted tables + Set broadcastTableNames = new HashSet(); + List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); + if (broadcasts != null) { + for (EnforceProperty eachBroadcast : broadcasts) { + broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); + } + } + + // localize the fetched data and skip the broadcast table + for (String inputTable: inputs) { + if (broadcastTableNames.contains(inputTable)) { + continue; + } + File tableDir = new File(context.getFetchIn(), inputTable); + FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); + context.updateAssignedFragments(inputTable, frags); + } + } + + @Override + public void run() throws Exception { + startTime = System.currentTimeMillis(); + Throwable error = null; + try { + if(!context.isStopped()) { + context.setState(TaskAttemptState.TA_RUNNING); + if (context.hasFetchPhase()) { + // If the fetch is still in progress, the query unit must wait for + // complete. + waitForFetch(); + context.setFetcherProgress(FETCHER_PROGRESS); + updateProgress(); + } + + this.executor = executionBlockContext.getTQueryEngine(). + createPlan(context, plan); + this.executor.init(); + + while(!context.isStopped() && executor.next() != null) { + } + } + } catch (Throwable e) { + error = e ; + LOG.error(e.getMessage(), e); + stopScriptExecutors(); + context.stop(); + } finally { + if (executor != null) { + try { + executor.close(); + reloadInputStats(); + } catch (IOException e) { + LOG.error(e, e); + } + this.executor = null; + } + + executionBlockContext.completedTasksNum.incrementAndGet(); + context.getHashShuffleAppenderManager().finalizeTask(getId()); + + QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); + if (context.isStopped()) { + context.setExecutorProgress(0.0f); + + if (context.getState() == TaskAttemptState.TA_KILLED) { + queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); + executionBlockContext.killedTasksNum.incrementAndGet(); + } else { + context.setState(TaskAttemptState.TA_FAILED); + TaskFatalErrorReport.Builder errorBuilder = + TaskFatalErrorReport.newBuilder() + .setId(getId().getProto()); + if (error != null) { + if (error.getMessage() == null) { + errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); + } else { + errorBuilder.setErrorMessage(error.getMessage()); + } + errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); + } + + queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); + executionBlockContext.failedTasksNum.incrementAndGet(); + } + } else { + // if successful + context.stop(); + context.setProgress(1.0f); + context.setState(TaskAttemptState.TA_SUCCEEDED); + executionBlockContext.succeededTasksNum.incrementAndGet(); + + TaskCompletionReport report = getTaskCompletionReport(); + queryMasterStub.done(null, report, NullCallback.get()); + } + finishTime = System.currentTimeMillis(); + LOG.info(context.getTaskId() + " completed. " + + "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + + ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() + + ", killed: " + executionBlockContext.killedTasksNum.intValue() + + ", failed: " + executionBlockContext.failedTasksNum.intValue()); + } + } + + @Override + public void cleanup() { + TaskHistory taskHistory = createTaskHistory(); + executionBlockContext.addTaskHistory(taskRunnerId, getId(), taskHistory); + executionBlockContext.getTasks().remove(getId()); + + fetcherRunners.clear(); + fetcherRunners = null; + try { + if(executor != null) { + executor.close(); + executor = null; + } + } catch (IOException e) { + LOG.fatal(e.getMessage(), e); + } + + executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); + stopScriptExecutors(); + } + + public TaskHistory createTaskHistory() { + TaskHistory taskHistory = null; + try { + taskHistory = new TaskHistory(context.getTaskId(), context.getState(), context.getProgress(), + startTime, finishTime, reloadInputStats()); + + if (context.getOutputPath() != null) { + taskHistory.setOutputPath(context.getOutputPath().toString()); + } + + if (context.getWorkDir() != null) { + taskHistory.setWorkingPath(context.getWorkDir().toString()); + } + + if (context.getResultStats() != null) { + taskHistory.setOutputStats(context.getResultStats().getProto()); + } + + if (hasFetchPhase()) { + taskHistory.setTotalFetchCount(fetcherRunners.size()); + int i = 0; + FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); + for (Fetcher fetcher : fetcherRunners) { + // TODO store the fetcher histories + if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { + builder.setStartTime(fetcher.getStartTime()); + builder.setFinishTime(fetcher.getFinishTime()); + builder.setFileLength(fetcher.getFileLen()); + builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); + builder.setState(fetcher.getState()); + + taskHistory.addFetcherHistory(builder.build()); + } + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; + } + taskHistory.setFinishedFetchCount(i); + } + } catch (Exception e) { + LOG.warn(e.getMessage(), e); + } + + return taskHistory; + } + + public int hashCode() { + return context.hashCode(); + } + + public boolean equals(Object obj) { + if (obj instanceof LegacyTaskImpl) { + LegacyTaskImpl other = (LegacyTaskImpl) obj; + return this.context.equals(other.context); + } + return false; + } + + private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) + throws IOException { + Configuration c = new Configuration(systemConf); + c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); + FileSystem fs = FileSystem.get(c); + Path tablePath = new Path(file.getAbsolutePath()); + + List listTablets = new ArrayList(); + FileFragment tablet; + + FileStatus[] fileLists = fs.listStatus(tablePath); + for (FileStatus f : fileLists) { + if (f.getLen() == 0) { + continue; + } + tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); + listTablets.add(tablet); + } + + // Special treatment for locally pseudo fetched chunks + synchronized (localChunks) { + for (FileChunk chunk : localChunks) { + if (name.equals(chunk.getEbId())) { + tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); + listTablets.add(tablet); + LOG.info("One local chunk is added to listTablets"); + } + } + } + + FileFragment[] tablets = new FileFragment[listTablets.size()]; + listTablets.toArray(tablets); + + return tablets; + } + + private class FetchRunner implements Runnable { + private final TaskAttemptContext ctx; + private final Fetcher fetcher; + private int maxRetryNum; + + public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { + this.ctx = ctx; + this.fetcher = fetcher; + this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); + } + + @Override + public void run() { + int retryNum = 0; + int retryWaitTime = 1000; //sec + + try { // for releasing fetch latch + while(!context.isStopped() && retryNum < maxRetryNum) { + if (retryNum > 0) { + try { + Thread.sleep(retryWaitTime); + retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds + } catch (InterruptedException e) { + LOG.error(e); + } + LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); + } + try { + FileChunk fetched = fetcher.get(); + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null + && fetched.getFile() != null) { + if (fetched.fromRemote() == false) { + localChunks.add(fetched); + LOG.info("Add a new FileChunk to local chunk list"); + } + break; + } + } catch (Throwable e) { + LOG.error("Fetch failed: " + fetcher.getURI(), e); + } + retryNum++; + } + } finally { + if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ + fetcherFinished(ctx); + } else { + if (retryNum == maxRetryNum) { + LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); + } + stopScriptExecutors(); + context.stop(); // retry task + ctx.getFetchLatch().countDown(); + } + } + } + } + + @VisibleForTesting + public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { + if (totalFetcher > 0) { + return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; + } else { + return 0.0f; + } + } + + private synchronized void fetcherFinished(TaskAttemptContext ctx) { + int fetcherSize = fetcherRunners.size(); + if(fetcherSize == 0) { + return; + } + + ctx.getFetchLatch().countDown(); + + int remainFetcher = (int) ctx.getFetchLatch().getCount(); + if (remainFetcher == 0) { + context.setFetcherProgress(FETCHER_PROGRESS); + } else { + context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); + } + } + + private List getFetchRunners(TaskAttemptContext ctx, + List fetches) throws IOException { + + if (fetches.size() > 0) { + Path inputDir = executionBlockContext.getLocalDirAllocator(). + getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); + + int i = 0; + File storeDir; + File defaultStoreFile; + FileChunk storeChunk = null; + List runnerList = Lists.newArrayList(); + + for (FetchImpl f : fetches) { + storeDir = new File(inputDir.toString(), f.getName()); + if (!storeDir.exists()) { + storeDir.mkdirs(); + } + + for (URI uri : f.getURIs()) { + defaultStoreFile = new File(storeDir, "in_" + i); + InetAddress address = InetAddress.getByName(uri.getHost()); + + WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); + if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { + boolean hasError = false; + try { + LOG.info("Try to get local file chunk at local host"); + storeChunk = getLocalStoredFileChunk(uri, systemConf); + } catch (Throwable t) { + hasError = true; + } + + // When a range request is out of range, storeChunk will be NULL. This case is normal state. + // So, we should skip and don't need to create storeChunk. + if (storeChunk == null && !hasError) { + continue; + } + + if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 + && hasError == false) { + storeChunk.setFromRemote(false); + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + + // If we decide that intermediate data should be really fetched from a remote host, storeChunk + // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it + storeChunk.setEbId(f.getName()); + Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); + LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); + runnerList.add(fetcher); + i++; + } + } + ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); + return runnerList; + } else { + return Lists.newArrayList(); + } + } + + private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { + // Parse the URI + LOG.info("getLocalStoredFileChunk starts"); + final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); + final List types = params.get("type"); + final List qids = params.get("qid"); + final List taskIdList = params.get("ta"); + final List stageIds = params.get("sid"); + final List partIds = params.get("p"); + final List offsetList = params.get("offset"); + final List lengthList = params.get("length"); + + 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 || 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 = stageIds.get(0); + String partId = partIds.get(0); + + if (shuffleType.equals("r") && taskIdList == null) { + LOG.error("Invalid URI - For range shuffle, taskId is required"); + return null; + } + List taskIds = splitMaps(taskIdList); + + FileChunk chunk = null; + long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; + long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; + + LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId + + ", taskIds=" + taskIdList); + + // The working directory of Tajo worker for each query, including stage + String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; + + // If the stage requires a range shuffle + if (shuffleType.equals("r")) { + String ta = taskIds.get(0); + if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { + LOG.warn("Range shuffle - file not exist"); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); + String startKey = params.get("start").get(0); + String endKey = params.get("end").get(0); + boolean last = params.get("final") != null; + + try { + chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); + } catch (Throwable t) { + LOG.error("getFileChunks() throws exception"); + return null; + } + + // 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; + if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { + LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); + File file = new File(path.toUri()); + long startPos = (offset >= 0 && length >= 0) ? offset : 0; + long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); + return null; + } + chunk = new FileChunk(file, startPos, readLen); + + } else { + LOG.error("Unknown shuffle type"); + return null; + } + + return chunk; + } + + private List splitMaps(List mapq) { + if (null == mapq) { + return null; + } + final List ret = new ArrayList(); + for (String s : mapq) { + Collections.addAll(ret, s.split(",")); + } + return ret; + } + + public static Path getTaskAttemptDir(TaskAttemptId quid) { + Path workDir = + StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), + String.valueOf(quid.getTaskId().getId()), + String.valueOf(quid.getId())); + return workDir; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index 20eec6b4e2..e763d13b4f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -30,27 +30,23 @@ import org.apache.tajo.storage.DiskUtil; import org.apache.tajo.unit.StorageUnit; import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.worker.event.NodeResourceAllocateEvent; -import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; -import org.apache.tajo.worker.event.NodeResourceManagerEvent; -import org.apache.tajo.worker.event.NodeStatusEvent; - -import java.util.concurrent.atomic.AtomicInteger; +import org.apache.tajo.worker.event.*; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; -public class NodeResourceManager extends AbstractService implements EventHandler { +public class NodeResourceManager extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(NodeResourceManager.class); private final Dispatcher dispatcher; + private final EventHandler taskEventHandler; private NodeResource totalResource; private NodeResource availableResource; - private AtomicInteger allocatedSize; private TajoConf tajoConf; - public NodeResourceManager(Dispatcher dispatcher){ + public NodeResourceManager(Dispatcher dispatcher, EventHandler taskEventHandler) { super(NodeResourceManager.class.getName()); this.dispatcher = dispatcher; + this.taskEventHandler = taskEventHandler; } @Override @@ -61,14 +57,14 @@ protected void serviceInit(Configuration conf) throws Exception { this.tajoConf = (TajoConf)conf; this.totalResource = createWorkerResource(tajoConf); this.availableResource = NodeResources.clone(totalResource); - this.dispatcher.register(NodeResourceManagerEvent.EventType.class, this); - this.allocatedSize = new AtomicInteger(); + this.dispatcher.register(NodeResourceEvent.EventType.class, this); + super.serviceInit(conf); LOG.info("Initialized NodeResourceManager for " + totalResource); } @Override - public void handle(NodeResourceManagerEvent event) { + public void handle(NodeResourceEvent event) { if (event instanceof NodeResourceAllocateEvent) { NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; @@ -76,22 +72,27 @@ public void handle(NodeResourceManagerEvent event) { for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { - allocatedSize.incrementAndGet(); - //TODO send task event to taskExecutor + if(allocateEvent.getRequest().hasExecutionBlockRequest()){ + //send ExecutionBlock start event to TaskManager + startExecutionBlock(allocateEvent.getRequest().getExecutionBlockRequest()); + } + + //send task start event to TaskExecutor + startTask(request.getTaskRequest(), resource); } else { + // reject the exceeded requests response.addCancellationTask(request); } } allocateEvent.getCallback().run(response.build()); } else if (event instanceof NodeResourceDeallocateEvent) { - allocatedSize.decrementAndGet(); NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; release(deallocateEvent.getResource()); // send current resource to ResourceTracker getDispatcher().getEventHandler().handle( - new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, getAvailableResource())); + new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); } } @@ -107,10 +108,6 @@ protected NodeResource getAvailableResource() { return availableResource; } - public int getAllocatedSize() { - return allocatedSize.get(); - } - private boolean allocate(NodeResource resource) { //TODO consider the jvm free memory if (NodeResources.fitsIn(resource, availableResource)) { @@ -120,6 +117,14 @@ private boolean allocate(NodeResource resource) { return false; } + protected void startExecutionBlock(RunExecutionBlockRequestProto request) { + taskEventHandler.handle(new ExecutionBlockStartEvent(request)); + } + + protected void startTask(TaskRequestProto request, NodeResource resource) { + taskEventHandler.handle(new TaskStartEvent(request, resource)); + } + private void release(NodeResource resource) { NodeResources.addTo(availableResource, resource); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 84ac419066..d13cd50a32 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -57,16 +57,16 @@ public class NodeStatusUpdater extends AbstractService implements EventHandler heartBeatRequestQueue; - private final WorkerConnectionInfo connectionInfo; + private final TajoWorker.WorkerContext workerContext; private final NodeResourceManager nodeResourceManager; private AsyncRpcClient rmClient; private ServiceTracker serviceTracker; private TajoResourceTrackerProtocolService.Interface resourceTracker; private int queueingLimit; - public NodeStatusUpdater(WorkerConnectionInfo connectionInfo, NodeResourceManager resourceManager) { + public NodeStatusUpdater(TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { super(NodeStatusUpdater.class.getSimpleName()); - this.connectionInfo = connectionInfo; + this.workerContext = workerContext; this.nodeResourceManager = resourceManager; } @@ -99,7 +99,8 @@ public void serviceStop() throws Exception { this.isStopped = true; synchronized (updaterThread) { - updaterThread.notifyAll(); + updaterThread.interrupt(); + updaterThread.join(); } super.serviceStop(); LOG.info("NodeStatusUpdater stopped."); @@ -107,14 +108,7 @@ public void serviceStop() throws Exception { @Override public void handle(NodeStatusEvent event) { - switch (event.getType()) { - case REPORT_RESOURCE: - heartBeatRequestQueue.add(event); //batch report to ResourceTracker - break; - case FLUSH_REPORTS: - heartBeatRequestQueue.add(event); //flush report to ResourceTracker - break; - } + heartBeatRequestQueue.add(event); } public int getQueueSize() { @@ -128,13 +122,13 @@ public int getQueueingLimit() { private NodeHeartbeatRequestProto createResourceReport(NodeResource resource) { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setAvailableResource(resource.getProto()); - requestProto.setWorkerId(connectionInfo.getId()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); return requestProto.build(); } private NodeHeartbeatRequestProto createHeartBeatReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setWorkerId(connectionInfo.getId()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); return requestProto.build(); } @@ -142,8 +136,8 @@ private NodeHeartbeatRequestProto createNodeStatusReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setTotalResource(nodeResourceManager.getTotalResource().getProto()); requestProto.setAvailableResource(nodeResourceManager.getAvailableResource().getProto()); - requestProto.setWorkerId(connectionInfo.getId()); - requestProto.setConnectionInfo(connectionInfo.getProto()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); + requestProto.setConnectionInfo(workerContext.getConnectionInfo().getProto()); //TODO set node status to requestProto.setStatus() return requestProto.build(); @@ -231,8 +225,8 @@ public void run() { } if (!events.isEmpty()) { - // send last available resource; - lastResponse = sendHeartbeat(createResourceReport(events.get(events.size() - 1).getResource())); + // send current available resource; + lastResponse = sendHeartbeat(createResourceReport(nodeResourceManager.getAvailableResource())); } else { // send ping; lastResponse = sendHeartbeat(createHeartBeatReport()); @@ -250,10 +244,10 @@ public void run() { } } catch (NoSuchMethodException nsme) { LOG.fatal(nsme.getMessage(), nsme); - Runtime.getRuntime().halt(1); + Runtime.getRuntime().halt(-1); } catch (ClassNotFoundException cnfe) { LOG.fatal(cnfe.getMessage(), cnfe); - Runtime.getRuntime().halt(1); + Runtime.getRuntime().halt(-1); } catch (Exception e) { LOG.error(e.getMessage(), e); if (!isStopped) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 4f07ca67d7..fbd070e5d0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -163,7 +163,7 @@ public void serviceInit(Configuration conf) throws Exception { serviceTracker = ServiceTrackerFactory.get(systemConf); - this.workerContext = new WorkerContext(); + this.workerContext = new TajoWorkerContext(); this.lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); String resourceManagerClassName = systemConf.getVar(ConfVars.RESOURCE_MANAGER_CLASS); @@ -386,7 +386,45 @@ public void serviceStop() throws Exception { LOG.info("TajoWorker main thread exiting"); } - public class WorkerContext { + public interface WorkerContext { + QueryMaster getQueryMaster(); + + TajoConf getConf(); + + ServiceTracker getServiceTracker(); + + QueryMasterManagerService getQueryMasterManagerService(); + + TaskRunnerManager getTaskRunnerManager(); + + CatalogService getCatalog(); + + WorkerConnectionInfo getConnectionInfo(); + + String getWorkerName(); + + LocalDirAllocator getLocalDirAllocator(); + + ClusterResourceSummary getClusterResource(); + + TajoSystemMetrics getWorkerSystemMetrics(); + + HashShuffleAppenderManager getHashShuffleAppenderManager(); + + HistoryWriter getTaskHistoryWriter(); + + HistoryReader getHistoryReader(); + + void cleanup(String strPath); + + void cleanupTemporalDirectories(); + + void setClusterResource(ClusterResourceSummary clusterResource); + + void setNumClusterNodes(int numClusterNodes); + } + + class TajoWorkerContext implements WorkerContext { public QueryMaster getQueryMaster() { if (queryMasterManagerService == null) { return null; @@ -430,7 +468,7 @@ public LocalDirAllocator getLocalDirAllocator(){ return lDirAllocator; } - protected void cleanup(String strPath) { + public void cleanup(String strPath) { if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); @@ -446,7 +484,7 @@ protected void cleanup(String strPath) { } } - protected void cleanupTemporalDirectories() { + public void cleanupTemporalDirectories() { if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); @@ -627,6 +665,7 @@ public static List getMountPath() throws IOException { } public static void main(String[] args) throws Exception { + Thread.setDefaultUncaughtExceptionHandler(new TajoUncaughtExceptionHandler()); StringUtils.startupShutdownMessage(TajoWorker.class, args, LOG); TajoConf tajoConf = new TajoConf(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index bbf8564566..de8afe8377 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -116,14 +116,7 @@ public void startExecutionBlock(RpcController controller, workerContext.getWorkerSystemMetrics().counter("query", "executedExecutionBlocksNum").inc(); try { - workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStartEvent( - new WorkerConnectionInfo(request.getQueryMaster()) - , new ExecutionBlockId(request.getExecutionBlockId()) - , request.getContainerId() - , new QueryContext(workerContext.getConf(), request.getQueryContext()), - request.getPlanJson(), - request.getShuffleType() - )); + workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStartEvent(request)); done.run(TajoWorker.TRUE_PROTO); } catch (Throwable t) { LOG.error(t.getMessage(), t); 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 4716dccad7..c84994003e 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 @@ -18,839 +18,35 @@ package org.apache.tajo.worker; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import io.netty.handler.codec.http.QueryStringDecoder; -import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.tajo.TajoProtos; -import org.apache.tajo.TajoProtos.TaskAttemptState; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.catalog.TableDesc; -import org.apache.tajo.catalog.TableMeta; -import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.planner.physical.PhysicalExec; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.engine.query.TaskRequest; -import org.apache.tajo.ipc.QueryMasterProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.*; -import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.function.python.TajoScriptEngine; -import org.apache.tajo.plan.logical.*; -import org.apache.tajo.plan.serder.LogicalNodeDeserializer; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.pullserver.TajoPullServerService; -import org.apache.tajo.pullserver.retriever.FileChunk; -import org.apache.tajo.rpc.NullCallback; -import org.apache.tajo.storage.*; -import org.apache.tajo.storage.fragment.FileFragment; -import org.apache.tajo.util.NetUtils; +import org.apache.tajo.ipc.TajoWorkerProtocol; -import java.io.File; import java.io.IOException; -import java.net.InetAddress; -import java.net.URI; -import java.util.*; -import java.util.Map.Entry; -import java.util.concurrent.ExecutorService; -import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; -import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; +public interface Task { -public class Task { - private static final Log LOG = LogFactory.getLog(Task.class); - private static final float FETCHER_PROGRESS = 0.5f; + void init() throws IOException; - private final TajoConf systemConf; - private final QueryContext queryContext; - private final ExecutionBlockContext executionBlockContext; - private final TaskAttemptId taskId; - private final String taskRunnerId; + void fetch(); - private final Path taskDir; - private final TaskRequest request; - private TaskAttemptContext context; - private List fetcherRunners; - private LogicalNode plan; - private final Map descs = Maps.newHashMap(); - private PhysicalExec executor; - private boolean interQuery; - private Path inputTableBaseDir; + void run() throws Exception; - private long startTime; - private long finishTime; + void kill(); - private final TableStats inputStats; - private List localChunks; + void abort(); - // TODO - to be refactored - private ShuffleType shuffleType = null; - private Schema finalSchema = null; - private TupleComparator sortComp = null; + void cleanup(); - public Task(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { - this(taskRunnerId, baseDir, taskId, executionBlockContext.getConf(), executionBlockContext, request); - } + boolean hasFetchPhase(); - public Task(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - TajoConf conf, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { - this.taskRunnerId = taskRunnerId; - this.request = request; - this.taskId = taskId; + boolean isProgressChanged(); - this.systemConf = conf; - this.queryContext = request.getQueryContext(systemConf); - this.executionBlockContext = executionBlockContext; - this.taskDir = StorageUtil.concatPath(baseDir, - taskId.getTaskId().getId() + "_" + taskId.getId()); + boolean isStopped(); - this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskId, - request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); - this.context.setDataChannel(request.getDataChannel()); - this.context.setEnforcer(request.getEnforcer()); - this.context.setState(TaskAttemptState.TA_PENDING); - this.inputStats = new TableStats(); - this.fetcherRunners = Lists.newArrayList(); - } + void updateProgress(); - public void initPlan() throws IOException { - plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); - LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); - if (scanNode != null) { - for (LogicalNode node : scanNode) { - ScanNode scan = (ScanNode) node; - descs.put(scan.getCanonicalName(), scan.getTableDesc()); - } - } + TaskAttemptContext getTaskContext(); - LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); - if (partitionScanNode != null) { - for (LogicalNode node : partitionScanNode) { - PartitionedTableScanNode scan = (PartitionedTableScanNode) node; - descs.put(scan.getCanonicalName(), scan.getTableDesc()); - } - } + ExecutionBlockContext getExecutionBlockContext(); - interQuery = request.getProto().getInterQuery(); - if (interQuery) { - context.setInterQuery(); - this.shuffleType = context.getDataChannel().getShuffleType(); - - if (shuffleType == ShuffleType.RANGE_SHUFFLE) { - SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); - this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); - this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); - } - } else { - Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) - .getAppenderFilePath(taskId, queryContext.getStagingDir()); - LOG.info("Output File Path: " + outFilePath); - context.setOutputPath(outFilePath); - } - - this.localChunks = Collections.synchronizedList(new ArrayList()); - LOG.info("=================================="); - LOG.info("* Stage " + request.getId() + " is initialized"); - LOG.info("* InterQuery: " + interQuery - + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + - ", Fragments (num: " + request.getFragments().size() + ")" + - ", Fetches (total:" + request.getFetches().size() + ") :"); - - if(LOG.isDebugEnabled()) { - for (FetchImpl f : request.getFetches()) { - LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); - } - } - LOG.info("* Local task dir: " + taskDir); - if(LOG.isDebugEnabled()) { - LOG.debug("* plan:\n"); - LOG.debug(plan.toString()); - } - LOG.info("=================================="); - } - - private void startScriptExecutors() throws IOException { - for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.start(systemConf); - } - } - - private void stopScriptExecutors() { - for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.shutdown(); - } - } - - public void init() throws IOException { - initPlan(); - startScriptExecutors(); - - if (context.getState() == TaskAttemptState.TA_PENDING) { - // initialize a task temporal dir - FileSystem localFS = executionBlockContext.getLocalFS(); - localFS.mkdirs(taskDir); - - if (request.getFetches().size() > 0) { - inputTableBaseDir = localFS.makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( - getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); - localFS.mkdirs(inputTableBaseDir); - Path tableDir; - for (String inputTable : context.getInputTables()) { - tableDir = new Path(inputTableBaseDir, inputTable); - if (!localFS.exists(tableDir)) { - LOG.info("the directory is created " + tableDir.toUri()); - localFS.mkdirs(tableDir); - } - } - } - // for localizing the intermediate data - fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); - } - } - - public TaskAttemptId getTaskId() { - return taskId; - } - - public TaskAttemptId getId() { - return context.getTaskId(); - } - - public TaskAttemptState getStatus() { - return context.getState(); - } - - public String toString() { - return "queryId: " + this.getId() + " status: " + this.getStatus(); - } - - public void setState(TaskAttemptState status) { - context.setState(status); - } - - public TaskAttemptContext getContext() { - return context; - } - - public boolean hasFetchPhase() { - return fetcherRunners.size() > 0; - } - - public List getFetchers() { - return new ArrayList(fetcherRunners); - } - - public void fetch() { - ExecutorService executorService = executionBlockContext.getTaskRunner(taskRunnerId).getFetchLauncher(); - for (Fetcher f : fetcherRunners) { - executorService.submit(new FetchRunner(context, f)); - } - } - - public void kill() { - stopScriptExecutors(); - context.setState(TaskAttemptState.TA_KILLED); - context.stop(); - } - - public void abort() { - stopScriptExecutors(); - context.stop(); - } - - public void cleanUp() { - // remove itself from worker - if (context.getState() == TaskAttemptState.TA_SUCCEEDED) { - synchronized (executionBlockContext.getTasks()) { - executionBlockContext.getTasks().remove(this.getId()); - } - } else { - LOG.error("TaskAttemptId: " + context.getTaskId() + " status: " + context.getState()); - } - } - - public TaskStatusProto getReport() { - TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); - builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); - builder.setId(context.getTaskId().getProto()) - .setProgress(context.getProgress()) - .setState(context.getState()); - - builder.setInputStats(reloadInputStats()); - - if (context.getResultStats() != null) { - builder.setResultStats(context.getResultStats().getProto()); - } - return builder.build(); - } - - public boolean isRunning(){ - return context.getState() == TaskAttemptState.TA_RUNNING; - } - - public boolean isProgressChanged() { - return context.isProgressChanged(); - } - - public void updateProgress() { - if(context != null && context.isStopped()){ - return; - } - - if (executor != null && context.getProgress() < 1.0f) { - context.setExecutorProgress(executor.getProgress()); - } - } - - private CatalogProtos.TableStatsProto reloadInputStats() { - synchronized(inputStats) { - if (this.executor == null) { - return inputStats.getProto(); - } - - TableStats executorInputStats = this.executor.getInputStats(); - - if (executorInputStats != null) { - inputStats.setValues(executorInputStats); - } - return inputStats.getProto(); - } - } - - private TaskCompletionReport getTaskCompletionReport() { - TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); - builder.setId(context.getTaskId().getProto()); - - builder.setInputStats(reloadInputStats()); - - if (context.hasResultStats()) { - builder.setResultStats(context.getResultStats().getProto()); - } else { - builder.setResultStats(new TableStats().getProto()); - } - - Iterator> it = context.getShuffleFileOutputs(); - if (it.hasNext()) { - do { - Entry entry = it.next(); - ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); - part.setPartId(entry.getKey()); - - // Set output volume - if (context.getPartitionOutputVolume() != null) { - for (Entry e : context.getPartitionOutputVolume().entrySet()) { - if (entry.getKey().equals(e.getKey())) { - part.setVolume(e.getValue().longValue()); - break; - } - } - } - - builder.addShuffleFileOutputs(part.build()); - } while (it.hasNext()); - } - - return builder.build(); - } - - private void waitForFetch() throws InterruptedException, IOException { - context.getFetchLatch().await(); - LOG.info(context.getTaskId() + " All fetches are done!"); - Collection inputs = Lists.newArrayList(context.getInputTables()); - - // Get all broadcasted tables - Set broadcastTableNames = new HashSet(); - List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); - if (broadcasts != null) { - for (EnforceProperty eachBroadcast : broadcasts) { - broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); - } - } - - // localize the fetched data and skip the broadcast table - for (String inputTable: inputs) { - if (broadcastTableNames.contains(inputTable)) { - continue; - } - File tableDir = new File(context.getFetchIn(), inputTable); - FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); - context.updateAssignedFragments(inputTable, frags); - } - } - - public void run() throws Exception { - startTime = System.currentTimeMillis(); - Throwable error = null; - try { - if(!context.isStopped()) { - context.setState(TaskAttemptState.TA_RUNNING); - if (context.hasFetchPhase()) { - // If the fetch is still in progress, the query unit must wait for - // complete. - waitForFetch(); - context.setFetcherProgress(FETCHER_PROGRESS); - context.setProgressChanged(true); - updateProgress(); - } - - this.executor = executionBlockContext.getTQueryEngine(). - createPlan(context, plan); - this.executor.init(); - - while(!context.isStopped() && executor.next() != null) { - } - } - } catch (Throwable e) { - error = e ; - LOG.error(e.getMessage(), e); - stopScriptExecutors(); - context.stop(); - } finally { - if (executor != null) { - try { - executor.close(); - reloadInputStats(); - } catch (IOException e) { - LOG.error(e, e); - } - this.executor = null; - } - - executionBlockContext.completedTasksNum.incrementAndGet(); - context.getHashShuffleAppenderManager().finalizeTask(taskId); - - QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); - if (context.isStopped()) { - context.setExecutorProgress(0.0f); - - if (context.getState() == TaskAttemptState.TA_KILLED) { - queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); - executionBlockContext.killedTasksNum.incrementAndGet(); - } else { - context.setState(TaskAttemptState.TA_FAILED); - TaskFatalErrorReport.Builder errorBuilder = - TaskFatalErrorReport.newBuilder() - .setId(getId().getProto()); - if (error != null) { - if (error.getMessage() == null) { - errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); - } else { - errorBuilder.setErrorMessage(error.getMessage()); - } - errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); - } - - queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); - executionBlockContext.failedTasksNum.incrementAndGet(); - } - } else { - // if successful - context.setProgress(1.0f); - context.setState(TaskAttemptState.TA_SUCCEEDED); - executionBlockContext.succeededTasksNum.incrementAndGet(); - - TaskCompletionReport report = getTaskCompletionReport(); - queryMasterStub.done(null, report, NullCallback.get()); - } - finishTime = System.currentTimeMillis(); - LOG.info(context.getTaskId() + " completed. " + - "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + - ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() - + ", killed: " + executionBlockContext.killedTasksNum.intValue() - + ", failed: " + executionBlockContext.failedTasksNum.intValue()); - cleanupTask(); - } - } - - public void cleanupTask() { - TaskHistory taskHistory = createTaskHistory(); - executionBlockContext.addTaskHistory(taskRunnerId, getId(), taskHistory); - executionBlockContext.getTasks().remove(getId()); - - fetcherRunners.clear(); - fetcherRunners = null; - try { - if(executor != null) { - executor.close(); - executor = null; - } - } catch (IOException e) { - LOG.fatal(e.getMessage(), e); - } - - executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); - stopScriptExecutors(); - } - - public TaskHistory createTaskHistory() { - TaskHistory taskHistory = null; - try { - taskHistory = new TaskHistory(getTaskId(), getStatus(), context.getProgress(), - startTime, finishTime, reloadInputStats()); - - if (context.getOutputPath() != null) { - taskHistory.setOutputPath(context.getOutputPath().toString()); - } - - if (context.getWorkDir() != null) { - taskHistory.setWorkingPath(context.getWorkDir().toString()); - } - - if (context.getResultStats() != null) { - taskHistory.setOutputStats(context.getResultStats().getProto()); - } - - if (hasFetchPhase()) { - taskHistory.setTotalFetchCount(fetcherRunners.size()); - int i = 0; - FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); - for (Fetcher fetcher : fetcherRunners) { - // TODO store the fetcher histories - if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { - builder.setStartTime(fetcher.getStartTime()); - builder.setFinishTime(fetcher.getFinishTime()); - builder.setFileLength(fetcher.getFileLen()); - builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); - builder.setState(fetcher.getState()); - - taskHistory.addFetcherHistory(builder.build()); - } - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; - } - taskHistory.setFinishedFetchCount(i); - } - } catch (Exception e) { - LOG.warn(e.getMessage(), e); - } - - return taskHistory; - } - - public int hashCode() { - return context.hashCode(); - } - - public boolean equals(Object obj) { - if (obj instanceof Task) { - Task other = (Task) obj; - return this.context.equals(other.context); - } - return false; - } - - private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) - throws IOException { - Configuration c = new Configuration(systemConf); - c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); - FileSystem fs = FileSystem.get(c); - Path tablePath = new Path(file.getAbsolutePath()); - - List listTablets = new ArrayList(); - FileFragment tablet; - - FileStatus[] fileLists = fs.listStatus(tablePath); - for (FileStatus f : fileLists) { - if (f.getLen() == 0) { - continue; - } - tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); - listTablets.add(tablet); - } - - // Special treatment for locally pseudo fetched chunks - synchronized (localChunks) { - for (FileChunk chunk : localChunks) { - if (name.equals(chunk.getEbId())) { - tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); - listTablets.add(tablet); - LOG.info("One local chunk is added to listTablets"); - } - } - } - - FileFragment[] tablets = new FileFragment[listTablets.size()]; - listTablets.toArray(tablets); - - return tablets; - } - - private class FetchRunner implements Runnable { - private final TaskAttemptContext ctx; - private final Fetcher fetcher; - private int maxRetryNum; - - public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { - this.ctx = ctx; - this.fetcher = fetcher; - this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); - } - - @Override - public void run() { - int retryNum = 0; - int retryWaitTime = 1000; //sec - - try { // for releasing fetch latch - while(!context.isStopped() && retryNum < maxRetryNum) { - if (retryNum > 0) { - try { - Thread.sleep(retryWaitTime); - retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds - } catch (InterruptedException e) { - LOG.error(e); - } - LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); - } - try { - FileChunk fetched = fetcher.get(); - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null - && fetched.getFile() != null) { - if (fetched.fromRemote() == false) { - localChunks.add(fetched); - LOG.info("Add a new FileChunk to local chunk list"); - } - break; - } - } catch (Throwable e) { - LOG.error("Fetch failed: " + fetcher.getURI(), e); - } - retryNum++; - } - } finally { - if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ - fetcherFinished(ctx); - } else { - if (retryNum == maxRetryNum) { - LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); - } - stopScriptExecutors(); - context.stop(); // retry task - ctx.getFetchLatch().countDown(); - } - } - } - } - - @VisibleForTesting - public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { - if (totalFetcher > 0) { - return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; - } else { - return 0.0f; - } - } - - private synchronized void fetcherFinished(TaskAttemptContext ctx) { - int fetcherSize = fetcherRunners.size(); - if(fetcherSize == 0) { - return; - } - - ctx.getFetchLatch().countDown(); - - int remainFetcher = (int) ctx.getFetchLatch().getCount(); - if (remainFetcher == 0) { - context.setFetcherProgress(FETCHER_PROGRESS); - } else { - context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); - context.setProgressChanged(true); - } - } - - private List getFetchRunners(TaskAttemptContext ctx, - List fetches) throws IOException { - - if (fetches.size() > 0) { - Path inputDir = executionBlockContext.getLocalDirAllocator(). - getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); - - int i = 0; - File storeDir; - File defaultStoreFile; - FileChunk storeChunk = null; - List runnerList = Lists.newArrayList(); - - for (FetchImpl f : fetches) { - storeDir = new File(inputDir.toString(), f.getName()); - if (!storeDir.exists()) { - storeDir.mkdirs(); - } - - for (URI uri : f.getURIs()) { - defaultStoreFile = new File(storeDir, "in_" + i); - InetAddress address = InetAddress.getByName(uri.getHost()); - - WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); - if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { - boolean hasError = false; - try { - LOG.info("Try to get local file chunk at local host"); - storeChunk = getLocalStoredFileChunk(uri, systemConf); - } catch (Throwable t) { - hasError = true; - } - - // When a range request is out of range, storeChunk will be NULL. This case is normal state. - // So, we should skip and don't need to create storeChunk. - if (storeChunk == null && !hasError) { - continue; - } - - if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 - && hasError == false) { - storeChunk.setFromRemote(false); - } else { - storeChunk = new FileChunk(defaultStoreFile, 0, -1); - storeChunk.setFromRemote(true); - } - } else { - storeChunk = new FileChunk(defaultStoreFile, 0, -1); - storeChunk.setFromRemote(true); - } - - // If we decide that intermediate data should be really fetched from a remote host, storeChunk - // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it - storeChunk.setEbId(f.getName()); - Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); - LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); - runnerList.add(fetcher); - i++; - } - } - ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); - return runnerList; - } else { - return Lists.newArrayList(); - } - } - - private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { - // Parse the URI - LOG.info("getLocalStoredFileChunk starts"); - final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); - final List types = params.get("type"); - final List qids = params.get("qid"); - final List taskIdList = params.get("ta"); - final List stageIds = params.get("sid"); - final List partIds = params.get("p"); - final List offsetList = params.get("offset"); - final List lengthList = params.get("length"); - - 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 || 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 = stageIds.get(0); - String partId = partIds.get(0); - - if (shuffleType.equals("r") && taskIdList == null) { - LOG.error("Invalid URI - For range shuffle, taskId is required"); - return null; - } - List taskIds = splitMaps(taskIdList); - - FileChunk chunk = null; - long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; - long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; - - LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId - + ", taskIds=" + taskIdList); - - // The working directory of Tajo worker for each query, including stage - String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; - - // If the stage requires a range shuffle - if (shuffleType.equals("r")) { - String ta = taskIds.get(0); - if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { - LOG.warn("Range shuffle - file not exist"); - return null; - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); - String startKey = params.get("start").get(0); - String endKey = params.get("end").get(0); - boolean last = params.get("final") != null; - - try { - chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); - } catch (Throwable t) { - LOG.error("getFileChunks() throws exception"); - return null; - } - - // 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; - if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { - LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); - return null; - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); - File file = new File(path.toUri()); - long startPos = (offset >= 0 && length >= 0) ? offset : 0; - long readLen = (offset >= 0 && length >= 0) ? length : file.length(); - - if (startPos >= file.length()) { - LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); - return null; - } - chunk = new FileChunk(file, startPos, readLen); - - } else { - LOG.error("Unknown shuffle type"); - return null; - } - - return chunk; - } - - private List splitMaps(List mapq) { - if (null == mapq) { - return null; - } - final List ret = new ArrayList(); - for (String s : mapq) { - Collections.addAll(ret, s.split(",")); - } - return ret; - } - - public static Path getTaskAttemptDir(TaskAttemptId quid) { - Path workDir = - StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), - String.valueOf(quid.getTaskId().getId()), - String.valueOf(quid.getId())); - return workDir; - } + TajoWorkerProtocol.TaskStatusProto getReport(); } 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 58028ac014..d020639f2d 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 @@ -46,7 +46,6 @@ import java.util.*; import java.util.Map.Entry; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; @@ -60,13 +59,13 @@ public class TaskAttemptContext { private volatile TaskAttemptState state; private TableStats resultStats; - private TaskAttemptId queryId; + private TaskAttemptId taskId; private final Path workDir; private boolean needFetch = false; private CountDownLatch doneFetchPhaseSignal; private float progress = 0.0f; private float fetcherProgress = 0.0f; - private AtomicBoolean progressChanged = new AtomicBoolean(false); + private volatile boolean progressChanged; /** a map of shuffled file outputs */ private Map shuffleFileOutputs; @@ -87,7 +86,7 @@ public class TaskAttemptContext { private EvalContext evalContext = new EvalContext(); public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext executionBlockContext, - final TaskAttemptId queryId, + final TaskAttemptId taskId, final FragmentProto[] fragments, final Path workDir) { this.queryContext = queryContext; @@ -97,7 +96,7 @@ public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext this.sharedResource = executionBlockContext.getSharedResource(); } - this.queryId = queryId; + this.taskId = taskId; if (fragments != null) { for (FragmentProto t : fragments) { @@ -114,25 +113,15 @@ public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext this.workDir = workDir; this.shuffleFileOutputs = Maps.newHashMap(); - state = TaskAttemptState.TA_PENDING; + this.state = TaskAttemptState.TA_PENDING; this.partitionOutputVolume = Maps.newHashMap(); - - if (workerContext != null) { - this.hashShuffleAppenderManager = workerContext.getHashShuffleAppenderManager(); - } else { - try { - this.hashShuffleAppenderManager = new HashShuffleAppenderManager(queryContext.getConf()); - } catch (IOException e) { - LOG.error(e.getMessage(), e); - } - } } @VisibleForTesting - public TaskAttemptContext(final QueryContext queryContext, final TaskAttemptId queryId, + public TaskAttemptContext(final QueryContext queryContext, final TaskAttemptId taskAttemptId, final Fragment [] fragments, final Path workDir) { - this(queryContext, null, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); + this(queryContext, null, taskAttemptId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); } public TajoConf getConf() { @@ -308,9 +297,10 @@ private List fragmentToPath(List tableFragments) { public Path getWorkDir() { return this.workDir; } - + + //TODO change to getTaskAttemptId() public TaskAttemptId getTaskId() { - return this.queryId; + return this.taskId; } public float getProgress() { @@ -326,17 +316,11 @@ public void setProgress(float progress) { this.progress = progress; } - if (previousProgress != progress) { - setProgressChanged(true); - } + this.progressChanged = previousProgress != progress; } public boolean isProgressChanged() { - return progressChanged.get(); - } - - public void setProgressChanged(boolean changed){ - progressChanged.set(changed); + return progressChanged; } public void setExecutorProgress(float executorProgress) { @@ -355,7 +339,9 @@ public void setFetcherProgress(float fetcherProgress) { if(Float.isNaN(fetcherProgress) || Float.isInfinite(fetcherProgress)){ fetcherProgress = 0.0f; } + float previousProgress = this.fetcherProgress; this.fetcherProgress = fetcherProgress; + this.progressChanged = previousProgress != fetcherProgress; } public FragmentProto getTable(String id) { @@ -383,13 +369,13 @@ public Collection getInputTables() { } public int hashCode() { - return Objects.hashCode(queryId); + return Objects.hashCode(taskId); } public boolean equals(Object obj) { if (obj instanceof TaskAttemptContext) { TaskAttemptContext other = (TaskAttemptContext) obj; - return queryId.equals(other.getTaskId()); + return taskId.equals(other.getTaskId()); } else { return false; } @@ -399,11 +385,18 @@ public QueryContext getQueryContext() { return queryContext; } - public TaskAttemptId getQueryId() { - return queryId; - } - public HashShuffleAppenderManager getHashShuffleAppenderManager() { + if(hashShuffleAppenderManager == null) { + if (workerContext != null) { + this.hashShuffleAppenderManager = workerContext.getHashShuffleAppenderManager(); + } else { + try { + this.hashShuffleAppenderManager = new HashShuffleAppenderManager(queryContext.getConf()); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + } + } return hashShuffleAppenderManager; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java new file mode 100644 index 0000000000..2576726840 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.TajoProtos; + +/** + * The driver class for Tajo Task processing. + */ +public class TaskContainer implements Runnable { + private static final Log LOG = LogFactory.getLog(TaskContainer.class); + + private final TaskExecutor executor; + private final int sequenceId; + + public TaskContainer(int sequenceId, TaskExecutor executor) { + this.sequenceId = sequenceId; + this.executor = executor; + } + + @Override + public void run() { + while (!executor.isStopped()) { + + Task task = null; + try { + task = executor.getNextTask(); + + task.getExecutionBlockContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); + + if (LOG.isDebugEnabled()) { + LOG.debug(sequenceId + TaskContainer.class.getSimpleName() + + " got task:" + task.getTaskContext().getTaskId()); + } + + TaskAttemptContext taskAttemptContext = task.getTaskContext(); + if (taskAttemptContext.isStopped()) return; + + task.init(); + + if (task.hasFetchPhase()) { + task.fetch(); // The fetch is performed in an asynchronous way. + } + + if (!taskAttemptContext.isStopped()) { + task.run(); + } + + task.cleanup(); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + if (task != null) { + try { + task.abort(); + task.getExecutionBlockContext().fatalError(task.getTaskContext().getTaskId(), e.getMessage()); + } catch (Throwable t) { + LOG.fatal(t.getMessage(), t); + } + } + } finally { + if (task != null) { + executor.stopTask(task.getTaskContext().getTaskId()); + } + } + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java new file mode 100644 index 0000000000..299952e7ae --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -0,0 +1,194 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.TaskId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.engine.query.TaskRequestImpl; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.worker.event.*; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * TaskExecutor uses a number of threads equal to the number of slots available for running tasks on the Worker + */ +public class TaskExecutor extends AbstractService implements EventHandler { + private static final Log LOG = LogFactory.getLog(TaskExecutor.class); + + private final TaskManager taskManager; + private final EventHandler rmEventHandler; + private final Map allocatedResourceMap; + private final BlockingQueue taskQueue; + private final AtomicInteger runningTasks; + private ThreadPoolExecutor fetcherExecutor; + private ExecutorService threadPool; + private TajoConf tajoConf; + private volatile boolean isStopped; + + public TaskExecutor(TaskManager taskManager, EventHandler rmEventHandler) { + super(TaskExecutor.class.getName()); + this.taskManager = taskManager; + this.rmEventHandler = rmEventHandler; + this.allocatedResourceMap = Maps.newConcurrentMap(); + this.runningTasks = new AtomicInteger(); + this.taskQueue = new LinkedBlockingQueue(); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + + this.tajoConf = (TajoConf) conf; + this.taskManager.getDispatcher().register(TaskExecutorEvent.EventType.class, this); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + int nThreads = this.tajoConf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); + this.threadPool = Executors.newFixedThreadPool(nThreads, + new ThreadFactoryBuilder().setNameFormat("Task executor #%d").build()); + + //TODO move to tajoConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + int maxFetcherThreads = Runtime.getRuntime().availableProcessors() * 2; + this.fetcherExecutor = new ThreadPoolExecutor(Math.min(nThreads, maxFetcherThreads), + maxFetcherThreads, + 60L, TimeUnit.SECONDS, + new SynchronousQueue(true)); + + + for (int i = 0; i < nThreads; i++) { + threadPool.submit(new TaskContainer(i, this)); + } + + super.serviceStart(); + LOG.info("Started TaskExecutor[" + nThreads + "], Fetcher executor[" + maxFetcherThreads + "]"); + } + + @Override + protected void serviceStop() throws Exception { + isStopped = true; + + threadPool.shutdown(); + fetcherExecutor.shutdown(); + super.serviceStop(); + } + + public boolean isStopped() { + return isStopped; + } + + public int getRunningTasks() { + return runningTasks.get(); + } + + /** + * This will block until a task is available. + */ + protected Task getNextTask() { + Task task = null; + try { + task = taskQueue.take(); + } catch (InterruptedException e) { + LOG.fatal(e); + } + return task; + } + + @SuppressWarnings("unchecked") + protected void stopTask(TaskAttemptId taskId) { + runningTasks.decrementAndGet(); + rmEventHandler.handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId))); + } + + protected ExecutorService getFetcherExecutor() { + return fetcherExecutor; + } + + + protected Task createTask(ExecutionBlockContext executionBlockContext, + TajoWorkerProtocol.TaskRequestProto taskRequest) throws IOException { + Task task = null; + TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + if (executionBlockContext.getTasks().containsKey(taskAttemptId)) { + String errorMessage = "Duplicate Task Attempt: " + taskAttemptId; + LOG.error(errorMessage); + executionBlockContext.fatalError(taskAttemptId, errorMessage); + } else { + task = new TaskImpl(new TaskRequestImpl(taskRequest), executionBlockContext, getFetcherExecutor()); + executionBlockContext.getTasks().put(task.getTaskContext().getTaskId(), task); + } + return task; + } + + @Override + public void handle(TaskExecutorEvent event) { + + if (event instanceof TaskStartEvent) { + TaskStartEvent startEvent = (TaskStartEvent) event; + allocatedResourceMap.put(startEvent.getTaskId(), startEvent.getAllocatedResource()); + + ExecutionBlockContext context = taskManager.getExecutionBlockContext( + startEvent.getTaskId().getTaskId().getExecutionBlockId()); + + try { + Task task = createTask(context, startEvent.getTaskRequest()); + if (task != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Arrival task: " + task.getTaskContext().getTaskId() + + ", allocated resource: " + startEvent.getAllocatedResource()); + } + taskQueue.put(task); + runningTasks.incrementAndGet(); + context.getWorkerContext().getWorkerSystemMetrics() + .histogram("tasks", "running").update(runningTasks.get()); + } else { + LOG.warn("Release duplicate task resource: " + startEvent.getAllocatedResource()); + stopTask(startEvent.getTaskId()); + } + } catch (InterruptedException e) { + if (!isStopped) { + LOG.fatal(e.getMessage(), e); + } + } catch (IOException e) { + stopTask(startEvent.getTaskId()); + } + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java new file mode 100644 index 0000000000..be3960b0ca --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -0,0 +1,838 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.netty.handler.codec.http.QueryStringDecoder; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.TaskAttemptState; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.planner.physical.PhysicalExec; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.ipc.QueryMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.*; +import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.TajoScriptEngine; +import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.serder.LogicalNodeDeserializer; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.rpc.NullCallback; +import org.apache.tajo.storage.*; +import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.util.NetUtils; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.URI; +import java.util.*; +import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; + +import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; + +public class TaskImpl implements Task { + private static final Log LOG = LogFactory.getLog(TaskImpl.class); + private static final float FETCHER_PROGRESS = 0.5f; + + private final TajoConf systemConf; + private final QueryContext queryContext; + private final ExecutionBlockContext executionBlockContext; + private final TaskRequest request; + private final Map descs; + private final TableStats inputStats; + private final ExecutorService fetcherExecutor; + private final Path taskDir; + + private final TaskAttemptContext context; + private List fetcherRunners; + private LogicalNode plan; + private PhysicalExec executor; + + private boolean interQuery; + private Path inputTableBaseDir; + + private long startTime; + private long endTime; + + private List localChunks; + // TODO - to be refactored + private ShuffleType shuffleType = null; + private Schema finalSchema = null; + + private TupleComparator sortComp = null; + + public TaskImpl(final TaskRequest request, + final ExecutionBlockContext executionBlockContext, + final ExecutorService fetcherExecutor) throws IOException { + + this.request = request; + this.executionBlockContext = executionBlockContext; + this.systemConf = executionBlockContext.getConf(); + this.queryContext = request.getQueryContext(systemConf); + this.inputStats = new TableStats(); + this.fetcherRunners = Lists.newArrayList(); + this.fetcherExecutor = fetcherExecutor; + this.descs = Maps.newHashMap(); + + Path baseDirPath = executionBlockContext.createBaseDir(); + LOG.info("Task basedir is created (" + baseDirPath +")"); + TaskAttemptId taskAttemptId = request.getId(); + + this.taskDir = StorageUtil.concatPath(baseDirPath, + taskAttemptId.getTaskId().getId() + "_" + taskAttemptId.getId()); + this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskAttemptId, + request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); + this.context.setDataChannel(request.getDataChannel()); + this.context.setEnforcer(request.getEnforcer()); + this.context.setState(TaskAttemptState.TA_PENDING); + } + + public void initPlan() throws IOException { + plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); + LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); + if (scanNode != null) { + for (LogicalNode node : scanNode) { + ScanNode scan = (ScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); + if (partitionScanNode != null) { + for (LogicalNode node : partitionScanNode) { + PartitionedTableScanNode scan = (PartitionedTableScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + interQuery = request.getProto().getInterQuery(); + if (interQuery) { + context.setInterQuery(); + this.shuffleType = context.getDataChannel().getShuffleType(); + + if (shuffleType == ShuffleType.RANGE_SHUFFLE) { + SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); + this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); + this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); + } + } else { + Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) + .getAppenderFilePath(getId(), queryContext.getStagingDir()); + LOG.info("Output File Path: " + outFilePath); + context.setOutputPath(outFilePath); + } + + this.localChunks = Collections.synchronizedList(new ArrayList()); + LOG.info("=================================="); + LOG.info("* Stage " + request.getId() + " is initialized"); + LOG.info("* InterQuery: " + interQuery + + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + + ", Fragments (num: " + request.getFragments().size() + ")" + + ", Fetches (total:" + request.getFetches().size() + ") :"); + + if(LOG.isDebugEnabled()) { + for (FetchImpl f : request.getFetches()) { + LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); + } + } + LOG.info("* Local task dir: " + taskDir); + if(LOG.isDebugEnabled()) { + LOG.debug("* plan:\n"); + LOG.debug(plan.toString()); + } + LOG.info("=================================="); + } + + private void startScriptExecutors() throws IOException { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.start(systemConf); + } + } + + private void stopScriptExecutors() { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.shutdown(); + } + } + + @Override + public void init() throws IOException { + LOG.info("Initializing: " + getId()); + + initPlan(); + startScriptExecutors(); + + if (context.getState() == TaskAttemptState.TA_PENDING) { + // initialize a task temporal dir + FileSystem localFS = executionBlockContext.getLocalFS(); + localFS.mkdirs(taskDir); + + if (request.getFetches().size() > 0) { + inputTableBaseDir = localFS.makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( + getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); + localFS.mkdirs(inputTableBaseDir); + Path tableDir; + for (String inputTable : context.getInputTables()) { + tableDir = new Path(inputTableBaseDir, inputTable); + if (!localFS.exists(tableDir)) { + LOG.info("the directory is created " + tableDir.toUri()); + localFS.mkdirs(tableDir); + } + } + } + // for localizing the intermediate data + fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); + } + } + + private TaskAttemptId getId() { + return context.getTaskId(); + } + + public String toString() { + return "TaskId: " + this.getId() + " Status: " + context.getState(); + } + + @Override + public boolean isStopped() { + return context.isStopped(); + } + + @Override + public TaskAttemptContext getTaskContext() { + return context; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return executionBlockContext; + } + + @Override + public boolean hasFetchPhase() { + return fetcherRunners.size() > 0; + } + + @Override + public void fetch() { + for (Fetcher f : fetcherRunners) { + fetcherExecutor.submit(new FetchRunner(context, f)); + } + } + + @Override + public void kill() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_KILLED); + context.stop(); + } + + @Override + public void abort() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_FAILED); + context.stop(); + } + + @Override + public TaskStatusProto getReport() { + TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); + builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); + builder.setId(context.getTaskId().getProto()) + .setProgress(context.getProgress()) + .setState(context.getState()); + + builder.setInputStats(reloadInputStats()); + + if (context.getResultStats() != null) { + builder.setResultStats(context.getResultStats().getProto()); + } + return builder.build(); + } + + @Override + public boolean isProgressChanged() { + return context.isProgressChanged(); + } + + @Override + public void updateProgress() { + if(context != null && context.isStopped()){ + return; + } + + if (executor != null && context.getProgress() < 1.0f) { + context.setExecutorProgress(executor.getProgress()); + } + } + + private CatalogProtos.TableStatsProto reloadInputStats() { + synchronized(inputStats) { + if (this.executor == null) { + return inputStats.getProto(); + } + + TableStats executorInputStats = this.executor.getInputStats(); + + if (executorInputStats != null) { + inputStats.setValues(executorInputStats); + } + return inputStats.getProto(); + } + } + + private TaskCompletionReport getTaskCompletionReport() { + TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); + builder.setId(context.getTaskId().getProto()); + + builder.setInputStats(reloadInputStats()); + + if (context.hasResultStats()) { + builder.setResultStats(context.getResultStats().getProto()); + } else { + builder.setResultStats(new TableStats().getProto()); + } + + Iterator> it = context.getShuffleFileOutputs(); + if (it.hasNext()) { + do { + Entry entry = it.next(); + ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); + part.setPartId(entry.getKey()); + + // Set output volume + if (context.getPartitionOutputVolume() != null) { + for (Entry e : context.getPartitionOutputVolume().entrySet()) { + if (entry.getKey().equals(e.getKey())) { + part.setVolume(e.getValue().longValue()); + break; + } + } + } + + builder.addShuffleFileOutputs(part.build()); + } while (it.hasNext()); + } + + return builder.build(); + } + + private void waitForFetch() throws InterruptedException, IOException { + context.getFetchLatch().await(); + LOG.info(context.getTaskId() + " All fetches are done!"); + Collection inputs = Lists.newArrayList(context.getInputTables()); + + // Get all broadcasted tables + Set broadcastTableNames = new HashSet(); + List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); + if (broadcasts != null) { + for (EnforceProperty eachBroadcast : broadcasts) { + broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); + } + } + + // localize the fetched data and skip the broadcast table + for (String inputTable: inputs) { + if (broadcastTableNames.contains(inputTable)) { + continue; + } + File tableDir = new File(context.getFetchIn(), inputTable); + FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); + context.updateAssignedFragments(inputTable, frags); + } + } + + @Override + public void run() throws Exception { + startTime = System.currentTimeMillis(); + Throwable error = null; + + try { + if(!context.isStopped()) { + context.setState(TajoProtos.TaskAttemptState.TA_RUNNING); + if (context.hasFetchPhase()) { + // If the fetch is still in progress, the query unit must wait for complete. + waitForFetch(); + context.setFetcherProgress(FETCHER_PROGRESS); + updateProgress(); + } + + this.executor = executionBlockContext.getTQueryEngine().createPlan(context, plan); + this.executor.init(); + + while(!context.isStopped() && executor.next() != null) { + } + } + } catch (Throwable e) { + error = e ; + LOG.error(e.getMessage(), e); + stopScriptExecutors(); + context.stop(); + } finally { + if (executor != null) { + try { + executor.close(); + reloadInputStats(); + } catch (IOException e) { + LOG.error(e, e); + } + this.executor = null; + } + + executionBlockContext.completedTasksNum.incrementAndGet(); + context.getHashShuffleAppenderManager().finalizeTask(getId()); + + QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); + if (context.isStopped()) { + context.setExecutorProgress(0.0f); + + if (context.getState() == TaskAttemptState.TA_KILLED) { + queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); + executionBlockContext.killedTasksNum.incrementAndGet(); + } else { + context.setState(TaskAttemptState.TA_FAILED); + TaskFatalErrorReport.Builder errorBuilder = + TaskFatalErrorReport.newBuilder() + .setId(getId().getProto()); + if (error != null) { + if (error.getMessage() == null) { + errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); + } else { + errorBuilder.setErrorMessage(error.getMessage()); + } + errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); + } + + queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); + executionBlockContext.failedTasksNum.incrementAndGet(); + } + } else { + // if successful + context.stop(); + context.setProgress(1.0f); + context.setState(TaskAttemptState.TA_SUCCEEDED); + executionBlockContext.succeededTasksNum.incrementAndGet(); + + TaskCompletionReport report = getTaskCompletionReport(); + queryMasterStub.done(null, report, NullCallback.get()); + } + endTime = System.currentTimeMillis(); + LOG.info(context.getTaskId() + " completed. " + + "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + + ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() + + ", killed: " + executionBlockContext.killedTasksNum.intValue() + + ", failed: " + executionBlockContext.failedTasksNum.intValue()); + } + } + + @Override + public void cleanup() { + TaskHistory taskHistory = createTaskHistory(); + executionBlockContext.addTaskHistory(getId().getTaskId(), taskHistory); + executionBlockContext.getTasks().remove(getId()); + + fetcherRunners.clear(); + fetcherRunners = null; + try { + if(executor != null) { + executor.close(); + executor = null; + } + } catch (IOException e) { + LOG.fatal(e.getMessage(), e); + } + + executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); + stopScriptExecutors(); + } + + public TaskHistory createTaskHistory() { + TaskHistory taskHistory = null; + try { + taskHistory = new TaskHistory(context.getTaskId(), context.getState(), context.getProgress(), + startTime, endTime, reloadInputStats()); + + if (context.getOutputPath() != null) { + taskHistory.setOutputPath(context.getOutputPath().toString()); + } + + if (context.getWorkDir() != null) { + taskHistory.setWorkingPath(context.getWorkDir().toString()); + } + + if (context.getResultStats() != null) { + taskHistory.setOutputStats(context.getResultStats().getProto()); + } + + if (hasFetchPhase()) { + taskHistory.setTotalFetchCount(fetcherRunners.size()); + int i = 0; + FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); + for (Fetcher fetcher : fetcherRunners) { + // TODO store the fetcher histories + if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { + builder.setStartTime(fetcher.getStartTime()); + builder.setFinishTime(fetcher.getFinishTime()); + builder.setFileLength(fetcher.getFileLen()); + builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); + builder.setState(fetcher.getState()); + + taskHistory.addFetcherHistory(builder.build()); + } + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; + } + taskHistory.setFinishedFetchCount(i); + } + } catch (Exception e) { + LOG.warn(e.getMessage(), e); + } + + return taskHistory; + } + + public int hashCode() { + return context.hashCode(); + } + + public boolean equals(Object obj) { + if (obj instanceof TaskImpl) { + TaskImpl other = (TaskImpl) obj; + return this.context.equals(other.context); + } + return false; + } + + private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) + throws IOException { + Configuration c = new Configuration(systemConf); + c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); + FileSystem fs = FileSystem.get(c); + Path tablePath = new Path(file.getAbsolutePath()); + + List listTablets = new ArrayList(); + FileFragment tablet; + + FileStatus[] fileLists = fs.listStatus(tablePath); + for (FileStatus f : fileLists) { + if (f.getLen() == 0) { + continue; + } + tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); + listTablets.add(tablet); + } + + // Special treatment for locally pseudo fetched chunks + synchronized (localChunks) { + for (FileChunk chunk : localChunks) { + if (name.equals(chunk.getEbId())) { + tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); + listTablets.add(tablet); + LOG.info("One local chunk is added to listTablets"); + } + } + } + + FileFragment[] tablets = new FileFragment[listTablets.size()]; + listTablets.toArray(tablets); + + return tablets; + } + + private class FetchRunner implements Runnable { + private final TaskAttemptContext ctx; + private final Fetcher fetcher; + private int maxRetryNum; + + public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { + this.ctx = ctx; + this.fetcher = fetcher; + this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); + } + + @Override + public void run() { + int retryNum = 0; + int retryWaitTime = 1000; //sec + + try { // for releasing fetch latch + while(!context.isStopped() && retryNum < maxRetryNum) { + if (retryNum > 0) { + try { + Thread.sleep(retryWaitTime); + retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds + } catch (InterruptedException e) { + LOG.error(e); + } + LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); + } + try { + FileChunk fetched = fetcher.get(); + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null + && fetched.getFile() != null) { + if (fetched.fromRemote() == false) { + localChunks.add(fetched); + LOG.info("Add a new FileChunk to local chunk list"); + } + break; + } + } catch (Throwable e) { + LOG.error("Fetch failed: " + fetcher.getURI(), e); + } + retryNum++; + } + } finally { + if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ + fetcherFinished(ctx); + } else { + if (retryNum == maxRetryNum) { + LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); + } + stopScriptExecutors(); + context.stop(); // retry task + ctx.getFetchLatch().countDown(); + } + } + } + } + + @VisibleForTesting + public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { + if (totalFetcher > 0) { + return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; + } else { + return 0.0f; + } + } + + private synchronized void fetcherFinished(TaskAttemptContext ctx) { + int fetcherSize = fetcherRunners.size(); + if(fetcherSize == 0) { + return; + } + + ctx.getFetchLatch().countDown(); + + int remainFetcher = (int) ctx.getFetchLatch().getCount(); + if (remainFetcher == 0) { + context.setFetcherProgress(FETCHER_PROGRESS); + } else { + context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); + } + } + + private List getFetchRunners(TaskAttemptContext ctx, + List fetches) throws IOException { + + if (fetches.size() > 0) { + Path inputDir = executionBlockContext.getLocalDirAllocator(). + getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); + + int i = 0; + File storeDir; + File defaultStoreFile; + FileChunk storeChunk = null; + List runnerList = Lists.newArrayList(); + + for (FetchImpl f : fetches) { + storeDir = new File(inputDir.toString(), f.getName()); + if (!storeDir.exists()) { + if (!storeDir.mkdirs()) throw new IOException("Failed to create " + storeDir); + } + + for (URI uri : f.getURIs()) { + defaultStoreFile = new File(storeDir, "in_" + i); + InetAddress address = InetAddress.getByName(uri.getHost()); + + WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); + if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { + boolean hasError = false; + try { + LOG.info("Try to get local file chunk at local host"); + storeChunk = getLocalStoredFileChunk(uri, systemConf); + } catch (Throwable t) { + hasError = true; + } + + // When a range request is out of range, storeChunk will be NULL. This case is normal state. + // So, we should skip and don't need to create storeChunk. + if (storeChunk == null && !hasError) { + continue; + } + + if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 + && hasError == false) { + storeChunk.setFromRemote(false); + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + + // If we decide that intermediate data should be really fetched from a remote host, storeChunk + // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it + storeChunk.setEbId(f.getName()); + Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); + LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); + runnerList.add(fetcher); + i++; + } + } + ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); + return runnerList; + } else { + return Lists.newArrayList(); + } + } + + private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { + // Parse the URI + LOG.info("getLocalStoredFileChunk starts"); + final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); + final List types = params.get("type"); + final List qids = params.get("qid"); + final List taskIdList = params.get("ta"); + final List stageIds = params.get("sid"); + final List partIds = params.get("p"); + final List offsetList = params.get("offset"); + final List lengthList = params.get("length"); + + 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 || 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 = stageIds.get(0); + String partId = partIds.get(0); + + if (shuffleType.equals("r") && taskIdList == null) { + LOG.error("Invalid URI - For range shuffle, taskId is required"); + return null; + } + List taskIds = splitMaps(taskIdList); + + FileChunk chunk; + long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; + long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; + + LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId + + ", taskIds=" + taskIdList); + + // The working directory of Tajo worker for each query, including stage + String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; + + // If the stage requires a range shuffle + if (shuffleType.equals("r")) { + String ta = taskIds.get(0); + if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { + LOG.warn("Range shuffle - file not exist"); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); + String startKey = params.get("start").get(0); + String endKey = params.get("end").get(0); + boolean last = params.get("final") != null; + + try { + chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); + } catch (Throwable t) { + LOG.error("getFileChunks() throws exception"); + return null; + } + + // 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; + if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { + LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); + File file = new File(path.toUri()); + long startPos = (offset >= 0 && length >= 0) ? offset : 0; + long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); + return null; + } + chunk = new FileChunk(file, startPos, readLen); + + } else { + LOG.error("Unknown shuffle type"); + return null; + } + + return chunk; + } + + private List splitMaps(List mapq) { + if (null == mapq) { + return null; + } + final List ret = new ArrayList(); + for (String s : mapq) { + Collections.addAll(ret, s.split(",")); + } + return ret; + } + + public static Path getTaskAttemptDir(TaskAttemptId quid) { + Path workDir = + StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), + String.valueOf(quid.getTaskId().getId()), + String.valueOf(quid.getId())); + return workDir; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java new file mode 100644 index 0000000000..7990a7213e --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -0,0 +1,180 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import com.google.common.collect.Maps; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TajoIdProtos; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.TaskId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.worker.event.*; + +import java.io.IOException; +import java.util.*; + +/** + * A TaskManager is responsible for managing executionBlock resource and tasks. + * */ +public class TaskManager extends AbstractService implements EventHandler { + private static final Log LOG = LogFactory.getLog(TaskManager.class); + + private final TajoWorker.WorkerContext workerContext; + private final Map executionBlockContextMap; + private final Dispatcher dispatcher; + private final EventHandler rmEventHandler; + + private TajoConf tajoConf; + + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { + super(TaskManager.class.getName()); + + this.dispatcher = dispatcher; + this.workerContext = workerContext; + this.executionBlockContextMap = Maps.newHashMap(); + this.rmEventHandler = rmEventHandler; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + + this.tajoConf = (TajoConf)conf; + dispatcher.register(TaskManagerEvent.EventType.class, this); + super.serviceInit(conf); + } + + @Override + protected void serviceStop() throws Exception { + + for(ExecutionBlockContext context: executionBlockContextMap.values()) { + context.stop(); + } + executionBlockContextMap.clear(); + super.serviceStop(); + } + + protected Dispatcher getDispatcher() { + return dispatcher; + } + + protected TajoWorker.WorkerContext getWorkerContext() { + return workerContext; + } + + protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + try { + ExecutionBlockContext context = new ExecutionBlockContext(getWorkerContext(), null, request); + + context.init(); + return context; + } catch (Throwable e) { + LOG.fatal(e.getMessage(), e); + throw new RuntimeException(e); + } + } + + protected void stopExecutionBlock(ExecutionBlockContext context, + TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + + if(context != null){ + try { + context.getSharedResource().releaseBroadcastCache(context.getExecutionBlockId()); + context.sendShuffleReport(); + getWorkerContext().getTaskHistoryWriter().flushTaskHistories(); + } catch (Exception e) { + LOG.fatal(e.getMessage(), e); + throw new RuntimeException(e); + } finally { + context.stop(); + + /* cleanup intermediate files */ + for (TajoIdProtos.ExecutionBlockIdProto ebId : cleanupList.getExecutionBlockIdList()) { + String inputDir = ExecutionBlockContext.getBaseInputDir(new ExecutionBlockId(ebId)).toString(); + workerContext.cleanup(inputDir); + String outputDir = ExecutionBlockContext.getBaseOutputDir(new ExecutionBlockId(ebId)).toString(); + workerContext.cleanup(outputDir); + } + } + LOG.info("Stopped execution block:" + context.getExecutionBlockId()); + } + } + + @Override + public void handle(TaskManagerEvent event) { + LOG.info("======================== Processing " + event.getExecutionBlockId() + " of type " + event.getType()); + + if (event instanceof ExecutionBlockStartEvent) { + + //receive event from NodeResourceManager + if(!executionBlockContextMap.containsKey(event.getExecutionBlockId())) { + ExecutionBlockContext context = createExecutionBlock(((ExecutionBlockStartEvent) event).getRequestProto()); + executionBlockContextMap.put(context.getExecutionBlockId(), context); + } else { + LOG.warn("Already initialized ExecutionBlock: " + event.getExecutionBlockId()); + } + } else if (event instanceof ExecutionBlockStopEvent) { + //receive event from QueryMaster + rmEventHandler.handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + stopExecutionBlock(executionBlockContextMap.remove(event.getExecutionBlockId()), + ((ExecutionBlockStopEvent) event).getCleanupList()); + } + } + + protected ExecutionBlockContext getExecutionBlockContext(ExecutionBlockId executionBlockId) { + return executionBlockContextMap.get(executionBlockId); + } + + public Task getTaskByTaskAttemptId(TaskAttemptId taskAttemptId) { + ExecutionBlockContext context = executionBlockContextMap.get(taskAttemptId.getTaskId().getExecutionBlockId()); + if (context != null) { + return context.getTask(taskAttemptId); + } + return null; + } + + public List getTaskHistories(ExecutionBlockId executionblockId) throws IOException { + List histories = new ArrayList(); + ExecutionBlockContext context = executionBlockContextMap.get(executionblockId); + if (context != null) { + histories.addAll(context.getTaskHistories().values()); + } + //TODO get List from HistoryReader + return histories; + } + + public TaskHistory getTaskHistory(TaskId taskId) { + TaskHistory history = null; + ExecutionBlockContext context = executionBlockContextMap.get(taskId.getExecutionBlockId()); + if (context != null) { + history = context.getTaskHistories().get(taskId); + } + //TODO get TaskHistory from HistoryReader + return history; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java index 774f3585c3..207b47e9f8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java @@ -34,10 +34,8 @@ import org.apache.tajo.master.container.TajoContainerIdPBImpl; import org.apache.tajo.master.container.TajoConverterUtils; import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NettyClientBase; import org.apache.tajo.rpc.NullCallback; -import java.net.ConnectException; import java.util.concurrent.*; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; @@ -45,6 +43,7 @@ /** * The driver class for Tajo Task processing. */ +@Deprecated public class TaskRunner extends AbstractService { /** class logger */ private static final Log LOG = LogFactory.getLog(TaskRunner.class); @@ -256,7 +255,7 @@ public void run() { LOG.info("Initializing: " + taskAttemptId); Task task = null; try { - task = new Task(getId(), getTaskBaseDir(), taskAttemptId, executionBlockContext, + task = new LegacyTaskImpl(getId(), getTaskBaseDir(), taskAttemptId, executionBlockContext, new TaskRequestImpl(taskRequest)); getContext().getTasks().put(taskAttemptId, task); @@ -269,10 +268,11 @@ public void run() { } catch (Throwable t) { LOG.error(t.getMessage(), t); fatalError(qmClientService, taskAttemptId, t.getMessage()); + } finally { if(task != null) { - task.cleanupTask(); + task.cleanup(); } - } finally { + callFuture = null; taskRequest = null; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java index 5c97ba81d8..16d32d443d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java @@ -36,6 +36,7 @@ /** * The history class for TaskRunner processing. */ +@Deprecated public class TaskRunnerHistory implements ProtoObject { private Service.STATE state; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java index 734a8a5b4b..d18a26295e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java @@ -36,6 +36,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; +@Deprecated public class TaskRunnerManager extends CompositeService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class); @@ -154,14 +155,7 @@ public void handle(TaskRunnerEvent event) { if(context == null){ try { - context = new ExecutionBlockContext(getTajoConf(), - getWorkerContext(), - this, - startEvent.getQueryContext(), - startEvent.getPlan(), - startEvent.getExecutionBlockId(), - startEvent.getQueryMaster(), - startEvent.getShuffleType()); + context = new ExecutionBlockContext(getWorkerContext(), this, startEvent.getRequest()); context.init(); } catch (Throwable e) { LOG.fatal(e.getMessage(), e); @@ -170,7 +164,7 @@ public void handle(TaskRunnerEvent event) { executionBlockContextMap.put(event.getExecutionBlockId(), context); } - TaskRunner taskRunner = new TaskRunner(context, startEvent.getContainerId()); + TaskRunner taskRunner = new TaskRunner(context, startEvent.getRequest().getContainerId()); LOG.info("Start TaskRunner:" + taskRunner.getId()); taskRunnerMap.put(taskRunner.getId(), taskRunner); taskRunnerHistoryMap.put(taskRunner.getId(), taskRunner.getHistory()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java new file mode 100644 index 0000000000..85d74e2d5a --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.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.worker.event; + +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.ipc.TajoWorkerProtocol; + +public class ExecutionBlockStartEvent extends TaskManagerEvent { + private TajoWorkerProtocol.RunExecutionBlockRequestProto requestProto; + + public ExecutionBlockStartEvent(TajoWorkerProtocol.RunExecutionBlockRequestProto requestProto) { + super(EventType.EB_START, new ExecutionBlockId(requestProto.getExecutionBlockId())); + this.requestProto = requestProto; + } + + public TajoWorkerProtocol.RunExecutionBlockRequestProto getRequestProto() { + return requestProto; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java new file mode 100644 index 0000000000..2b967ab1d8 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TajoIdProtos; +import org.apache.tajo.ipc.TajoWorkerProtocol; + +public class ExecutionBlockStopEvent extends TaskManagerEvent { + private TajoWorkerProtocol.ExecutionBlockListProto cleanupList; + + public ExecutionBlockStopEvent(TajoIdProtos.ExecutionBlockIdProto executionBlockId, + TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + super(EventType.EB_STOP, new ExecutionBlockId(executionBlockId)); + this.cleanupList = cleanupList; + } + + public TajoWorkerProtocol.ExecutionBlockListProto getCleanupList() { + return cleanupList; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java index 2f411e8c5c..9a1c106865 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -24,7 +24,7 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; -public class NodeResourceAllocateEvent extends NodeResourceManagerEvent { +public class NodeResourceAllocateEvent extends NodeResourceEvent { private BatchAllocationRequestProto request; private RpcCallback callback; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java index a298d77e20..31d9229ad4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java @@ -21,7 +21,7 @@ import org.apache.tajo.TajoProtos; import org.apache.tajo.resource.NodeResource; -public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { +public class NodeResourceDeallocateEvent extends NodeResourceEvent { private NodeResource resource; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java similarity index 80% rename from tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java rename to tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java index bcb34486c0..6fd2e0d517 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java @@ -19,16 +19,17 @@ package org.apache.tajo.worker.event; import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.resource.NodeResource; -public class NodeResourceManagerEvent extends AbstractEvent { +public class NodeResourceEvent extends AbstractEvent { + //consumer: NodeResourceManager public enum EventType { + // producer: TajoWorkerManagerService ALLOCATE, + // producer: TaskExecutor DEALLOCATE } - public NodeResourceManagerEvent(EventType eventType) { + public NodeResourceEvent(EventType eventType) { super(eventType); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java index 58ab74af0a..9eb8ae9d7b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java @@ -22,19 +22,16 @@ import org.apache.tajo.resource.NodeResource; public class NodeStatusEvent extends AbstractEvent { - private final NodeResource resource; + // consumer: NodeStatusUpdater public enum EventType { + // producer: NodeResourceManager REPORT_RESOURCE, + // producer: TaskManager FLUSH_REPORTS } - public NodeStatusEvent(EventType eventType, NodeResource resource) { + public NodeStatusEvent(EventType eventType) { super(eventType); - this.resource = resource; - } - - public NodeResource getResource() { - return resource; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java new file mode 100644 index 0000000000..c609c67150 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.tajo.TaskAttemptId; + +public class TaskExecutorEvent extends AbstractEvent { + + // producer: NodeResourceManager, consumer: TaskExecutorEvent + public enum EventType { + START, + KILL, + ABORT + } + + private TaskAttemptId taskAttemptId; + + public TaskExecutorEvent(EventType eventType, + TaskAttemptId taskAttemptId) { + super(eventType); + this.taskAttemptId = taskAttemptId; + } + + public TaskAttemptId getTaskId() { + return taskAttemptId; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java new file mode 100644 index 0000000000..39b541b097 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.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.worker.event; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TaskAttemptId; + +public class TaskManagerEvent extends AbstractEvent { + // producer: NodeResourceManager, consumer: TaskManager + public enum EventType { + EB_START, + EB_STOP + } + + private ExecutionBlockId executionBlockId; + + public TaskManagerEvent(EventType eventType, + ExecutionBlockId executionBlockId) { + super(eventType); + this.executionBlockId = executionBlockId; + } + + public ExecutionBlockId getExecutionBlockId() { + return executionBlockId; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java index aac8973b81..717525197d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java @@ -21,6 +21,7 @@ import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.ExecutionBlockId; +@Deprecated public class TaskRunnerEvent extends AbstractEvent { public enum EventType { START, diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java index 908afa2293..9406794254 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java @@ -20,48 +20,20 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; - +@Deprecated public class TaskRunnerStartEvent extends TaskRunnerEvent { - private final QueryContext queryContext; - private final WorkerConnectionInfo queryMaster; - private final String containerId; - private final String plan; - private final PlanProto.ShuffleType shuffleType; - - public TaskRunnerStartEvent(WorkerConnectionInfo queryMaster, - ExecutionBlockId executionBlockId, - String containerId, - QueryContext context, - String plan, - PlanProto.ShuffleType shuffleType) { - super(EventType.START, executionBlockId); - this.queryMaster = queryMaster; - this.containerId = containerId; - this.queryContext = context; - this.plan = plan; - this.shuffleType = shuffleType; - } - - public WorkerConnectionInfo getQueryMaster() { - return queryMaster; - } - - public String getContainerId() { - return containerId; - } - - public QueryContext getQueryContext() { - return queryContext; - } + private final TajoWorkerProtocol.RunExecutionBlockRequestProto request; - public String getPlan() { - return plan; + public TaskRunnerStartEvent(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + super(EventType.START, new ExecutionBlockId(request.getExecutionBlockId())); + this.request = request; } - public PlanProto.ShuffleType getShuffleType() { - return shuffleType; + public TajoWorkerProtocol.RunExecutionBlockRequestProto getRequest() { + return request; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java index c8ec20d138..297f30c037 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java @@ -20,6 +20,7 @@ import org.apache.tajo.ExecutionBlockId; +@Deprecated public class TaskRunnerStopEvent extends TaskRunnerEvent { public TaskRunnerStopEvent(ExecutionBlockId executionBlockId) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java new file mode 100644 index 0000000000..f60e7c4212 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.resource.NodeResource; +import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; + +public class TaskStartEvent extends TaskExecutorEvent { + + private NodeResource allocatedResource; + private TaskRequestProto taskRequest; + + public TaskStartEvent(TaskRequestProto taskRequest, + NodeResource allocatedResource) { + super(EventType.START, new TaskAttemptId(taskRequest.getId())); + this.taskRequest = taskRequest; + this.allocatedResource = allocatedResource; + } + + public NodeResource getAllocatedResource() { + return allocatedResource; + } + + public TaskRequestProto getTaskRequest() { + return taskRequest; + } +} diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 2324596b1b..715b1e681a 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -208,6 +208,7 @@ message TaskAllocationRequestProto { message BatchAllocationRequestProto { required ExecutionBlockIdProto executionBlockId = 1; repeated TaskAllocationRequestProto taskRequest = 2; + optional RunExecutionBlockRequestProto executionBlockRequest = 3; //TODO should be refactored } message BatchAllocationResponseProto { diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index eca7f6d7c4..0cec3dac49 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -19,6 +19,7 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Lists; +import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.tajo.*; @@ -33,16 +34,25 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequestImpl; -import org.apache.tajo.ipc.ClientProtos; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; import org.apache.tajo.plan.LogicalOptimizer; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.LogicalPlanner; import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.session.Session; +import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; import org.apache.tajo.worker.ExecutionBlockContext; -import org.apache.tajo.worker.Task; +import org.apache.tajo.worker.LegacyTaskImpl; +import org.apache.tajo.worker.TajoWorker; +import org.apache.tajo.worker.TaskRunnerManager; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -230,7 +240,7 @@ public void testKillTask() throws Throwable { QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId eid = QueryIdFactory.newExecutionBlockId(qid, 1); TaskId tid = QueryIdFactory.newTaskId(eid); - TajoConf conf = new TajoConf(); + final TajoConf conf = new TajoConf(); TaskRequestImpl taskRequest = new TaskRequestImpl(); taskRequest.set(null, new ArrayList(), @@ -238,18 +248,37 @@ public void testKillTask() throws Throwable { taskRequest.setInterQuery(); TaskAttemptId attemptId = new TaskAttemptId(tid, 1); + WorkerConnectionInfo queryMaster = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + requestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + + requestProto.setExecutionBlockId(eid.getProto()) + .setQueryMaster(queryMaster.getProto()) + .setNodeId(queryMaster.getHost()+":" + queryMaster.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + @Override + public TajoConf getConf() { + return conf; + } + }; + ExecutionBlockContext context = - new ExecutionBlockContext(conf, null, null, new QueryContext(conf), null, eid, null, null); + new ExecutionBlockContext(workerContext, null, requestProto.build()); - org.apache.tajo.worker.Task task = new Task("test", CommonTestingUtil.getTestDir(), attemptId, + org.apache.tajo.worker.Task task = new LegacyTaskImpl("test", CommonTestingUtil.getTestDir(), attemptId, conf, context, taskRequest); task.kill(); - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); try { task.run(); - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); } catch (Exception e) { - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); } } @@ -271,4 +300,94 @@ protected void dispatch(Event event) { super.dispatch(event); } } + + abstract class MockWorkerContext implements TajoWorker.WorkerContext { + + @Override + public QueryMaster getQueryMaster() { + return null; + } + + public abstract TajoConf getConf(); + + @Override + public ServiceTracker getServiceTracker() { + return null; + } + + @Override + public QueryMasterManagerService getQueryMasterManagerService() { + return null; + } + + @Override + public TaskRunnerManager getTaskRunnerManager() { + return null; + } + + @Override + public CatalogService getCatalog() { + return null; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + return null; + } + + @Override + public String getWorkerName() { + return null; + } + + @Override + public LocalDirAllocator getLocalDirAllocator() { + return null; + } + + @Override + public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { + return null; + } + + @Override + public TajoSystemMetrics getWorkerSystemMetrics() { + return null; + } + + @Override + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return null; + } + + @Override + public HistoryWriter getTaskHistoryWriter() { + return null; + } + + @Override + public HistoryReader getHistoryReader() { + return null; + } + + @Override + public void cleanup(String strPath) { + + } + + @Override + public void cleanupTemporalDirectories() { + + } + + @Override + public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { + + } + + @Override + public void setNumClusterNodes(int numClusterNodes) { + + } + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java new file mode 100644 index 0000000000..9d4e1f3392 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.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.worker; + +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.ipc.TajoWorkerProtocol; + +import java.io.IOException; + +public class MockExecutionBlock extends ExecutionBlockContext { + + public MockExecutionBlock(TajoWorker.WorkerContext workerContext, + TajoWorkerProtocol.RunExecutionBlockRequestProto request) throws IOException { + super(workerContext, null, request); + } + + @Override + public void init() throws Throwable { + //skip + } + + @Override + public void fatalError(TaskAttemptId taskAttemptId, String message) { + + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java new file mode 100644 index 0000000000..18b9405c8a --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.QueryIdFactory; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.worker.event.NodeResourceEvent; + +import java.util.Queue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; + +public class MockNodeResourceManager extends NodeResourceManager { + + volatile boolean enableTaskHandlerEvent = true; + private final Semaphore barrier; + + public MockNodeResourceManager(Semaphore barrier, Dispatcher dispatcher, EventHandler taskEventHandler) { + super(dispatcher, taskEventHandler); + this.barrier = barrier; + } + + @Override + public void handle(NodeResourceEvent event) { + super.handle(event); + barrier.release(); + } + + @Override + protected void startExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + if(enableTaskHandlerEvent) { + super.startExecutionBlock(request); + } + } + + @Override + protected void startTask(TajoWorkerProtocol.TaskRequestProto request, NodeResource resource) { + if(enableTaskHandlerEvent) { + super.startTask(request, resource); + } + } + + /** + * skip task execution and deallocation for testing + * */ + public void setTaskHandlerEvent(boolean flag) { + enableTaskHandlerEvent = flag; + } + + protected static Queue createTaskRequests( + ExecutionBlockId ebId, int memory, int size) { + + Queue + requestProtoList = new LinkedBlockingQueue(); + for (int i = 0; i < size; i++) { + + TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(ebId, i), 0); + TajoWorkerProtocol.TaskRequestProto.Builder builder = + TajoWorkerProtocol.TaskRequestProto.newBuilder(); + builder.setId(taskAttemptId.getProto()); + builder.setShouldDie(true); + builder.setOutputTable(""); + builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); + builder.setClusteredOutput(false); + + + requestProtoList.add(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + .setResource(NodeResources.createResource(memory).getProto()) + .setTaskRequest(builder.build()).build()); + } + return requestProtoList; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java index 2d7d0be6d8..dfcfd4f435 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java @@ -39,9 +39,9 @@ public class MockNodeStatusUpdater extends NodeStatusUpdater { private Map resources = Maps.newHashMap(); private MockResourceTracker resourceTracker; - public MockNodeStatusUpdater(CountDownLatch barrier, WorkerConnectionInfo connectionInfo, + public MockNodeStatusUpdater(CountDownLatch barrier, TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { - super(connectionInfo, resourceManager); + super(workerContext, resourceManager); this.barrier = barrier; this.resourceTracker = new MockResourceTracker(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java new file mode 100644 index 0000000000..f62733f1a2 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.worker.event.TaskExecutorEvent; + +import java.io.IOException; +import java.util.concurrent.Semaphore; + +public class MockTaskExecutor extends TaskExecutor { + + protected final Semaphore barrier; + + public MockTaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { + super(taskManager, rmEventHandler); + this.barrier = barrier; + } + + @Override + public void handle(TaskExecutorEvent event) { + super.handle(event); + barrier.release(); + } + + @Override + protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + + //ignore status changed log + final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null) { + private TajoProtos.TaskAttemptState state; + + @Override + public TajoProtos.TaskAttemptState getState() { + return state; + } + + @Override + public void setState(TajoProtos.TaskAttemptState state) { + this.state = state; + } + }; + + return new Task() { + @Override + public void init() throws IOException { + + } + + @Override + public void fetch() { + + } + + @Override + public void run() throws Exception { + taskAttemptContext.stop(); + taskAttemptContext.setProgress(1.0f); + taskAttemptContext.setState(TajoProtos.TaskAttemptState.TA_SUCCEEDED); + } + + @Override + public void kill() { + + } + + @Override + public void abort() { + + } + + @Override + public void cleanup() { + + } + + @Override + public boolean hasFetchPhase() { + return false; + } + + @Override + public boolean isProgressChanged() { + return false; + } + + @Override + public boolean isStopped() { + return taskAttemptContext.isStopped(); + } + + @Override + public void updateProgress() { + + } + + @Override + public TaskAttemptContext getTaskContext() { + return taskAttemptContext; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return context; + } + + @Override + public TajoWorkerProtocol.TaskStatusProto getReport() { + TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + builder.setWorkerName("localhost:0"); + builder.setId(taskAttemptContext.getTaskId().getProto()) + .setProgress(taskAttemptContext.getProgress()) + .setState(taskAttemptContext.getState()); + + builder.setInputStats(new TableStats().getProto()); + return builder.build(); + } + }; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java new file mode 100644 index 0000000000..678b0636c4 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.worker.event.TaskManagerEvent; + +import java.io.IOException; +import java.util.concurrent.Semaphore; + +public class MockTaskManager extends TaskManager { + + private final Semaphore barrier; + + public MockTaskManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { + super(dispatcher, workerContext, rmEventHandler); + this.barrier = barrier; + } + + @Override + protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + try { + return new MockExecutionBlock(getWorkerContext(), request); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + protected void stopExecutionBlock(ExecutionBlockContext context, + TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + //skip for testing + } + + @Override + public void handle(TaskManagerEvent event) { + super.handle(event); + barrier.release(); + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java new file mode 100644 index 0000000000..e8c2b9c369 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.querymaster.QueryMaster; +import org.apache.tajo.querymaster.QueryMasterManagerService; +import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; + +public abstract class MockWorkerContext implements TajoWorker.WorkerContext { + TajoSystemMetrics tajoSystemMetrics; + + @Override + public QueryMaster getQueryMaster() { + return null; + } + + public abstract TajoConf getConf(); + + @Override + public ServiceTracker getServiceTracker() { + return null; + } + + @Override + public QueryMasterManagerService getQueryMasterManagerService() { + return null; + } + + @Override + public TaskRunnerManager getTaskRunnerManager() { + return null; + } + + @Override + public CatalogService getCatalog() { + return null; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + return null; + } + + @Override + public String getWorkerName() { + return null; + } + + @Override + public LocalDirAllocator getLocalDirAllocator() { + return null; + } + + @Override + public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { + return null; + } + + @Override + public TajoSystemMetrics getWorkerSystemMetrics() { + + if (tajoSystemMetrics == null) { + tajoSystemMetrics = new TajoSystemMetrics(getConf(), "test-file-group", "localhost"); + tajoSystemMetrics.start(); + } + return tajoSystemMetrics; + } + + @Override + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return null; + } + + @Override + public HistoryWriter getTaskHistoryWriter() { + return null; + } + + @Override + public HistoryReader getHistoryReader() { + return null; + } + + @Override + public void cleanup(String strPath) { + + } + + @Override + public void cleanupTemporalDirectories() { + + } + + @Override + public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { + + } + + @Override + public void setNumClusterNodes(int numClusterNodes) { + + } +} + diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java index 513eb692b7..65627c1767 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java @@ -104,13 +104,13 @@ public void testGet() throws IOException { @Test public void testAdjustFetchProcess() { - assertEquals(0.0f, Task.adjustFetchProcess(0, 0), 0); - assertEquals(0.0f, Task.adjustFetchProcess(10, 10), 0); - assertEquals(0.05f, Task.adjustFetchProcess(10, 9), 0); - assertEquals(0.1f, Task.adjustFetchProcess(10, 8), 0); - assertEquals(0.25f, Task.adjustFetchProcess(10, 5), 0); - assertEquals(0.45f, Task.adjustFetchProcess(10, 1), 0); - assertEquals(0.5f, Task.adjustFetchProcess(10, 0), 0); + assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(0, 0), 0); + assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(10, 10), 0); + assertEquals(0.05f, LegacyTaskImpl.adjustFetchProcess(10, 9), 0); + assertEquals(0.1f, LegacyTaskImpl.adjustFetchProcess(10, 8), 0); + assertEquals(0.25f, LegacyTaskImpl.adjustFetchProcess(10, 5), 0); + assertEquals(0.45f, LegacyTaskImpl.adjustFetchProcess(10, 1), 0); + assertEquals(0.5f, LegacyTaskImpl.adjustFetchProcess(10, 0), 0); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 7407acc1f9..2cee7d025f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -19,13 +19,15 @@ package org.apache.tajo.worker; import com.google.common.collect.Lists; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.tajo.*; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; -import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; @@ -42,9 +44,15 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; public class TestNodeResourceManager { - private NodeResourceManager resourceManager; - private MockNodeStatusUpdater statusUpdater; + private MockNodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private TajoWorker.WorkerContext workerContext; + + private CompositeService service; private int taskMemory; private TajoConf conf; @@ -61,29 +69,55 @@ public void setup() { conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); dispatcher = new AsyncDispatcher(); - dispatcher.init(conf); - dispatcher.start(); - - resourceManager = new NodeResourceManager(dispatcher); - resourceManager.init(conf); - resourceManager.start(); - - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); - statusUpdater.init(conf); - statusUpdater.start(); + taskDispatcher = new AsyncDispatcher(); + + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; + + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); + resourceManager = new MockNodeResourceManager(new Semaphore(0), dispatcher, taskDispatcher.getEventHandler()); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + }; + + service.init(conf); + service.start(); } @After public void tearDown() { - resourceManager.stop(); - statusUpdater.stop(); - dispatcher.stop(); + service.stop(); } @Test public void testNodeResourceAllocateEvent() throws Exception { int requestSize = 4; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -91,14 +125,14 @@ public void testNodeResourceAllocateEvent() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + // allocated all assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); } @@ -106,6 +140,7 @@ public void testNodeResourceAllocateEvent() throws Exception { public void testNodeResourceCancellation() throws Exception { int requestSize = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); int overSize = 10; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -113,18 +148,19 @@ public void testNodeResourceCancellation() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize + overSize)); + requestProto.addAllTaskRequest( + MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize + overSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); assertEquals(overSize, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); } @Test public void testNodeResourceDeallocateEvent() throws Exception { int requestSize = 4; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -132,21 +168,20 @@ public void testNodeResourceDeallocateEvent() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); //deallocate for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { // direct invoke handler for testing resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); } - assertEquals(0, resourceManager.getAllocatedSize()); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); } @@ -154,12 +189,38 @@ public void testNodeResourceDeallocateEvent() throws Exception { public void testParallelRequest() throws Exception { final int parallelCount = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES) * 2; final int taskSize = 100000; + resourceManager.setTaskHandlerEvent(true); + final AtomicInteger totalComplete = new AtomicInteger(); final AtomicInteger totalCanceled = new AtomicInteger(); final ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); - final Queue totalTasks = createTaskRequests(taskMemory, taskSize); + final Queue + totalTasks = MockNodeResourceManager.createTaskRequests(ebId, taskMemory, taskSize); + + // first request with starting ExecutionBlock + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + TaskAllocationRequestProto task = totalTasks.poll(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.addTaskRequest(task); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + CallFuture callFuture = new CallFuture(); + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + assertTrue(callFuture.get().getCancellationTaskCount() == 0); + totalComplete.incrementAndGet(); + // start parallel request ExecutorService executor = Executors.newFixedThreadPool(parallelCount); List futureList = Lists.newArrayList(); @@ -187,7 +248,6 @@ public void run() { totalCanceled.addAndGet(proto.getCancellationTaskCount()); } else { complete++; - dispatcher.getEventHandler().handle(new NodeResourceDeallocateEvent(task.getResource())); } } catch (Exception e) { fail(e.getMessage()); @@ -209,27 +269,4 @@ public void run() { executor.shutdown(); assertEquals(taskSize, totalComplete.get()); } - - protected static Queue createTaskRequests(int memory, int size) { - Queue requestProtoList = new LinkedBlockingQueue(); - for (int i = 0; i < size; i++) { - - ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0); - TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, i), 0); - - TajoWorkerProtocol.TaskRequestProto.Builder builder = - TajoWorkerProtocol.TaskRequestProto.newBuilder(); - builder.setId(taskAttemptId.getProto()); - builder.setShouldDie(true); - builder.setOutputTable(""); - builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); - builder.setClusteredOutput(false); - - - requestProtoList.add(TaskAllocationRequestProto.newBuilder() - .setResource(NodeResources.createResource(memory).getProto()) - .setTaskRequest(builder.build()).build()); - } - return requestProtoList; - } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index fb3c77e27c..af40554f31 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -22,6 +22,7 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.master.rm.Worker; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeStatusEvent; import org.junit.After; @@ -37,18 +38,36 @@ public class TestNodeStatusUpdater { private MockNodeStatusUpdater statusUpdater; private AsyncDispatcher dispatcher; private TajoConf conf; + private TajoWorker.WorkerContext workerContext; + @Before public void setup() { conf = new TajoConf(); conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL, 1000); dispatcher = new AsyncDispatcher(); dispatcher.init(conf); dispatcher.start(); - resourceManager = new NodeResourceManager(dispatcher); + resourceManager = new NodeResourceManager(dispatcher, null); resourceManager.init(conf); resourceManager.start(); } @@ -63,27 +82,25 @@ public void tearDown() { @Test(timeout = 20000) public void testNodeMembership() throws Exception { CountDownLatch barrier = new CountDownLatch(1); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); MockNodeStatusUpdater.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); barrier.await(); - assertTrue(resourceTracker.getTotalResource().containsKey(worker.getId())); + assertTrue(resourceTracker.getTotalResource().containsKey(workerContext.getConnectionInfo().getId())); assertEquals(resourceManager.getTotalResource(), - resourceTracker.getTotalResource().get(worker.getId())); + resourceTracker.getTotalResource().get(workerContext.getConnectionInfo().getId())); assertEquals(resourceManager.getAvailableResource(), - resourceTracker.getAvailableResource().get(worker.getId())); + resourceTracker.getAvailableResource().get(workerContext.getConnectionInfo().getId())); } @Test(timeout = 20000) public void testPing() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); @@ -100,16 +117,29 @@ public void testPing() throws Exception { @Test(timeout = 20000) public void testResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); + assertEquals(0, statusUpdater.getQueueSize()); for (int i = 0; i < statusUpdater.getQueueingLimit(); i++) { - dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, - resourceManager.getAvailableResource())); + dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); } barrier.await(); assertEquals(0, statusUpdater.getQueueSize()); } + + @Test(timeout = 20000) + public void testFlushResourceReport() throws Exception { + CountDownLatch barrier = new CountDownLatch(2); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); + statusUpdater.init(conf); + statusUpdater.start(); + + assertEquals(0, statusUpdater.getQueueSize()); + dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + + barrier.await(); + assertEquals(0, statusUpdater.getQueueSize()); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java new file mode 100644 index 0000000000..98b187b013 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -0,0 +1,330 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.*; +import org.apache.tajo.annotation.ThreadSafe; +import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.querymaster.QueryMaster; +import org.apache.tajo.querymaster.QueryMasterManagerService; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; +import org.apache.tajo.worker.event.ExecutionBlockStartEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Queue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.junit.Assert.*; + +public class TestTaskExecutor { + + private NodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; + private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private TajoWorker.WorkerContext workerContext; + + private CompositeService service; + private TajoConf conf; + private Semaphore barrier; + private Semaphore resourceManagerBarrier; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + dispatcher = new AsyncDispatcher(); + taskDispatcher = new AsyncDispatcher(); + + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; + + barrier = new Semaphore(0); + resourceManagerBarrier = new Semaphore(0); + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new TaskExecutor(barrier, taskManager, dispatcher.getEventHandler()); + resourceManager = new MockNodeResourceManager(resourceManagerBarrier, dispatcher, taskDispatcher.getEventHandler()); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } + }; + + service.init(conf); + service.start(); + } + + @After + public void tearDown() { + service.stop(); + } + + @Test + public void testTaskRequest() throws Exception { + int requestSize = 1; + + RunExecutionBlockRequestProto.Builder + ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + //verify running task + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(1, taskExecutor.getRunningTasks()); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(0, taskExecutor.getRunningTasks()); + assertEquals(1, taskExecutor.completeTasks); + + //verify the released resources + Thread.sleep(100); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + } + + @Test + public void testTaskException() throws Exception { + int requestSize = 1; + + RunExecutionBlockRequestProto.Builder + ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost()+":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); + + taskExecutor.throwException.set(true); + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + //verify running task + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(1, taskExecutor.getRunningTasks()); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(0, taskExecutor.getRunningTasks()); + assertEquals(0, taskExecutor.completeTasks); + + //verify the released resources + Thread.sleep(100); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + } + + class TaskExecutor extends MockTaskExecutor { + int completeTasks; + AtomicBoolean throwException = new AtomicBoolean(); + + public TaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { + super(barrier, taskManager, rmEventHandler); + } + + @Override + protected void stopTask(TaskAttemptId taskId) { + super.stopTask(taskId); + super.barrier.release(); + } + + @Override + protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null); + + return new Task() { + @Override + public void init() throws IOException { + + try { + Thread.sleep(50); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + @Override + public void fetch() { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + @Override + public void run() throws Exception { + Thread.sleep(50); + + if(throwException.get()) throw new RuntimeException(); + + taskAttemptContext.stop(); + taskAttemptContext.setProgress(1.0f); + taskAttemptContext.setState(TajoProtos.TaskAttemptState.TA_SUCCEEDED); + completeTasks++; + } + + @Override + public void kill() { + + } + + @Override + public void abort() { + + } + + @Override + public void cleanup() { + } + + @Override + public boolean hasFetchPhase() { + return false; + } + + @Override + public boolean isProgressChanged() { + return false; + } + + @Override + public boolean isStopped() { + return taskAttemptContext.isStopped(); + } + + @Override + public void updateProgress() { + + } + + @Override + public TaskAttemptContext getTaskContext() { + return taskAttemptContext; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return context; + } + + @Override + public TajoWorkerProtocol.TaskStatusProto getReport() { + TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + builder.setWorkerName("localhost:0"); + builder.setId(taskAttemptContext.getTaskId().getProto()) + .setProgress(taskAttemptContext.getProgress()) + .setState(taskAttemptContext.getState()); + + builder.setInputStats(new TableStats().getProto()); + return builder.build(); + } + }; + } + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java new file mode 100644 index 0000000000..8bca489e20 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -0,0 +1,185 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.tajo.*; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.worker.event.ExecutionBlockStartEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.concurrent.*; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.junit.Assert.*; + +public class TestTaskManager { + + private NodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; + private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private TajoWorker.WorkerContext workerContext; + + private CompositeService service; + private int taskMemory; + private TajoConf conf; + private Semaphore barrier; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + + taskMemory = 512; + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, + taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); + + dispatcher = new AsyncDispatcher(); + taskDispatcher = new AsyncDispatcher(); + + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; + barrier = new Semaphore(0); + taskManager = new MockTaskManager(barrier, taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); + resourceManager = new NodeResourceManager(dispatcher, taskDispatcher.getEventHandler()); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } + }; + + service.init(conf); + service.start(); + } + + @After + public void tearDown() { + service.stop(); + } + + @Test(timeout = 10000) + public void testExecutionBlockStart() throws Exception { + int requestSize = 1; + + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNotNull(taskManager.getExecutionBlockContext(ebId)); + assertEquals(ebId, taskManager.getExecutionBlockContext(ebId).getExecutionBlockId()); + } + + @Test(timeout = 10000) + public void testExecutionBlockStop() throws Exception { + + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost()+":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + taskDispatcher.getEventHandler().handle(new ExecutionBlockStartEvent(ebRequestProto.build())); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNotNull(taskManager.getExecutionBlockContext(ebId)); + assertEquals(ebId, taskManager.getExecutionBlockContext(ebId).getExecutionBlockId()); + + ExecutionBlockListProto.Builder ebList = ExecutionBlockListProto.newBuilder(); + taskDispatcher.getEventHandler().handle(new ExecutionBlockStopEvent(ebId.getProto(), ebList.build())); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNull(taskManager.getExecutionBlockContext(ebId)); + } +} From dc4904987dc16164dca4ff37c1178698537201c6 Mon Sep 17 00:00:00 2001 From: DaeMyung Kang Date: Sat, 13 Jun 2015 18:49:04 +0900 Subject: [PATCH 3/7] TAJO-1636: query rest api uri should change from /databases/{database_name}/queies to /queries. Closes #598 Signed-off-by: JaeHwa Jung --- CHANGES | 4 ++ .../tajo/ws/rs/resources/QueryResource.java | 37 +++++++------------ .../ws/rs/resources/QueryResultResource.java | 21 +---------- .../ws/rs/resources/TestQueryResource.java | 2 +- .../rs/resources/TestQueryResultResource.java | 2 +- 5 files changed, 21 insertions(+), 45 deletions(-) diff --git a/CHANGES b/CHANGES index 066e08612b..045f94465a 100644 --- a/CHANGES +++ b/CHANGES @@ -27,6 +27,10 @@ Release 0.11.0 - unreleased IMPROVEMENT + TAJO-1636: query rest api uri should change + from /databases/{database_name}/queies to /queries. + (Contributed by DaeMyung Kang, Committed by jaehwa) + TAJO-1624: Add managed table or external description in Table management section. (hyunsik) diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java index 8f45c7b59f..a662c4d877 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java @@ -44,7 +44,7 @@ import java.util.List; import java.util.Map; -@Path("/databases/{databaseName}/queries") +@Path("/queries") public class QueryResource { private static final Log LOG = LogFactory.getLog(QueryResource.class); @@ -54,15 +54,11 @@ public class QueryResource { @Context Application application; - - @PathParam("databaseName") - String databaseName; - + JerseyResourceDelegateContext context; protected static final String tajoSessionIdHeaderName = "X-Tajo-Session"; - private static final String databaseNameKeyName = "databaseName"; private static final String stateKeyName = "state"; private static final String startTimeKeyName = "startTime"; private static final String endTimeKeyName = "endTime"; @@ -70,7 +66,6 @@ public class QueryResource { private static final String submitQueryRequestKeyName = "submitQueryRequest"; private static final String printTypeKeyName = "printType"; private static final String queryIdKeyName = "queryId"; - private static final String defaultQueryInfoPrintType = "COMPLICATED"; private void initializeContext() { @@ -78,9 +73,6 @@ private void initializeContext() { JerseyResourceDelegateContextKey uriInfoKey = JerseyResourceDelegateContextKey.valueOf(JerseyResourceDelegateUtil.UriInfoKey, UriInfo.class); context.put(uriInfoKey, uriInfo); - JerseyResourceDelegateContextKey databaseNameKey = - JerseyResourceDelegateContextKey.valueOf(databaseNameKeyName, String.class); - context.put(databaseNameKey, databaseName); } @GET @@ -218,9 +210,9 @@ public Response submitQuery(@HeaderParam(tajoSessionIdHeaderName) String session if (LOG.isDebugEnabled()) { LOG.debug("Client sent a submit query request."); } - + Response response = null; - + try { initializeContext(); JerseyResourceDelegateContextKey sessionIdKey = @@ -229,21 +221,22 @@ public Response submitQuery(@HeaderParam(tajoSessionIdHeaderName) String session JerseyResourceDelegateContextKey submitQueryRequestKey = JerseyResourceDelegateContextKey.valueOf(submitQueryRequestKeyName, SubmitQueryRequest.class); context.put(submitQueryRequestKey, request); - + response = JerseyResourceDelegateUtil.runJerseyResourceDelegate( new SubmitQueryDelegate(), application, context, LOG); + } catch (Throwable e) { LOG.error(e.getMessage(), e); - + response = ResourcesUtil.createExceptionResponse(null, e.getMessage()); } - + return response; } - + private static class SubmitQueryDelegate implements JerseyResourceDelegate { @Override @@ -254,20 +247,17 @@ public Response run(JerseyResourceDelegateContext context) { JerseyResourceDelegateContextKey submitQueryRequestKey = JerseyResourceDelegateContextKey.valueOf(submitQueryRequestKeyName, SubmitQueryRequest.class); SubmitQueryRequest request = context.get(submitQueryRequestKey); - JerseyResourceDelegateContextKey databaseNameKey = - JerseyResourceDelegateContextKey.valueOf(databaseNameKeyName, String.class); - String databaseName = context.get(databaseNameKey); JerseyResourceDelegateContextKey masterContextKey = JerseyResourceDelegateContextKey.valueOf(JerseyResourceDelegateUtil.MasterContextKey, MasterContext.class); MasterContext masterContext = context.get(masterContextKey); - + if (sessionId == null || sessionId.isEmpty()) { return ResourcesUtil.createBadRequestResponse(LOG, "Session Id is null or empty string."); } if (request == null || request.getQuery() == null || request.getQuery().isEmpty()) { return ResourcesUtil.createBadRequestResponse(LOG, "query is null or emptry string."); } - + Session session; try { session = masterContext.getSessionManager().getSession(sessionId); @@ -287,7 +277,7 @@ public Response run(JerseyResourceDelegateContext context) { URI queryURI = uriInfo.getBaseUriBuilder() .path(QueryResource.class) .path(QueryResource.class, "getQuery") - .build(databaseName, new QueryId(response.getQueryId()).toString()); + .build(new QueryId(response.getQueryId()).toString()); return Response.created(queryURI).build(); } } @@ -422,13 +412,12 @@ public Response run(JerseyResourceDelegateContext context) { return Response.ok().build(); } } - + @Path("/{queryId}/result") public QueryResultResource getQueryResult(@PathParam("queryId") String queryId) { QueryResultResource queryResultResource = new QueryResultResource(); queryResultResource.setUriInfo(uriInfo); queryResultResource.setApplication(application); - queryResultResource.setDatabaseName(databaseName); queryResultResource.setQueryId(queryId); return queryResultResource; } diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java index 92b95d24f0..6d110e5c19 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java @@ -58,13 +58,10 @@ public class QueryResultResource { private Application application; - private String databaseName; - private String queryId; private JerseyResourceDelegateContext context; - private static final String databaseNameKeyName = "databaseName"; private static final String queryIdKeyName = "queryId"; private static final String sessionIdKeyName = "sessionId"; private static final String cacheIdKeyName = "cacheId"; @@ -89,14 +86,6 @@ public void setApplication(Application application) { this.application = application; } - public String getDatabaseName() { - return databaseName; - } - - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - public String getQueryId() { return queryId; } @@ -110,9 +99,6 @@ private void initializeContext() { JerseyResourceDelegateContextKey uriInfoKey = JerseyResourceDelegateContextKey.valueOf(JerseyResourceDelegateUtil.UriInfoKey, UriInfo.class); context.put(uriInfoKey, uriInfo); - JerseyResourceDelegateContextKey databaseNameKey = - JerseyResourceDelegateContextKey.valueOf(databaseNameKeyName, String.class); - context.put(databaseNameKey, databaseName); JerseyResourceDelegateContextKey queryIdKey = JerseyResourceDelegateContextKey.valueOf(queryIdKeyName, String.class); context.put(queryIdKey, queryId); @@ -200,10 +186,7 @@ public Response run(JerseyResourceDelegateContext context) { JerseyResourceDelegateContextKey uriInfoKey = JerseyResourceDelegateContextKey.valueOf(JerseyResourceDelegateUtil.UriInfoKey, UriInfo.class); UriInfo uriInfo = context.get(uriInfoKey); - JerseyResourceDelegateContextKey databaseNameKey = - JerseyResourceDelegateContextKey.valueOf(databaseNameKeyName, String.class); - String databaseName = context.get(databaseNameKey); - + try { masterContext.getSessionManager().touch(sessionId); Session session = masterContext.getSessionManager().getSession(sessionId); @@ -234,7 +217,7 @@ public Response run(JerseyResourceDelegateContext context) { .path(QueryResource.class) .path(QueryResource.class, "getQueryResult") .path(QueryResultResource.class, "getQueryResultSet") - .build(databaseName, queryId, cacheId); + .build(queryId, cacheId); ResultSetInfoResponse resultSetInfoResponse = new ResultSetInfoResponse(); resultSetInfoResponse.setId(cacheId); resultSetInfoResponse.setLink(resultSetCacheUri); diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java index 77c0117864..5d0b150c72 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java @@ -65,7 +65,7 @@ public void setUp() throws Exception { int restPort = testBase.getTestingCluster().getConfiguration().getIntVar(ConfVars.REST_SERVICE_PORT); restServiceURI = new URI("http", null, "127.0.0.1", restPort, "/rest", null, null); sessionsURI = new URI(restServiceURI + "/sessions"); - queriesURI = new URI(restServiceURI + "/databases/" + TajoConstants.DEFAULT_DATABASE_NAME + "/queries"); + queriesURI = new URI(restServiceURI + "/queries"); restClient = ClientBuilder.newBuilder() .register(new GsonFeature(RestTestUtils.registerTypeAdapterMap())) .register(LoggingFilter.class) diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java index d83eb6064c..d4f17855f2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java @@ -73,7 +73,7 @@ public void setUp() throws Exception { int restPort = testBase.getTestingCluster().getConfiguration().getIntVar(ConfVars.REST_SERVICE_PORT); restServiceURI = new URI("http", null, "127.0.0.1", restPort, "/rest", null, null); sessionsURI = new URI(restServiceURI + "/sessions"); - queriesURI = new URI(restServiceURI + "/databases/" + TajoConstants.DEFAULT_DATABASE_NAME + "/queries"); + queriesURI = new URI(restServiceURI + "/queries"); restClient = ClientBuilder.newBuilder() .register(new GsonFeature(RestTestUtils.registerTypeAdapterMap())) .register(LoggingFilter.class) From 011de8bab375b01d8c2d07abebb6bf7ca94ae8c0 Mon Sep 17 00:00:00 2001 From: DaeMyung Kang Date: Tue, 16 Jun 2015 06:43:49 +0900 Subject: [PATCH 4/7] TAJO-1634: REST API: fix error when offset is zero. Closes #597 Signed-off-by: JaeHwa Jung --- CHANGES | 3 +++ .../org/apache/tajo/ws/rs/resources/QueryResultResource.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES b/CHANGES index 045f94465a..57b8baacc4 100644 --- a/CHANGES +++ b/CHANGES @@ -154,6 +154,9 @@ Release 0.11.0 - unreleased BUG FIXES + TAJO-1634: REST API: fix error when offset is zero. + (Contributed by DaeMyung Kang, Committed by jaehwa) + TAJO-1630: Test failure after TAJO-1130. (jihoon) TAJO-1623: INSERT INTO with wrong target columns causes NPE. (hyunsik) diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java index 6d110e5c19..3384c90f84 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java @@ -349,7 +349,7 @@ public Response run(JerseyResourceDelegateContext context) { } private void skipOffsetRow(NonForwardQueryResultScanner queryResultScanner, int offset) throws IOException { - if (offset < 0) { + if (offset <= 0) { return; } From d926247dcaba0af8d810c54a7c668272c3c86ea5 Mon Sep 17 00:00:00 2001 From: Hyunsik Choi Date: Mon, 15 Jun 2015 19:33:33 -0700 Subject: [PATCH 5/7] TAJO-1633: Cleanup TajoMasterClientService. Closes #594 --- .../tajo/client/CatalogAdminClientImpl.java | 14 +++-- .../apache/tajo/client/QueryClientImpl.java | 8 +-- tajo-client/src/main/proto/ClientProtos.proto | 16 +---- .../main/proto/TajoMasterClientProtocol.proto | 8 +-- .../tajo/master/TajoMasterClientService.java | 60 +++++++++---------- 5 files changed, 47 insertions(+), 59 deletions(-) diff --git a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java index 9397fcf3e4..1fe856a393 100644 --- a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java +++ b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java @@ -27,8 +27,10 @@ import org.apache.tajo.catalog.partition.PartitionMethodDesc; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.ipc.ClientProtos; +import org.apache.tajo.ipc.ClientProtos.SessionedStringProto; import org.apache.tajo.jdbc.SQLStates; import org.apache.tajo.rpc.NettyClientBase; +import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import java.io.IOException; import java.net.URI; @@ -145,13 +147,13 @@ public List getTableList(@Nullable final String databaseName) throws Ser connection.checkSessionAndGet(client); BlockingInterface tajoMasterService = client.getStub(); - ClientProtos.GetTableListRequest.Builder builder = ClientProtos.GetTableListRequest.newBuilder(); + SessionedStringProto.Builder builder = SessionedStringProto.newBuilder(); builder.setSessionId(connection.sessionId); if (databaseName != null) { - builder.setDatabaseName(databaseName); + builder.setValue(databaseName); } - ClientProtos.GetTableListResponse res = tajoMasterService.getTableList(null, builder.build()); - return res.getTablesList(); + PrimitiveProtos.StringListProto res = tajoMasterService.getTableList(null, builder.build()); + return res.getValuesList(); } @Override @@ -161,9 +163,9 @@ public TableDesc getTableDesc(final String tableName) throws ServiceException { connection.checkSessionAndGet(client); BlockingInterface tajoMasterService = client.getStub(); - ClientProtos.GetTableDescRequest.Builder builder = ClientProtos.GetTableDescRequest.newBuilder(); + SessionedStringProto.Builder builder = SessionedStringProto.newBuilder(); builder.setSessionId(connection.sessionId); - builder.setTableName(tableName); + builder.setValue(tableName); ClientProtos.TableResponse res = tajoMasterService.getTableDesc(null, builder.build()); if (res.getResultCode() == ClientProtos.ResultCode.OK) { return CatalogUtil.newTableDesc(res.getTableDesc()); diff --git a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java index 53889fe016..ac25933c90 100644 --- a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java +++ b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java @@ -437,8 +437,8 @@ public List getRunningQueryList() throws ServiceExc connection.checkSessionAndGet(client); TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub(); - ClientProtos.GetQueryListRequest.Builder builder = ClientProtos.GetQueryListRequest.newBuilder(); - builder.setSessionId(connection.sessionId); + TajoIdProtos.SessionIdProto.Builder builder = TajoIdProtos.SessionIdProto.newBuilder(); + builder.setId(connection.sessionId.getId()); ClientProtos.GetQueryListResponse res = tajoMasterService.getRunningQueryList(null, builder.build()); return res.getQueryListList(); } @@ -450,8 +450,8 @@ public List getFinishedQueryList() throws ServiceEx connection.checkSessionAndGet(client); TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub(); - ClientProtos.GetQueryListRequest.Builder builder = ClientProtos.GetQueryListRequest.newBuilder(); - builder.setSessionId(connection.sessionId); + TajoIdProtos.SessionIdProto.Builder builder = TajoIdProtos.SessionIdProto.newBuilder(); + builder.setId(connection.sessionId.getId()); ClientProtos.GetQueryListResponse res = tajoMasterService.getFinishedQueryList(null, builder.build()); return res.getQueryListList(); } diff --git a/tajo-client/src/main/proto/ClientProtos.proto b/tajo-client/src/main/proto/ClientProtos.proto index ecb136e13e..5497faade5 100644 --- a/tajo-client/src/main/proto/ClientProtos.proto +++ b/tajo-client/src/main/proto/ClientProtos.proto @@ -57,7 +57,7 @@ message SessionUpdateResponse { message SessionedStringProto { optional SessionIdProto sessionId = 1; - required string value = 2; + optional string value = 2; } message ExplainQueryResponse { @@ -196,20 +196,6 @@ message GetClusterInfoResponse { repeated WorkerResourceInfo workerList = 1; } -message GetTableListRequest { - optional SessionIdProto sessionId = 1; - optional string databaseName = 2; -} - -message GetTableListResponse { - repeated string tables = 1; -} - -message GetTableDescRequest { - optional SessionIdProto sessionId = 1; - required string tableName = 2; -} - message CreateTableRequest { optional SessionIdProto sessionId = 1; required string name = 2; diff --git a/tajo-client/src/main/proto/TajoMasterClientProtocol.proto b/tajo-client/src/main/proto/TajoMasterClientProtocol.proto index 10ca268540..468a998d97 100644 --- a/tajo-client/src/main/proto/TajoMasterClientProtocol.proto +++ b/tajo-client/src/main/proto/TajoMasterClientProtocol.proto @@ -46,8 +46,8 @@ service TajoMasterClientProtocolService { // Query And Resource Management APIs rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse); - rpc getRunningQueryList(GetQueryListRequest) returns (GetQueryListResponse); - rpc getFinishedQueryList(GetQueryListRequest) returns (GetQueryListResponse); + rpc getRunningQueryList(SessionIdProto) returns (GetQueryListResponse); + rpc getFinishedQueryList(SessionIdProto) returns (GetQueryListResponse); rpc killQuery(QueryIdRequest) returns (BoolProto); rpc getClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse); rpc closeNonForwardQuery(QueryIdRequest) returns (BoolProto); @@ -65,7 +65,7 @@ service TajoMasterClientProtocolService { rpc createExternalTable(CreateTableRequest) returns (TableResponse); rpc existTable(SessionedStringProto) returns (BoolProto); rpc dropTable(DropTableRequest) returns (BoolProto); - rpc getTableList(GetTableListRequest) returns (GetTableListResponse); - rpc getTableDesc(GetTableDescRequest) returns (TableResponse); + rpc getTableList(SessionedStringProto) returns (StringListProto); + rpc getTableDesc(SessionedStringProto) returns (TableResponse); rpc getFunctionList(SessionedStringProto) returns (FunctionResponse); } 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 4fcdc888e0..2602d7dc4b 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 @@ -47,19 +47,20 @@ import org.apache.tajo.master.TajoMaster.MasterContext; import org.apache.tajo.master.exec.NonForwardQueryResultFileScanner; import org.apache.tajo.master.exec.NonForwardQueryResultScanner; +import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.master.rm.WorkerResource; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.logical.PartitionedTableScanNode; import org.apache.tajo.plan.logical.ScanNode; import org.apache.tajo.querymaster.QueryJobEvent; -import org.apache.tajo.master.rm.Worker; -import org.apache.tajo.master.rm.WorkerResource; -import org.apache.tajo.session.InvalidSessionException; -import org.apache.tajo.session.NoSuchSessionVariableException; -import org.apache.tajo.session.Session; import org.apache.tajo.rpc.BlockingRpcServer; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto; +import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListProto; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto; +import org.apache.tajo.session.InvalidSessionException; +import org.apache.tajo.session.NoSuchSessionVariableException; +import org.apache.tajo.session.Session; import org.apache.tajo.util.KeyValueSet; import org.apache.tajo.util.NetUtils; import org.apache.tajo.util.ProtoUtil; @@ -307,11 +308,6 @@ public UpdateQueryResponse updateQuery(RpcController controller, QueryRequest re try { Session session = context.getSessionManager().getSession(request.getSessionId().getId()); QueryContext queryContext = new QueryContext(conf, session); - if (queryContext.getCurrentDatabase() == null) { - for (Map.Entry e : queryContext.getAllKeyValus().entrySet()) { - System.out.println(e.getKey() + "=" + e.getValue()); - } - } UpdateQueryResponse.Builder builder = UpdateQueryResponse.newBuilder(); try { @@ -379,12 +375,12 @@ public GetQueryResultResponse getQueryResult(RpcController controller, } @Override - public GetQueryListResponse getRunningQueryList(RpcController controller, GetQueryListRequest request) + public GetQueryListResponse getRunningQueryList(RpcController controller, TajoIdProtos.SessionIdProto request) throws ServiceException { try { - context.getSessionManager().touch(request.getSessionId().getId()); + context.getSessionManager().touch(request.getId()); GetQueryListResponse.Builder builder= GetQueryListResponse.newBuilder(); Collection queries = new ArrayList(context.getQueryJobManager().getSubmittedQueries()); @@ -416,11 +412,11 @@ public GetQueryListResponse getRunningQueryList(RpcController controller, GetQue } @Override - public GetQueryListResponse getFinishedQueryList(RpcController controller, GetQueryListRequest request) + public GetQueryListResponse getFinishedQueryList(RpcController controller, TajoIdProtos.SessionIdProto request) throws ServiceException { try { - context.getSessionManager().touch(request.getSessionId().getId()); + context.getSessionManager().touch(request.getId()); GetQueryListResponse.Builder builder = GetQueryListResponse.newBuilder(); Collection queries @@ -723,7 +719,7 @@ public BoolProto dropDatabase(RpcController controller, SessionedStringProto req } @Override - public PrimitiveProtos.StringListProto getAllDatabases(RpcController controller, TajoIdProtos.SessionIdProto + public StringListProto getAllDatabases(RpcController controller, TajoIdProtos.SessionIdProto request) throws ServiceException { try { context.getSessionManager().touch(request.getId()); @@ -749,10 +745,6 @@ public BoolProto existTable(RpcController controller, SessionedStringProto reque tableName = request.getValue(); } - if (databaseName == null) { - System.out.println("A"); - } - if (catalog.existsTable(databaseName, tableName)) { return BOOL_TRUE; } else { @@ -764,19 +756,19 @@ public BoolProto existTable(RpcController controller, SessionedStringProto reque } @Override - public GetTableListResponse getTableList(RpcController controller, - GetTableListRequest request) throws ServiceException { + public StringListProto getTableList(RpcController controller, + SessionedStringProto request) throws ServiceException { try { Session session = context.getSessionManager().getSession(request.getSessionId().getId()); String databaseName; - if (request.hasDatabaseName()) { - databaseName = request.getDatabaseName(); + if (request.hasValue()) { + databaseName = request.getValue(); } else { databaseName = session.getCurrentDatabase(); } Collection tableNames = catalog.getAllTableNames(databaseName); - GetTableListResponse.Builder builder = GetTableListResponse.newBuilder(); - builder.addAllTables(tableNames); + StringListProto.Builder builder = StringListProto.newBuilder(); + builder.addAllValues(tableNames); return builder.build(); } catch (Throwable t) { throw new ServiceException(t); @@ -784,19 +776,27 @@ public GetTableListResponse getTableList(RpcController controller, } @Override - public TableResponse getTableDesc(RpcController controller, GetTableDescRequest request) throws ServiceException { + public TableResponse getTableDesc(RpcController controller, SessionedStringProto request) throws ServiceException { try { + + if (!request.hasValue()) { + return TableResponse.newBuilder() + .setResultCode(ResultCode.ERROR) + .setErrorMessage("table name is required.") + .build(); + } + Session session = context.getSessionManager().getSession(request.getSessionId().getId()); String databaseName; String tableName; - if (CatalogUtil.isFQTableName(request.getTableName())) { - String [] splitted = CatalogUtil.splitFQTableName(request.getTableName()); + if (CatalogUtil.isFQTableName(request.getValue())) { + String [] splitted = CatalogUtil.splitFQTableName(request.getValue()); databaseName = splitted[0]; tableName = splitted[1]; } else { databaseName = session.getCurrentDatabase(); - tableName = request.getTableName(); + tableName = request.getValue(); } if (catalog.existsTable(databaseName, tableName)) { @@ -807,7 +807,7 @@ public TableResponse getTableDesc(RpcController controller, GetTableDescRequest } else { return TableResponse.newBuilder() .setResultCode(ResultCode.ERROR) - .setErrorMessage("ERROR: no such a table: " + request.getTableName()) + .setErrorMessage("ERROR: no such a table: " + request.getValue()) .build(); } } catch (Throwable t) { From 19840f5c1f77288f2f03fe664a07c6c6c379ff82 Mon Sep 17 00:00:00 2001 From: "clark.kang" Date: Fri, 5 Jun 2015 22:56:42 +0900 Subject: [PATCH 6/7] remove-offset --- .../ws/rs/resources/QueryResultResource.java | 35 +++------ .../rs/resources/TestQueryResultResource.java | 75 +++---------------- 2 files changed, 21 insertions(+), 89 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java index 3384c90f84..2f521986bb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResultResource.java @@ -69,6 +69,9 @@ public class QueryResultResource { private static final String countKeyName = "count"; private static final String tajoDigestHeaderName = "X-Tajo-Digest"; + private static final String tajoOffsetHeaderName = "X-Tajo-Offset"; + private static final String tajoCountHeaderName = "X-Tajo-Count"; + private static final String tajoEOSHeaderName = "X-Tajo-EOS"; public UriInfo getUriInfo() { return uriInfo; @@ -242,7 +245,6 @@ public Response run(JerseyResourceDelegateContext context) { @Produces(MediaType.APPLICATION_OCTET_STREAM) public Response getQueryResultSet(@HeaderParam(QueryResource.tajoSessionIdHeaderName) String sessionId, @PathParam("cacheId") String cacheId, - @DefaultValue("-1") @QueryParam("offset") int offset, @DefaultValue("100") @QueryParam("count") int count) { if (LOG.isDebugEnabled()) { LOG.debug("Client sent a get query result set request."); @@ -257,9 +259,6 @@ public Response getQueryResultSet(@HeaderParam(QueryResource.tajoSessionIdHeader JerseyResourceDelegateContextKey cacheIdKey = JerseyResourceDelegateContextKey.valueOf(cacheIdKeyName, Long.class); context.put(cacheIdKey, Long.valueOf(cacheId)); - JerseyResourceDelegateContextKey offsetKey = - JerseyResourceDelegateContextKey.valueOf(offsetKeyName, Integer.class); - context.put(offsetKey, offset); JerseyResourceDelegateContextKey countKey = JerseyResourceDelegateContextKey.valueOf(countKeyName, Integer.class); context.put(countKey, count); @@ -294,9 +293,6 @@ public Response run(JerseyResourceDelegateContext context) { JerseyResourceDelegateContextKey clientApplicationKey = JerseyResourceDelegateContextKey.valueOf(JerseyResourceDelegateUtil.ClientApplicationKey, ClientApplication.class); ClientApplication clientApplication = context.get(clientApplicationKey); - JerseyResourceDelegateContextKey offsetKey = - JerseyResourceDelegateContextKey.valueOf(offsetKeyName, Integer.class); - int offset = context.get(offsetKey); JerseyResourceDelegateContextKey countKey = JerseyResourceDelegateContextKey.valueOf(countKeyName, Integer.class); int count = context.get(countKey); @@ -329,14 +325,17 @@ public Response run(JerseyResourceDelegateContext context) { clientApplication.getCachedNonForwardResultScanner(queryIdObj, cacheId.longValue()); try { - skipOffsetRow(cachedQueryResultScanner, offset); - + int start_offset = cachedQueryResultScanner.getCurrentRowNumber(); List output = cachedQueryResultScanner.getNextRows(count); String digestString = getEncodedBase64DigestString(output); + boolean eos = count != output.size(); return Response.ok(new QueryResultStreamingOutput(output)) - .header(tajoDigestHeaderName, digestString) - .build(); + .header(tajoDigestHeaderName, digestString) + .header(tajoOffsetHeaderName, start_offset) + .header(tajoCountHeaderName, output.size()) + .header(tajoEOSHeaderName, eos) + .build(); } catch (IOException e) { LOG.error(e.getMessage(), e); @@ -348,20 +347,6 @@ public Response run(JerseyResourceDelegateContext context) { } } - private void skipOffsetRow(NonForwardQueryResultScanner queryResultScanner, int offset) throws IOException { - if (offset <= 0) { - return; - } - - int currentRow = queryResultScanner.getCurrentRowNumber(); - - if (offset < (currentRow+1)) { - throw new RuntimeException("Offset must be over the current row number"); - } - - queryResultScanner.getNextRows(offset - currentRow - 1); - } - private String getEncodedBase64DigestString(List outputList) throws NoSuchAlgorithmException { MessageDigest messageDigest = MessageDigest.getInstance("SHA-1"); diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java index d4f17855f2..b111f6daa9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java @@ -63,6 +63,9 @@ public class TestQueryResultResource extends QueryTestCaseBase { private static final String tajoSessionIdHeaderName = "X-Tajo-Session"; private static final String tajoDigestHeaderName = "X-Tajo-Digest"; + private static final String tajoOffsetHeaderName = "X-Tajo-Offset"; + private static final String tajoCountHeaderName = "X-Tajo-Count"; + private static final String tajoEOSHeaderName = "X-Tajo-EOS"; public TestQueryResultResource() { super(TajoConstants.DEFAULT_DATABASE_NAME); @@ -219,69 +222,6 @@ public void testGetQueryResultSet() throws Exception { } } - @Test - public void testGetQueryResultSetWithOffset() throws Exception { - String sessionId = generateNewSessionAndGetId(); - URI queryIdURI = sendNewQueryResquest(sessionId, "select * from lineitem"); - URI queryResultURI = new URI(queryIdURI + "/result"); - - GetQueryResultDataResponse response = restClient.target(queryResultURI) - .request().header(tajoSessionIdHeaderName, sessionId) - .get(new GenericType(GetQueryResultDataResponse.class)); - - assertNotNull(response); - assertNotNull(response.getResultCode()); - assertEquals(ResultCode.OK, response.getResultCode()); - assertNotNull(response.getSchema()); - assertEquals(16, response.getSchema().getRootColumns().size()); - assertNotNull(response.getResultset()); - assertTrue(response.getResultset().getId() != 0); - assertNotNull(response.getResultset().getLink()); - - URI queryResultSetURI = response.getResultset().getLink(); - - Response queryResultSetResponse = restClient.target(queryResultSetURI) - .queryParam("count", 100) - .queryParam("offset", 3) - .request().header(tajoSessionIdHeaderName, sessionId) - .get(); - - assertNotNull(queryResultSetResponse); - String tajoDigest = queryResultSetResponse.getHeaderString(tajoDigestHeaderName); - assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); - - DataInputStream queryResultSetInputStream = - new DataInputStream(new BufferedInputStream(queryResultSetResponse.readEntity(InputStream.class))); - - assertNotNull(queryResultSetInputStream); - - boolean isFinished = false; - List tupleList = TUtil.newList(); - RowStoreUtil.RowStoreDecoder decoder = RowStoreUtil.createDecoder(response.getSchema()); - MessageDigest messageDigest = MessageDigest.getInstance("SHA-1"); - while (!isFinished) { - try { - int length = queryResultSetInputStream.readInt(); - byte[] dataByteArray = new byte[length]; - int readBytes = queryResultSetInputStream.read(dataByteArray); - - assertEquals(length, readBytes); - - tupleList.add(decoder.toTuple(dataByteArray)); - messageDigest.update(dataByteArray); - } catch (EOFException eof) { - isFinished = true; - } - } - - assertEquals(3, tupleList.size()); - assertEquals(tajoDigest, Base64.encodeBase64String(messageDigest.digest())); - - for (Tuple aTuple: tupleList) { - assertTrue(aTuple.getInt4(response.getSchema().getColumnId("l_orderkey")) > 0); - } - } - @Test public void testGetQueryResultSetWithDefaultCount() throws Exception { String sessionId = generateNewSessionAndGetId(); @@ -309,7 +249,14 @@ public void testGetQueryResultSetWithDefaultCount() throws Exception { assertNotNull(queryResultSetResponse); String tajoDigest = queryResultSetResponse.getHeaderString(tajoDigestHeaderName); - assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); + int offset = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoOffsetHeaderName)); + int count = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoCountHeaderName)); + boolean eos = Boolean.valueOf(queryResultSetResponse.getHeaderString(tajoEOSHeaderName)); + + assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); + assertTrue(eos); + assertEquals(0, offset); + assertEquals(5, count); DataInputStream queryResultSetInputStream = new DataInputStream(new BufferedInputStream(queryResultSetResponse.readEntity(InputStream.class))); From 850fd47717917bc21f477f0cec85bc1c0317c8e3 Mon Sep 17 00:00:00 2001 From: "clark.kang" Date: Tue, 16 Jun 2015 16:08:39 +0900 Subject: [PATCH 7/7] change indent --- .../ws/rs/resources/TestQueryResultResource.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java index b111f6daa9..fec162688c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java @@ -249,14 +249,14 @@ public void testGetQueryResultSetWithDefaultCount() throws Exception { assertNotNull(queryResultSetResponse); String tajoDigest = queryResultSetResponse.getHeaderString(tajoDigestHeaderName); - int offset = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoOffsetHeaderName)); - int count = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoCountHeaderName)); - boolean eos = Boolean.valueOf(queryResultSetResponse.getHeaderString(tajoEOSHeaderName)); - - assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); - assertTrue(eos); - assertEquals(0, offset); - assertEquals(5, count); + int offset = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoOffsetHeaderName)); + int count = Integer.valueOf(queryResultSetResponse.getHeaderString(tajoCountHeaderName)); + boolean eos = Boolean.valueOf(queryResultSetResponse.getHeaderString(tajoEOSHeaderName)); + + assertTrue(tajoDigest != null && !tajoDigest.isEmpty()); + assertTrue(eos); + assertEquals(0, offset); + assertEquals(5, count); DataInputStream queryResultSetInputStream = new DataInputStream(new BufferedInputStream(queryResultSetResponse.readEntity(InputStream.class)));