public class ExecutionJobVertex extends Object implements AccessExecutionJobVertex, Archiveable<ArchivedExecutionJobVertex>
ExecutionJobVertex
is part of the ExecutionGraph
, and the peer to the JobVertex
.
The ExecutionJobVertex
corresponds to a parallelized operation. It contains an ExecutionVertex
for each parallel instance of that operation.
Modifier and Type | Field and Description |
---|---|
static int |
VALUE_NOT_SET |
Constructor and Description |
---|
ExecutionJobVertex(ExecutionGraph graph,
JobVertex jobVertex,
int defaultParallelism,
int maxPriorAttemptsHistoryLength,
Time timeout,
long initialGlobalModVersion,
long createTimestamp) |
public static final int VALUE_NOT_SET
public ExecutionJobVertex(ExecutionGraph graph, JobVertex jobVertex, int defaultParallelism, int maxPriorAttemptsHistoryLength, Time timeout, long initialGlobalModVersion, long createTimestamp) throws JobException
JobException
public List<OperatorIDPair> getOperatorIDs()
public void setMaxParallelism(int maxParallelismDerived)
public ExecutionGraph getGraph()
public JobVertex getJobVertex()
public String getName()
AccessExecutionJobVertex
getName
in interface AccessExecutionJobVertex
public int getParallelism()
AccessExecutionJobVertex
getParallelism
in interface AccessExecutionJobVertex
public int getMaxParallelism()
AccessExecutionJobVertex
getMaxParallelism
in interface AccessExecutionJobVertex
public ResourceProfile getResourceProfile()
AccessExecutionJobVertex
getResourceProfile
in interface AccessExecutionJobVertex
public boolean isMaxParallelismConfigured()
public JobID getJobId()
public JobVertexID getJobVertexId()
AccessExecutionJobVertex
JobVertexID
for this job vertex.getJobVertexId
in interface AccessExecutionJobVertex
public ExecutionVertex[] getTaskVertices()
AccessExecutionJobVertex
getTaskVertices
in interface AccessExecutionJobVertex
public IntermediateResult[] getProducedDataSets()
public InputSplitAssigner getSplitAssigner()
@Nullable public SlotSharingGroup getSlotSharingGroup()
public CoLocationGroup getCoLocationGroup()
public List<IntermediateResult> getInputs()
public InputDependencyConstraint getInputDependencyConstraint()
public Collection<OperatorCoordinatorHolder> getOperatorCoordinators()
public Either<SerializedValue<TaskInformation>,PermanentBlobKey> getTaskInformationOrBlobKey() throws IOException
IOException
public ExecutionState getAggregateState()
AccessExecutionJobVertex
ExecutionState
for this job vertex.getAggregateState
in interface AccessExecutionJobVertex
public void connectToPredecessors(Map<IntermediateDataSetID,IntermediateResult> intermediateDataSets) throws JobException
JobException
public void cancel()
public CompletableFuture<Void> cancelWithFuture()
public CompletableFuture<Void> suspend()
public void fail(Throwable t)
public void resetForNewExecution(long timestamp, long expectedGlobalModVersion) throws GlobalModVersionMismatch
GlobalModVersionMismatch
public StringifiedAccumulatorResult[] getAggregatedUserAccumulatorsStringified()
AccessExecutionJobVertex
getAggregatedUserAccumulatorsStringified
in interface AccessExecutionJobVertex
public ArchivedExecutionJobVertex archive()
archive
in interface Archiveable<ArchivedExecutionJobVertex>
public static ExecutionState getAggregateJobVertexState(int[] verticesPerState, int parallelism)
This state is not used anywhere in the coordination, but can be used for display in dashboards to as a summary for how the particular parallel operation represented by this ExecutionJobVertex is currently behaving.
For example, if at least one parallel task is failed, the aggregate state is failed. If not, and at least one parallel task is cancelling (or cancelled), the aggregate state is cancelling (or cancelled). If all tasks are finished, the aggregate state is finished, and so on.
verticesPerState
- The number of vertices in each state (indexed by the ordinal of the
ExecutionState values).parallelism
- The parallelism of the ExecutionJobVertexCopyright © 2014–2021 The Apache Software Foundation. All rights reserved.