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.getSelf(); nodes.add(uri.getHost() + ":" + uri.getPort()); } for (StageInfo subStage : stageInfo.getSubStages()) { nodes.addAll(globalUniqueNodes(subStage)); } return nodes.build(); }
.map(task -> task.getTaskInfo().getSelf()) .collect(toImmutableSet()); parent.get().addExchangeLocations(currentStageFragmentId, newExchangeLocations, noMoreTasks);
/** * 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())); }
private synchronized void updateExchangeClient(StageInfo outputStage) { // add any additional output locations if (!outputStage.getState().isDone()) { for (TaskInfo taskInfo : outputStage.getTasks()) { SharedBufferInfo outputBuffers = taskInfo.getOutputBuffers(); List<BufferInfo> buffers = outputBuffers.getBuffers(); if (buffers.isEmpty() || outputBuffers.getState().canAddBuffers()) { // output buffer has not been created yet continue; } Preconditions.checkState(buffers.size() == 1, "Expected a single output buffer for task %s, but found %s", taskInfo.getTaskId(), buffers); TaskId bufferId = Iterables.getOnlyElement(buffers).getBufferId(); URI uri = uriBuilderFrom(taskInfo.getSelf()).appendPath("results").appendPath(bufferId.toString()).build(); exchangeClient.addLocation(uri); } } if (allOutputBuffersCreated(outputStage)) { exchangeClient.noMoreLocations(); } }
.setUri(uriBuilderFrom(taskInfo.get().getSelf()).addParameter("summarize").build()) .setHeader(HttpHeaders.CONTENT_TYPE, MediaType.JSON_UTF_8.toString()) .setBodyGenerator(jsonBodyGenerator(taskUpdateRequestCodec, updateRequest))
String failureHost = task.map(x -> x.getSelf().getHost()).orElse(null); String failureTask = task.map(x -> x.getTaskId().toString()).orElse(null);
task.getSelf().getHost(), last - task.getStats().getCreateTime().getMillis(), task.getStats().getTotalCpuTime().roundTo(TimeUnit.MILLISECONDS),
for (TaskInfo task : stageInfo.getTasks()) { URI uri = task.getSelf(); uniqueNodes.add(uri.getHost() + ":" + uri.getPort());
@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"); } }
URI uri = taskInfo.getSelf();
private synchronized void scheduleNextRequest() { // stopped or done? TaskInfo taskInfo = HttpRemoteTask.this.taskInfo.get(); if (!running || taskInfo.getState().isDone()) { return; } // outstanding request? if (future != null && !future.isDone()) { // this should never happen log.error("Can not reschedule update because an update is already running"); return; } // if throttled due to error, asynchronously wait for timeout and try again ListenableFuture<?> errorRateLimit = getErrorTracker.acquireRequestPermit(); if (!errorRateLimit.isDone()) { errorRateLimit.addListener(this::scheduleNextRequest, executor); return; } Request request = prepareGet() .setUri(uriBuilderFrom(taskInfo.getSelf()).addParameter("summarize").build()) .setHeader(HttpHeaders.CONTENT_TYPE, MediaType.JSON_UTF_8.toString()) .setHeader(PrestoHeaders.PRESTO_CURRENT_STATE, taskInfo.getState().toString()) .setHeader(PrestoHeaders.PRESTO_MAX_WAIT, refreshMaxWait.toString()) .build(); getErrorTracker.startRequest(); future = httpClient.executeAsync(request, createFullJsonResponseHandler(taskInfoCodec)); Futures.addCallback(future, new SimpleHttpResponseHandler<>(this, request.getUri()), executor); }