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,
Time timeout,
long initialGlobalModVersion,
long createTimestamp) |
Modifier and Type | Method and Description |
---|---|
Collection<CompletableFuture<Execution>> |
allocateResourcesForAll(SlotProvider resourceProvider,
boolean queued,
LocationPreferenceConstraint locationPreferenceConstraint,
Set<AllocationID> allPreviousExecutionGraphAllocationIds,
Time allocationTimeout)
Acquires a slot for all the execution vertices of this ExecutionJobVertex.
|
ArchivedExecutionJobVertex |
archive() |
void |
cancel()
Cancels all currently running vertex executions.
|
CompletableFuture<Void> |
cancelWithFuture()
Cancels all currently running vertex executions.
|
void |
connectToPredecessors(Map<IntermediateDataSetID,IntermediateResult> intermediateDataSets) |
void |
fail(Throwable t) |
StringifiedAccumulatorResult[] |
getAggregatedUserAccumulatorsStringified()
Returns the aggregated user-defined accumulators as strings.
|
static ExecutionState |
getAggregateJobVertexState(int[] verticesPerState,
int parallelism)
A utility function that computes an "aggregated" state for the vertex.
|
ExecutionState |
getAggregateState()
Returns the aggregated
ExecutionState for this job vertex. |
CoLocationGroup |
getCoLocationGroup() |
ExecutionGraph |
getGraph() |
InputDependencyConstraint |
getInputDependencyConstraint() |
List<IntermediateResult> |
getInputs() |
JobID |
getJobId() |
JobVertex |
getJobVertex() |
JobVertexID |
getJobVertexId()
Returns the
JobVertexID for this job vertex. |
int |
getMaxParallelism()
Returns the max parallelism for this job vertex.
|
String |
getName()
Returns the name for this job vertex.
|
List<OperatorID> |
getOperatorIDs()
Returns a list containing the IDs of all operators contained in this execution job vertex.
|
int |
getParallelism()
Returns the parallelism for this job vertex.
|
IntermediateResult[] |
getProducedDataSets() |
SlotSharingGroup |
getSlotSharingGroup() |
InputSplitAssigner |
getSplitAssigner() |
Either<SerializedValue<TaskInformation>,PermanentBlobKey> |
getTaskInformationOrBlobKey() |
ExecutionVertex[] |
getTaskVertices()
Returns all execution vertices for this job vertex.
|
List<OperatorID> |
getUserDefinedOperatorIDs()
Returns a list containing the alternative IDs of all operators contained in this execution job vertex.
|
static Map<OperatorID,ExecutionJobVertex> |
includeAlternativeOperatorIDs(Map<OperatorID,ExecutionJobVertex> operatorMapping) |
static Map<JobVertexID,ExecutionJobVertex> |
includeLegacyJobVertexIDs(Map<JobVertexID,ExecutionJobVertex> tasks) |
boolean |
isMaxParallelismConfigured() |
void |
resetForNewExecution(long timestamp,
long expectedGlobalModVersion) |
CompletableFuture<Void> |
scheduleAll(SlotProvider slotProvider,
boolean queued,
LocationPreferenceConstraint locationPreferenceConstraint,
Set<AllocationID> allPreviousExecutionGraphAllocationIds)
Schedules all execution vertices of this ExecutionJobVertex.
|
void |
setMaxParallelism(int maxParallelismDerived) |
CompletableFuture<Void> |
suspend() |
public static final int VALUE_NOT_SET
public ExecutionJobVertex(ExecutionGraph graph, JobVertex jobVertex, int defaultParallelism, Time timeout, long initialGlobalModVersion, long createTimestamp) throws JobException
JobException
public List<OperatorID> getOperatorIDs()
public List<OperatorID> getUserDefinedOperatorIDs()
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 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()
public SlotSharingGroup getSlotSharingGroup()
public CoLocationGroup getCoLocationGroup()
public List<IntermediateResult> getInputs()
public InputDependencyConstraint getInputDependencyConstraint()
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 CompletableFuture<Void> scheduleAll(SlotProvider slotProvider, boolean queued, LocationPreferenceConstraint locationPreferenceConstraint, @Nonnull Set<AllocationID> allPreviousExecutionGraphAllocationIds)
slotProvider
- to allocate the slots fromqueued
- if the allocations can be queuedlocationPreferenceConstraint
- constraint for the location preferencesallPreviousExecutionGraphAllocationIds
- set with all previous allocation ids in the job graph.
Can be empty if the allocation ids are not required for scheduling.Execution
could be deployedpublic Collection<CompletableFuture<Execution>> allocateResourcesForAll(SlotProvider resourceProvider, boolean queued, LocationPreferenceConstraint locationPreferenceConstraint, @Nonnull Set<AllocationID> allPreviousExecutionGraphAllocationIds, Time allocationTimeout)
If this method throws an exception, it makes sure to release all so far requested slots.
resourceProvider
- The resource provider from whom the slots are requested.queued
- if the allocation can be queuedlocationPreferenceConstraint
- constraint for the location preferencesallPreviousExecutionGraphAllocationIds
- the allocation ids of all previous executions in the execution job graph.allocationTimeout
- timeout for allocating the individual slotspublic 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 ExecutionJobVertexpublic static Map<JobVertexID,ExecutionJobVertex> includeLegacyJobVertexIDs(Map<JobVertexID,ExecutionJobVertex> tasks)
public static Map<OperatorID,ExecutionJobVertex> includeAlternativeOperatorIDs(Map<OperatorID,ExecutionJobVertex> operatorMapping)
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.