private List<TaskInfo> getActiveTasks() { return sqlTaskManager.getAllTaskInfo() .stream() .filter(taskInfo -> !taskInfo.getState().isDone()) .collect(ImmutableCollectors.toImmutableList()); }
private void addTask(RemoteTask task) { if (remoteTasks.add(task)) { task.addStateChangeListener(taskInfo -> { if (taskInfo.getState().isDone()) { remoteTasks.remove(task); } }); // Check if task state is already done before adding the listener if (task.getTaskInfo().getState().isDone()) { remoteTasks.remove(task); } } }
private static Optional<TaskInfo> findFailedTask(StageInfo stageInfo) { if (stageInfo == null) { return Optional.empty(); } for (StageInfo subStage : stageInfo.getSubStages()) { Optional<TaskInfo> task = findFailedTask(subStage); if (task.isPresent()) { return task; } } return stageInfo.getTasks().stream() .filter(taskInfo -> taskInfo.getState() == TaskState.FAILED) .findFirst(); }
@Override public synchronized void setOutputBuffers(OutputBuffers newOutputBuffers) { if (getTaskInfo().getState().isDone()) { return; } if (newOutputBuffers.getVersion() > outputBuffers.get().getVersion()) { outputBuffers.set(newOutputBuffers); needsUpdate.set(true); scheduleUpdate(); } }
private void updateStats() { SqlTaskIoStats tempIoStats = new SqlTaskIoStats(); tempIoStats.merge(finishedTaskStats); // there is a race here between task completion, which merges stats into // finishedTaskStats, and getting the stats from the task. Since we have // already merged the final stats, we could miss the stats from this task // which would result in an under-count, but we will not get an over-count. tasks.asMap().values().stream() .filter(task -> !task.getTaskInfo().getState().isDone()) .forEach(task -> tempIoStats.merge(task.getIoStats())); cachedStats.resetTo(tempIoStats); }
@Override public CompletableFuture<TaskInfo> getStateChange(TaskInfo taskInfo) { return taskStateMachine.getStateChange(taskInfo.getState()).thenApply(ignored -> getTaskInfo()); }
@Override public int getQueuedPartitionedSplitCount() { TaskInfo taskInfo = this.taskInfo.get(); if (taskInfo.getState().isDone()) { return 0; } return getPendingSourceSplitCount() + taskInfo.getStats().getQueuedPartitionedDrivers(); }
@Test public void testAbort() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(SPLIT), true)), INITIAL_EMPTY_OUTPUT_BUFFERS.withBuffer(OUT, 0).withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); sqlTask.abortTaskResults(OUT); taskInfo = sqlTask.getTaskInfo(taskInfo.getState()).get(1, SECONDS); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.FINISHED); }
@Test public void testEmptyQuery() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), ImmutableList.<TaskSource>of(), INITIAL_EMPTY_OUTPUT_BUFFERS); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.<ScheduledSplit>of(), true)), INITIAL_EMPTY_OUTPUT_BUFFERS.withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.FINISHED); }
@Test public void testEmptyQuery() throws Exception { try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) { TaskId taskId = TASK_ID; TaskInfo taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), ImmutableList.<TaskSource>of(), INITIAL_EMPTY_OUTPUT_BUFFERS); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.<ScheduledSplit>of(), true)), INITIAL_EMPTY_OUTPUT_BUFFERS.withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getState(), TaskState.FINISHED); } }
@Test public void testAbortResults() throws Exception { try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) { TaskId taskId = TASK_ID; TaskInfo taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(SPLIT), true)), INITIAL_EMPTY_OUTPUT_BUFFERS.withBuffer(OUT, 0).withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getState(), TaskState.RUNNING); sqlTaskManager.abortTaskResults(taskId, OUT); taskInfo = sqlTaskManager.getTaskInfo(taskId, taskInfo.getState()).get(1, TimeUnit.SECONDS); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getState(), TaskState.FINISHED); } }
@Test public void testRemoveOldTasks() throws Exception { try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig().setInfoMaxAge(new Duration(5, TimeUnit.MILLISECONDS)))) { TaskId taskId = TASK_ID; TaskInfo taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), ImmutableList.<TaskSource>of(), INITIAL_EMPTY_OUTPUT_BUFFERS); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTaskManager.cancelTask(taskId); assertEquals(taskInfo.getState(), TaskState.CANCELED); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getState(), TaskState.CANCELED); Thread.sleep(100); sqlTaskManager.removeOldTasks(); for (TaskInfo info : sqlTaskManager.getAllTaskInfo()) { assertNotEquals(info.getTaskId(), taskId); } } }
public void failAbandonedTasks() { DateTime now = DateTime.now(); DateTime oldestAllowedHeartbeat = now.minus(clientTimeout.toMillis()); for (SqlTask sqlTask : tasks.asMap().values()) { try { TaskInfo taskInfo = sqlTask.getTaskInfo(); if (taskInfo.getState().isDone()) { continue; } DateTime lastHeartbeat = taskInfo.getLastHeartbeat(); if (lastHeartbeat != null && lastHeartbeat.isBefore(oldestAllowedHeartbeat)) { log.info("Failing abandoned task %s", taskInfo.getTaskId()); sqlTask.failed(new AbandonedException("Task " + taskInfo.getTaskId(), lastHeartbeat, now)); } } catch (RuntimeException e) { log.warn(e, "Error while inspecting age of task %s", sqlTask.getTaskId()); } } }
@Override public int getPartitionedSplitCount() { TaskInfo taskInfo = this.taskInfo.get(); if (taskInfo.getState().isDone()) { return 0; } return getPendingSourceSplitCount() + taskInfo.getStats().getQueuedPartitionedDrivers() + taskInfo.getStats().getRunningPartitionedDrivers(); }
@Test public void testCancel() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), ImmutableList.<TaskSource>of(), INITIAL_EMPTY_OUTPUT_BUFFERS); assertEquals(taskInfo.getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTask.cancel(); assertEquals(taskInfo.getState(), TaskState.CANCELED); assertNotNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.CANCELED); assertNotNull(taskInfo.getStats().getEndTime()); }
@Test public void testBufferNotCloseOnFail() throws Exception { SqlTask sqlTask = createInitialTask(); updateTask(sqlTask, EMPTY_SOURCES, INITIAL_EMPTY_OUTPUT_BUFFERS.withBuffer(OUT, 0)); CompletableFuture<BufferResult> bufferResult = sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)); assertFalse(bufferResult.isDone()); TaskState taskState = sqlTask.getTaskInfo().getState(); sqlTask.failed(new Exception("test")); assertEquals(sqlTask.getTaskInfo(taskState).get(1, SECONDS).getState(), TaskState.FAILED); // buffer will not be closed by fail event. event is async so wait a bit for event to fire try { assertTrue(bufferResult.get(1, SECONDS).isBufferComplete()); fail("expected TimeoutException"); } catch (TimeoutException expected) { // expected } assertFalse(sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)).isDone()); }
@Test public void testSimpleQuery() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(SPLIT), true)), INITIAL_EMPTY_OUTPUT_BUFFERS.withBuffer(OUT, 0).withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); BufferResult results = sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)).get(); assertEquals(results.isBufferComplete(), false); assertEquals(results.getPages().size(), 1); assertEquals(results.getPages().get(0).getPositionCount(), 1); results = sqlTask.getTaskResults(OUT, results.getToken() + results.getPages().size(), new DataSize(1, MEGABYTE)).get(); assertEquals(results.isBufferComplete(), true); assertEquals(results.getPages().size(), 0); // complete the task by calling abort on it TaskInfo info = sqlTask.abortTaskResults(OUT); assertEquals(info.getOutputBuffers().getState(), BufferState.FINISHED); taskInfo = sqlTask.getTaskInfo(taskInfo.getState()).get(1, SECONDS); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.FINISHED); }
@Test public void testBufferCloseOnCancel() throws Exception { SqlTask sqlTask = createInitialTask(); updateTask(sqlTask, EMPTY_SOURCES, INITIAL_EMPTY_OUTPUT_BUFFERS.withBuffer(OUT, 0)); CompletableFuture<BufferResult> bufferResult = sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)); assertFalse(bufferResult.isDone()); sqlTask.cancel(); assertEquals(sqlTask.getTaskInfo().getState(), TaskState.CANCELED); // buffer will be closed by cancel event.. the event is async so wait a bit for event to propagate assertTrue(bufferResult.get(1, SECONDS).isBufferComplete()); bufferResult = sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)); assertTrue(bufferResult.isDone()); assertTrue(bufferResult.get().isBufferComplete()); }
/** * Move the task directly to the failed state */ private void failTask(Throwable cause) { TaskInfo taskInfo = getTaskInfo(); if (!taskInfo.getState().isDone()) { log.debug(cause, "Remote task failed: %s", taskInfo.getSelf()); } updateTaskInfo(new TaskInfo(taskInfo.getTaskId(), taskInfo.getTaskInstanceId(), TaskInfo.MAX_VERSION, TaskState.FAILED, taskInfo.getSelf(), taskInfo.getLastHeartbeat(), taskInfo.getOutputBuffers(), taskInfo.getNoMoreSplits(), taskInfo.getStats(), ImmutableList.of(toFailure(cause)), taskInfo.isNeedsPlan())); }
@Override public synchronized void cancel() { try (SetThreadName ignored = new SetThreadName("HttpRemoteTask-%s", taskId)) { if (getTaskInfo().getState().isDone()) { return; } checkState(continuousTaskInfoFetcher.isRunning(), "Cannot cancel task when it is not running"); URI uri = getTaskInfo().getSelf(); if (uri == null) { return; } // send cancel to task and ignore response Request request = prepareDelete() .setUri(uriBuilderFrom(uri).addParameter("abort", "false").addParameter("summarize").build()) .build(); scheduleAsyncCleanupRequest(new Backoff(MAX_CLEANUP_RETRY_TIME), request, "cancel"); } }