private Collection<ExecutionVertexID> getConsumerExecutionVerticesAllToAll() { Collection<ExecutionVertexID> consumerVertices = new ArrayList<>(); for (int i = 0; i < target.getParallelism(); i++) { consumerVertices.add(new ExecutionVertexID(target.getID(), i)); } return consumerVertices; }
@Override public ExecutionVertexStatus getExecutionVertexStatus(ExecutionVertexID executionVertexID) { checkNotNull(executionVertexID); ExecutionJobVertex vertex = executionGraph.getJobVertex(executionVertexID.getJobVertexID()); if (vertex == null) { throw new IllegalArgumentException("Cannot find any vertex with id " + executionVertexID.getJobVertexID()); } return vertex.getTaskVertices()[executionVertexID.getSubTaskIndex()].getCurrentStatus(); }
public boolean areInputsReady(ExecutionVertexID vertexID) { // It is source vertex if no input info if (!vertexInputsMap.containsKey(vertexID)) { return true; } if (config.getInputDependencyConstraint(vertexID.getJobVertexID()) == InputDependencyConstraint.ALL) { for (VertexInput input : vertexInputsMap.get(vertexID).values()) { if (!input.isConsumable(scheduler)) { return false; } } return true; } else { for (VertexInput input : vertexInputsMap.get(vertexID).values()) { if (input.isConsumable(scheduler)) { return true; } } return false; } }
ExecutionVertex consumerVertex = taskVertices[executionVertexID.getSubTaskIndex()]; ExecutionEdge ee = new ExecutionEdge(partition, consumerVertex, num); partition.addConsumer(ee, consumerIndex); executionEdges.get(executionVertexID.getSubTaskIndex()).add(ee);
@Override public void onExecutionVertexFailover(ExecutionVertexFailoverEvent event) { scheduleQueue.clear(); currentScheduledUnit = null; scheduledRunningUnitSet.clear(); event.getAffectedExecutionVertexIDs().forEach(t -> logicalJobVertices.get(t.getJobVertexID()).failoverTask()); runningUnitMap.values().forEach(LogicalJobVertexRunningUnit::reset); logicalJobVertices.values().stream().filter(LogicalJobVertex::allTasksDeploying).forEach(j -> { allTaskDeploying(j); for (int i = 0; i < j.getParallelism(); i++) { produceResultPartition(j.getJobVertexID()); } }); onSchedulingStarted(); }
@Override public void onSchedulingStarted() { final List<ExecutionVertexID> verticesToSchedule = new ArrayList<>(); for (JobVertex vertex : jobGraph.getVerticesSortedTopologicallyFromSources()) { for (int i = 0; i < vertex.getParallelism(); i++) { verticesToSchedule.add(new ExecutionVertexID(vertex.getID(), i)); } } scheduler.scheduleExecutionVertices(verticesToSchedule); }
final List<ExecutionVertex> vertices = new ArrayList<>(verticesToSchedule.size()); for (ExecutionVertexID executionVertexID : verticesToSchedule) { ExecutionVertex ev = getJobVertex(executionVertexID.getJobVertexID()).getTaskVertices()[executionVertexID.getSubTaskIndex()]; if (ev.getExecutionState() == ExecutionState.CREATED) { vertices.add(ev);
@Override public synchronized void onExecutionVertexStateChanged(ExecutionVertexStateChangedEvent event) { if (event.getNewExecutionState() == ExecutionState.DEPLOYING) { LogicalJobVertex jobVertex = logicalJobVertices.get(event.getExecutionVertexID().getJobVertexID()); jobVertex.deployingTask(); if (jobVertex.allTasksDeploying()) { allTaskDeploying(jobVertex); } checkScheduleNewRunningUnit(); } }
private synchronized void scheduleRunningUnit(LogicalJobVertexRunningUnit jobVertexRunningUnit) { currentScheduledUnit = jobVertexRunningUnit; LOG.info("begin to schedule runningUnit: "); currentScheduledUnit.getJobVertexSet().forEach(x-> LOG.info(x.toString())); jobVertexRunningUnit.getToScheduleJobVertices().forEach(j -> { for (int i = 0; i < j.getParallelism(); i++) { scheduler.scheduleExecutionVertices(Collections.singletonList(new ExecutionVertexID(j.getJobVertexID(), i))); } }); }
JobVertexID jobVertexID = egOperationLog.getExecutionVertexID().getJobVertexID(); int subTaskIndex = egOperationLog.getExecutionVertexID().getSubTaskIndex(); executionGraph.getJobVertex(jobVertexID).getTaskVertices()[subTaskIndex].recoverStatus( egOperationLog.getExecutionState(), ResultPartitionOperationLog rpOperationLog = (ResultPartitionOperationLog) opLog; JobVertexID jobVertexID = rpOperationLog.getExecutionVertexID().getJobVertexID(); int subTaskIndex = rpOperationLog.getExecutionVertexID().getSubTaskIndex(); executionGraph.getJobVertex(jobVertexID).getTaskVertices()[subTaskIndex].recoverResultPartitionStatus( rpOperationLog.getResultID(), List<ExecutionVertex> evs = new ArrayList<>(ids.size()); for (ExecutionVertexID id : ids) { evs.add(executionGraph.getJobVertex(id.getJobVertexID()).getTaskVertices()[id.getSubTaskIndex()]);
private boolean isReadyToSchedule(ExecutionVertexID vertexID) { ExecutionVertexStatus vertexStatus = scheduler.getExecutionVertexStatus(vertexID); // only CREATED vertices can be scheduled if (vertexStatus.getExecutionState() != ExecutionState.CREATED) { return false; } // source vertices can be scheduled at once if (jobGraph.findVertexByID(vertexID.getJobVertexID()).isInputVertex()) { return true; } // query whether the inputs are ready overall return inputTracker.areInputsReady(vertexID); }
@Override public void onSchedulingStarted() { final List<ExecutionVertexID> verticesToSchedule = new ArrayList<>(); for (JobVertex vertex : jobGraph.getVerticesSortedTopologicallyFromSources()) { if (vertex.isInputVertex()) { for (int i = 0; i < vertex.getParallelism(); i++) { verticesToSchedule.add(new ExecutionVertexID(vertex.getID(), i)); } } } scheduleOneByOne(verticesToSchedule); }
vertexID.getJobVertexID(), resultID, jobGraph.getResult(resultID).getResultType().isPipelined());
consumerVertices.add(new ExecutionVertexID(target.getID(), partitionNumber)); } else if (sourceCount > targetCount) { int vertexSubtaskIndex; consumerVertices.add(new ExecutionVertexID(target.getID(), vertexSubtaskIndex)); } else { if (targetCount % sourceCount == 0) { consumerVertices.add(new ExecutionVertexID(target.getID(), startIndex + i)); int mirrorVertexSubTaskIndex = start + i; int vertexSubtaskIndex = targetCount - 1 - mirrorVertexSubTaskIndex; consumerVertices.add(new ExecutionVertexID(target.getID(), vertexSubtaskIndex));
this.executionVertexID = new ExecutionVertexID(jobVertex.getJobVertexId(), subTaskIndex); this.taskNameWithSubtask = String.format("%s (%d/%d)", jobVertex.getJobVertex().getName(), subTaskIndex + 1, jobVertex.getParallelism());
String optionalMessage) { ExecutionVertexID executionVertexID = new ExecutionVertexID(vertexID, subtaskIndex); Map<OperatorID, List<InputSplit>> inputSplits = null; ResultDescriptor resultDescriptor = null;