private synchronized void updateFinalTaskInfo(TaskInfo finalTaskInfo) { tasksWithFinalInfo.add(finalTaskInfo.getTaskStatus().getTaskId()); checkAllTaskFinal(); }
private void updateTaskInfo(TaskInfo taskInfo) { taskStatusFetcher.updateTaskStatus(taskInfo.getTaskStatus()); taskInfoFetcher.updateTaskInfo(taskInfo); }
private static Set<String> globalUniqueNodes(StageInfo stageInfo) { if (stageInfo == null) { return ImmutableSet.of(); } ImmutableSet.Builder<String> nodes = ImmutableSet.builder(); for (TaskInfo task : stageInfo.getTasks()) { // todo add nodeId to TaskInfo URI uri = task.getTaskStatus().getSelf(); nodes.add(uri.getHost() + ":" + uri.getPort()); } for (StageInfo subStage : stageInfo.getSubStages()) { nodes.addAll(globalUniqueNodes(subStage)); } return nodes.build(); }
public boolean isCompleteInfo() { return state.isDone() && tasks.stream().allMatch(taskInfo -> taskInfo.getTaskStatus().getState().isDone()); } }
private static Optional<TaskInfo> findFailedTask(StageInfo stageInfo) { for (StageInfo subStage : stageInfo.getSubStages()) { Optional<TaskInfo> task = findFailedTask(subStage); if (task.isPresent()) { return task; } } return stageInfo.getTasks().stream() .filter(taskInfo -> taskInfo.getTaskStatus().getState() == TaskState.FAILED) .findFirst(); }
private static boolean isDone(TaskInfo taskInfo) { return taskInfo.getTaskStatus().getState().isDone(); } }
private List<TaskInfo> getActiveTasks() { return sqlTaskManager.getAllTaskInfo() .stream() .filter(taskInfo -> !taskInfo.getTaskStatus().getState().isDone()) .collect(toImmutableList()); }
public boolean isFinalStageInfo() { return state.isDone() && tasks.stream().allMatch(taskInfo -> taskInfo.getTaskStatus().getState().isDone()); }
private Optional<QueryFailureInfo> createQueryFailureInfo(ExecutionFailureInfo failureInfo, Optional<StageInfo> outputStage) { if (failureInfo == null) { return Optional.empty(); } Optional<TaskInfo> failedTask = outputStage.flatMap(QueryMonitor::findFailedTask); return Optional.of(new QueryFailureInfo( failureInfo.getErrorCode(), Optional.ofNullable(failureInfo.getType()), Optional.ofNullable(failureInfo.getMessage()), failedTask.map(task -> task.getTaskStatus().getTaskId().toString()), failedTask.map(task -> task.getTaskStatus().getSelf().getHost()), executionFailureInfoCodec.toJson(failureInfo))); }
public void removeOldTasks() { DateTime oldestAllowedTask = DateTime.now().minus(infoCacheTime.toMillis()); for (TaskInfo taskInfo : filter(transform(tasks.asMap().values(), SqlTask::getTaskInfo), notNull())) { TaskId taskId = taskInfo.getTaskStatus().getTaskId(); try { DateTime endTime = taskInfo.getStats().getEndTime(); if (endTime != null && endTime.isBefore(oldestAllowedTask)) { tasks.asMap().remove(taskId); } } catch (RuntimeException e) { log.warn(e, "Error while inspecting age of complete task %s", taskId); } } }
synchronized void updateTaskInfo(TaskInfo newValue) { boolean updated = taskInfo.setIf(newValue, oldValue -> { TaskStatus oldTaskStatus = oldValue.getTaskStatus(); TaskStatus newTaskStatus = newValue.getTaskStatus(); if (oldTaskStatus.getState().isDone()) { // never update if the task has reached a terminal state return false; } // don't update to an older version (same version is ok) return newTaskStatus.getVersion() >= oldTaskStatus.getVersion(); }); if (updated && newValue.getTaskStatus().getState().isDone()) { finalTaskInfo.compareAndSet(Optional.empty(), Optional.of(newValue)); stop(); } }
@Override public void onSuccess(JsonResponse<TaskInfo> result) { try { updateTaskInfo(result.getValue()); } finally { if (!getTaskInfo().getTaskStatus().getState().isDone()) { cleanUpLocally(); } } }
@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)), createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds(), OptionalInt.empty()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); sqlTask.abortTaskResults(OUT); taskInfo = sqlTask.getTaskInfo(taskInfo.getTaskStatus().getState()).get(1, SECONDS); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED); }
@Test public void testAbortResults() throws Exception { try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) { TaskId taskId = TASK_ID; TaskInfo taskInfo = createTask(sqlTaskManager, taskId, ImmutableSet.of(SPLIT), createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); sqlTaskManager.abortTaskResults(taskId, OUT); taskInfo = sqlTaskManager.getTaskInfo(taskId, taskInfo.getTaskStatus().getState()).get(1, TimeUnit.SECONDS); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED); } }
@Test public void testEmptyQuery() { try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) { TaskId taskId = TASK_ID; TaskInfo taskInfo = createTask(sqlTaskManager, taskId, createInitialEmptyOutputBuffers(PARTITIONED).withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); taskInfo = createTask(sqlTaskManager, taskId, ImmutableSet.of(), createInitialEmptyOutputBuffers(PARTITIONED).withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().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 = createTask(sqlTaskManager, taskId, createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); taskInfo = sqlTaskManager.cancelTask(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.CANCELED); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.CANCELED); Thread.sleep(100); sqlTaskManager.removeOldTasks(); for (TaskInfo info : sqlTaskManager.getAllTaskInfo()) { assertNotEquals(info.getTaskStatus().getTaskId(), taskId); } } }
@Test public void testCancel() { try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) { TaskId taskId = TASK_ID; TaskInfo taskInfo = createTask(sqlTaskManager, taskId, createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTaskManager.cancelTask(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.CANCELED); assertNotNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.CANCELED); assertNotNull(taskInfo.getStats().getEndTime()); } }
@Test public void testAbort() { try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) { TaskId taskId = TASK_ID; TaskInfo taskInfo = createTask(sqlTaskManager, taskId, createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTaskManager.abortTask(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.ABORTED); assertNotNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTaskManager.getTaskInfo(taskId); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.ABORTED); assertNotNull(taskInfo.getStats().getEndTime()); } }
@Test public void testBufferNotCloseOnFail() throws Exception { SqlTask sqlTask = createInitialTask(); updateTask(sqlTask, EMPTY_SOURCES, createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds()); ListenableFuture<BufferResult> bufferResult = sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)); assertFalse(bufferResult.isDone()); TaskState taskState = sqlTask.getTaskInfo().getTaskStatus().getState(); sqlTask.failed(new Exception("test")); assertEquals(sqlTask.getTaskInfo(taskState).get(1, SECONDS).getTaskStatus().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 testBufferCloseOnCancel() throws Exception { SqlTask sqlTask = createInitialTask(); updateTask(sqlTask, EMPTY_SOURCES, createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds()); ListenableFuture<BufferResult> bufferResult = sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)); assertFalse(bufferResult.isDone()); sqlTask.cancel(); assertEquals(sqlTask.getTaskInfo().getTaskStatus().getState(), TaskState.CANCELED); // buffer future will complete.. the event is async so wait a bit for event to propagate bufferResult.get(1, SECONDS); bufferResult = sqlTask.getTaskResults(OUT, 0, new DataSize(1, MEGABYTE)); assertTrue(bufferResult.isDone()); assertTrue(bufferResult.get().isBufferComplete()); }