private StageInfo buildStageInfo(StageId stageId, Map<StageId, StageInfo> stageInfos) { StageInfo parent = stageInfos.get(stageId); checkArgument(parent != null, "No stageInfo for %s", parent); List<StageInfo> childStages = stageLinkages.get(stageId).getChildStageIds().stream() .map(childStageId -> buildStageInfo(childStageId, stageInfos)) .collect(toImmutableList()); if (childStages.isEmpty()) { return parent; } return new StageInfo( parent.getStageId(), parent.getState(), parent.getSelf(), parent.getPlan(), parent.getTypes(), parent.getStageStats(), parent.getTasks(), childStages, parent.getFailureCause()); }
private static void assertState(StageStateMachine stateMachine, StageState expectedState) { assertEquals(stateMachine.getStageId(), STAGE_ID); assertEquals(stateMachine.getLocation(), LOCATION); assertSame(stateMachine.getSession(), TEST_SESSION); StageInfo stageInfo = stateMachine.getStageInfo(ImmutableList::of); assertEquals(stageInfo.getStageId(), STAGE_ID); assertEquals(stageInfo.getSelf(), LOCATION); assertEquals(stageInfo.getSubStages(), ImmutableList.of()); assertEquals(stageInfo.getTasks(), ImmutableList.of()); assertEquals(stageInfo.getTypes(), ImmutableList.of(VARCHAR)); assertSame(stageInfo.getPlan(), PLAN_FRAGMENT); assertEquals(stateMachine.getState(), expectedState); assertEquals(stageInfo.getState(), expectedState); if (expectedState == StageState.FAILED) { ExecutionFailureInfo failure = stageInfo.getFailureCause(); assertEquals(failure.getMessage(), FAILED_CAUSE.getMessage()); assertEquals(failure.getType(), FAILED_CAUSE.getClass().getName()); } else { assertNull(stageInfo.getFailureCause()); } }
List<StageInfo> stages = StageInfo.getAllStages(queryInfo.getOutputStage()); for (StageInfo stage : stages) { if (!stage.getSubStages().isEmpty()) { continue; for (TaskInfo taskInfo : stage.getTasks()) { TaskStats taskStats = taskInfo.getStats();
private static void addAllStages(Optional<StageInfo> stageInfo, ImmutableList.Builder<StageInfo> collector) { stageInfo.ifPresent(stage -> { collector.add(stage); stage.getSubStages().stream() .forEach(subStage -> addAllStages(Optional.ofNullable(subStage), collector)); }); }
private static URI findCancelableLeafStage(StageInfo stage) { // if this stage is already done, we can't cancel it if (stage.getState().isDone()) { return null; } // attempt to find a cancelable sub stage // check in reverse order since build side of a join will be later in the list for (StageInfo subStage : Lists.reverse(stage.getSubStages())) { URI leafStage = findCancelableLeafStage(subStage); if (leafStage != null) { return leafStage; } } // no matching sub stage, so return this stage return stage.getSelf(); }
Optional<StageInfo> prunedOutputStage = queryInfo.getOutputStage().map(outputStage -> new StageInfo( outputStage.getStageId(), outputStage.getState(), outputStage.getSelf(), outputStage.getTypes(), outputStage.getStageStats(), outputStage.getFailureCause()));
com.facebook.presto.execution.StageStats stageStats = stageInfo.getStageStats(); for (StageInfo subStage : stageInfo.getSubStages()) { subStages.add(toStageStats(subStage)); for (TaskInfo task : stageInfo.getTasks()) { .setStageId(String.valueOf(stageInfo.getStageId().getId())) .setState(stageInfo.getState().toString()) .setDone(stageInfo.getState().isDone()) .setNodes(uniqueNodes.size()) .setTotalSplits(stageStats.getTotalDrivers())
for (StageInfo stageInfo : getAllStages(rootStage)) { StageStats stageStats = stageInfo.getStageStats(); totalTasks += stageStats.getTotalTasks(); runningTasks += stageStats.getRunningTasks(); totalCpuTime += stageStats.getTotalCpuTime().roundTo(MILLISECONDS); totalBlockedTime += stageStats.getTotalBlockedTime().roundTo(MILLISECONDS); if (!stageInfo.getState().isDone()) { fullyBlocked &= stageStats.isFullyBlocked(); blockedReasons.addAll(stageStats.getBlockedReasons()); PlanFragment plan = stageInfo.getPlan(); if (plan != null && plan.getPartitionedSourceNodes().stream().anyMatch(TableScanNode.class::isInstance)) { rawInputDataSize += stageStats.getRawInputDataSize().toBytes(); completeInfo = completeInfo && stageInfo.isCompleteInfo(); operatorStatsSummary.addAll(stageInfo.getStageStats().getOperatorSummaries()); StageStats outputStageStats = rootStage.get().getStageStats(); outputDataSize += outputStageStats.getOutputDataSize().toBytes(); outputPositions += outputStageStats.getOutputPositions();
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 allOutputBuffersCreated(StageInfo outputStage) { StageState stageState = outputStage.getState(); // if the stage is already done, then there will be no more buffers if (stageState.isDone()) { return true; } // have all stage tasks been scheduled? if (stageState == StageState.PLANNED || stageState == StageState.SCHEDULING) { return false; } // have all tasks finished adding buffers return outputStage.getTasks().stream() .allMatch(taskInfo -> !taskInfo.getOutputBuffers().getState().canAddBuffers()); }
private static boolean atLeastOneOperatorWaitingForMemory(QueryInfo query) { return getAllStages(query.getOutputStage()).stream() .flatMap(stage -> stage.getTasks().stream()) .map(TaskInfo::getStats) .anyMatch(task -> task.getBlockedReasons().contains(WAITING_FOR_MEMORY)); }
private static void getSubStages(StageId stageId, StageInfo rootStage, ImmutableList.Builder<StageInfo> collector, boolean add) { if (rootStage.getStageId().equals(stageId)) { add = true; } List<StageInfo> subStages = rootStage.getSubStages(); for (StageInfo subStage : subStages) { getSubStages(stageId, subStage, collector, add); } if (add && !rootStage.getStageId().equals(stageId)) { collector.add(rootStage); } } }
private static StageCpuDistribution computeCpuDistribution(StageInfo stageInfo) { Distribution cpuDistribution = new Distribution(); for (TaskInfo taskInfo : stageInfo.getTasks()) { cpuDistribution.add(taskInfo.getStats().getTotalCpuTime().toMillis()); } DistributionSnapshot snapshot = cpuDistribution.snapshot(); return new StageCpuDistribution( stageInfo.getStageId().getId(), stageInfo.getTasks().size(), snapshot.getP25(), snapshot.getP50(), snapshot.getP75(), snapshot.getP90(), snapshot.getP95(), snapshot.getP99(), snapshot.getMin(), snapshot.getMax(), (long) snapshot.getTotal(), snapshot.getTotal() / snapshot.getCount()); } }
public static String textDistributedPlan(StageInfo outputStageInfo, FunctionRegistry functionRegistry, Session session, boolean verbose) { StringBuilder builder = new StringBuilder(); List<StageInfo> allStages = getAllStages(Optional.of(outputStageInfo)); List<PlanFragment> allFragments = allStages.stream() .map(StageInfo::getPlan) .collect(toImmutableList()); Map<PlanNodeId, PlanNodeStats> aggregatedStats = aggregatePlanNodeStats(allStages); for (StageInfo stageInfo : allStages) { builder.append(formatFragment(functionRegistry, session, stageInfo.getPlan(), Optional.of(stageInfo), Optional.of(aggregatedStats), verbose, allFragments)); } return builder.toString(); }
StageStats stageStats = stageInfo.get().getStageStats(); double avgPositionsPerTask = stageInfo.get().getTasks().stream().mapToLong(task -> task.getStats().getProcessedInputPositions()).average().orElse(Double.NaN); double squaredDifferences = stageInfo.get().getTasks().stream().mapToDouble(task -> Math.pow(task.getStats().getProcessedInputPositions() - avgPositionsPerTask, 2)).sum(); double sdAmongTasks = Math.sqrt(squaredDifferences / stageInfo.get().getTasks().size());
public static Map<PlanNodeId, PlanNodeStats> aggregatePlanNodeStats(List<StageInfo> stageInfos) { Map<PlanNodeId, PlanNodeStats> aggregatedStats = new HashMap<>(); List<PlanNodeStats> planNodeStats = stageInfos.stream() .flatMap(stageInfo -> stageInfo.getTasks().stream()) .map(TaskInfo::getStats) .flatMap(taskStats -> getPlanNodeStats(taskStats).stream()) .collect(toList()); for (PlanNodeStats stats : planNodeStats) { aggregatedStats.merge(stats.getPlanNodeId(), stats, (left, right) -> left.mergeWith(right)); } return aggregatedStats; }
private static void populateDistribution(StageInfo stageInfo, ImmutableList.Builder<StageCpuDistribution> distributions) { distributions.add(computeCpuDistribution(stageInfo)); for (StageInfo subStage : stageInfo.getSubStages()) { populateDistribution(subStage, distributions); } }
if (finalStageInfo.isPresent()) { return finalStageInfo.get() .getStageStats() .toBasicStageStats(finalStageInfo.get().getState());
@JsonProperty public boolean isFinalQueryInfo() { return state.isDone() && getAllStages(outputStage).stream().allMatch(StageInfo::isFinalStageInfo); }
queryStateMachine.transitionToFailed(stage.getStageInfo().getFailureCause().toException());