wanglijie95 commented on code in PR #20082:
URL: https://github.com/apache/flink/pull/20082#discussion_r909443929
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java:
##########
@@ -241,32 +254,36 @@ public Execution getCurrentExecutionAttempt() {
return currentExecution;
}
+ public Collection<Execution> getCurrentExecutions() {
Review Comment:
This method is nerver used.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraph.java:
##########
@@ -1161,29 +1169,44 @@ public void jobVertexFinished() {
assertRunningInJobMasterMainThread();
final int numFinished = ++numFinishedJobVertices;
if (numFinished == numJobVerticesTotal) {
- // done :-)
+ FutureUtils.assertNoException(
+ waitForAllExecutionsTermination().thenAccept(ignored ->
jobFinished()));
+ }
+ }
- // check whether we are still in "RUNNING" and trigger the final
cleanup
- if (state == JobStatus.RUNNING) {
- // we do the final cleanup in the I/O executor, because it may
involve
- // some heavier work
+ private CompletableFuture<?> waitForAllExecutionsTermination() {
+ final List<CompletableFuture<?>> terminationFutures =
+ verticesInCreationOrder.stream()
+ .flatMap(ejv -> Arrays.stream(ejv.getTaskVertices()))
+ .map(ev -> ev.getTerminationFuture())
Review Comment:
Maybe `ev -> ev.getTerminationFuture()` ->
`ExecutionVertex::getTerminationFuture` ?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionJobVertex.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.VertexParallelismInformation;
+
+/** The ExecutionJobVertex which supports speculative execution. */
+public class SpeculativeExecutionJobVertex extends ExecutionJobVertex {
Review Comment:
Is this class really needed? It 's only used to create
`SpeculativeExecutionVertex`, how about remove it and introduce
`ExecutionVertex.Factory` ?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
+import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The ExecutionVertex which supports speculative execution. */
+public class SpeculativeExecutionVertex extends ExecutionVertex {
+
+ private final Map<ExecutionAttemptID, Execution> currentExecutions;
+
+ public SpeculativeExecutionVertex(
+ ExecutionJobVertex jobVertex,
+ int subTaskIndex,
+ IntermediateResult[] producedDataSets,
+ Time timeout,
+ long createTimestamp,
+ int maxPriorExecutionHistoryLength,
+ int initialAttemptCount) {
+ super(
+ jobVertex,
+ subTaskIndex,
+ producedDataSets,
+ timeout,
+ createTimestamp,
+ maxPriorExecutionHistoryLength,
+ initialAttemptCount);
+
+ this.currentExecutions = new LinkedHashMap<>();
+ this.currentExecutions.put(currentExecution.getAttemptId(),
currentExecution);
+ }
+
+ public boolean containsSources() {
+ return getJobVertex().getJobVertex().containsSources();
+ }
+
+ public boolean containsSinks() {
+ return getJobVertex().getJobVertex().containsSinks();
+ }
+
+ public Execution createNewSpeculativeExecution(final long timestamp) {
+ final Execution newExecution = createNewExecution(timestamp);
+ getExecutionGraphAccessor().registerExecution(newExecution);
+ currentExecutions.put(newExecution.getAttemptId(), newExecution);
+ return newExecution;
+ }
+
+ @Override
+ public Collection<Execution> getCurrentExecutions() {
+ return Collections.unmodifiableCollection(currentExecutions.values());
+ }
+
+ @Override
+ public Execution getPartitionProducer() {
+ final Execution finishedExecution = getCurrentExecutionAttempt();
+ checkState(
+ finishedExecution.getState() == FINISHED,
+ "It's not allowed to get the partition producer of an
un-finished speculative execution vertex");
+ return finishedExecution;
+ }
+
+ @Override
+ public CompletableFuture<?> cancel() {
+ final List<CompletableFuture<?>> cancelResultFutures = new
ArrayList<>();
+ for (Execution execution : currentExecutions.values()) {
+ execution.cancel();
+ cancelResultFutures.add(execution.getReleaseFuture());
+ }
+ return FutureUtils.combineAll(cancelResultFutures);
+ }
+
+ @Override
+ public CompletableFuture<?> suspend() {
+ return FutureUtils.combineAll(
+ currentExecutions.values().stream()
+ .map(Execution::suspend)
+ .collect(Collectors.toList()));
+ }
+
+ @Override
+ public void fail(Throwable t) {
+ currentExecutions.values().forEach(e -> e.fail(t));
+ }
+
+ @Override
+ public void markFailed(Throwable t) {
+ currentExecutions.values().forEach(e -> e.markFailed(t));
+ }
+
+ @Override
+ CompletableFuture<?> getTerminationFuture() {
+ final List<CompletableFuture<?>> terminationFutures =
+ currentExecutions.values().stream()
+ .map(e -> e.getTerminalStateFuture())
+ .collect(Collectors.toList());
+ return FutureUtils.waitForAll(terminationFutures);
+ }
+
+ @Override
+ public void resetForNewExecution() {
+ super.resetForNewExecution();
+
+ currentExecutions.clear();
+ currentExecutions.put(currentExecution.getAttemptId(),
currentExecution);
+ }
+
+ @Override
+ void resetExecutionsInternal() {
+ for (Execution execution : currentExecutions.values()) {
+ resetExecution(execution);
+ }
+ }
+
+ /**
+ * Remove execution from currentExecutions if it is failed. It is needed
to make room for
+ * possible future speculative executions.
+ *
+ * @param executionAttemptId attemptID of the execution to be removed
+ */
+ public void archiveFailedExecution(ExecutionAttemptID executionAttemptId) {
+ if (this.currentExecutions.size() <= 1) {
+ // Leave the last execution because currentExecutions should never
be empty. This should
+ // happen only if all current executions have FAILED. A vertex
reset will happen soon
+ // and will archive the remaining execution.
+ return;
+ }
+
+ final Execution removedExecution =
this.currentExecutions.remove(executionAttemptId);
+ checkNotNull(
+ removedExecution,
+ String.format(
+ "Cannot remove execution %s which does not exist.",
executionAttemptId));
+ checkState(
+ removedExecution.getState() == FAILED,
+ "Cannot remove execution %s which is not FAILED.",
+ executionAttemptId);
+
+ executionHistory.add(removedExecution.archive());
+ if (removedExecution == this.currentExecution) {
+ this.currentExecution =
this.currentExecutions.values().iterator().next();
+ }
+ }
+
+ @Override
+ public Execution getCurrentExecutionAttempt() {
Review Comment:
I'm not quite sure whether the `currentExecution` will be used
directly(instead of by `getCurrentExecutionAttempt()`). How about updating
`currentExecution` at each `updateTaskExecutionState` ?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
+import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The ExecutionVertex which supports speculative execution. */
+public class SpeculativeExecutionVertex extends ExecutionVertex {
+
+ private final Map<ExecutionAttemptID, Execution> currentExecutions;
+
+ public SpeculativeExecutionVertex(
+ ExecutionJobVertex jobVertex,
+ int subTaskIndex,
+ IntermediateResult[] producedDataSets,
+ Time timeout,
+ long createTimestamp,
+ int maxPriorExecutionHistoryLength,
+ int initialAttemptCount) {
+ super(
+ jobVertex,
+ subTaskIndex,
+ producedDataSets,
+ timeout,
+ createTimestamp,
+ maxPriorExecutionHistoryLength,
+ initialAttemptCount);
+
+ this.currentExecutions = new LinkedHashMap<>();
+ this.currentExecutions.put(currentExecution.getAttemptId(),
currentExecution);
+ }
+
+ public boolean containsSources() {
Review Comment:
`containsSources` and `containsSinks` is never used in this PR
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionJobVertex.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.VertexParallelismInformation;
+
+/** The ExecutionJobVertex which supports speculative execution. */
+public class SpeculativeExecutionJobVertex extends ExecutionJobVertex {
Review Comment:
can be private
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
+import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The ExecutionVertex which supports speculative execution. */
+public class SpeculativeExecutionVertex extends ExecutionVertex {
+
+ private final Map<ExecutionAttemptID, Execution> currentExecutions;
+
+ public SpeculativeExecutionVertex(
+ ExecutionJobVertex jobVertex,
+ int subTaskIndex,
+ IntermediateResult[] producedDataSets,
+ Time timeout,
+ long createTimestamp,
+ int maxPriorExecutionHistoryLength,
+ int initialAttemptCount) {
+ super(
+ jobVertex,
+ subTaskIndex,
+ producedDataSets,
+ timeout,
+ createTimestamp,
+ maxPriorExecutionHistoryLength,
+ initialAttemptCount);
+
+ this.currentExecutions = new LinkedHashMap<>();
+ this.currentExecutions.put(currentExecution.getAttemptId(),
currentExecution);
+ }
+
+ public boolean containsSources() {
+ return getJobVertex().getJobVertex().containsSources();
+ }
+
+ public boolean containsSinks() {
+ return getJobVertex().getJobVertex().containsSinks();
+ }
+
+ public Execution createNewSpeculativeExecution(final long timestamp) {
+ final Execution newExecution = createNewExecution(timestamp);
+ getExecutionGraphAccessor().registerExecution(newExecution);
+ currentExecutions.put(newExecution.getAttemptId(), newExecution);
+ return newExecution;
+ }
+
+ @Override
+ public Collection<Execution> getCurrentExecutions() {
+ return Collections.unmodifiableCollection(currentExecutions.values());
+ }
+
+ @Override
+ public Execution getPartitionProducer() {
+ final Execution finishedExecution = getCurrentExecutionAttempt();
+ checkState(
+ finishedExecution.getState() == FINISHED,
+ "It's not allowed to get the partition producer of an
un-finished speculative execution vertex");
+ return finishedExecution;
+ }
+
+ @Override
+ public CompletableFuture<?> cancel() {
+ final List<CompletableFuture<?>> cancelResultFutures = new
ArrayList<>();
+ for (Execution execution : currentExecutions.values()) {
+ execution.cancel();
+ cancelResultFutures.add(execution.getReleaseFuture());
+ }
+ return FutureUtils.combineAll(cancelResultFutures);
+ }
+
+ @Override
+ public CompletableFuture<?> suspend() {
+ return FutureUtils.combineAll(
+ currentExecutions.values().stream()
+ .map(Execution::suspend)
+ .collect(Collectors.toList()));
+ }
+
+ @Override
+ public void fail(Throwable t) {
+ currentExecutions.values().forEach(e -> e.fail(t));
+ }
+
+ @Override
+ public void markFailed(Throwable t) {
+ currentExecutions.values().forEach(e -> e.markFailed(t));
+ }
+
+ @Override
+ CompletableFuture<?> getTerminationFuture() {
+ final List<CompletableFuture<?>> terminationFutures =
+ currentExecutions.values().stream()
+ .map(e -> e.getTerminalStateFuture())
Review Comment:
Maybe `Execution::getTerminalStateFuture`
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedSpeculativeExecutionVertex.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+/**
+ * {@code ArchivedSpeculativeExecutionVertex} is a readonly representation of
{@link
+ * SpeculativeExecutionVertex}.
+ */
+public class ArchivedSpeculativeExecutionVertex extends
ArchivedExecutionVertex {
+
+ private static final long serialVersionUID = 1L;
+
+ public ArchivedSpeculativeExecutionVertex(SpeculativeExecutionVertex
vertex) {
+ super(
+ vertex.getParallelSubtaskIndex(),
+ vertex.getTaskNameWithSubtaskIndex(),
+ vertex.getCurrentExecutionAttempt().archive(),
+ getCopyOfExecutionHistory(vertex));
+ }
+
+ private static ExecutionHistory
getCopyOfExecutionHistory(SpeculativeExecutionVertex vertex) {
+ final ExecutionHistory executionHistory =
+ ArchivedExecutionVertex.getCopyOfExecutionHistory(vertex);
+
+ // add all the executions to the execution history except for the only
admitted current
+ // execution
Review Comment:
I can't fully understand the logic here. Should a `RUNNING` execution
attempt be archived?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
+import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The ExecutionVertex which supports speculative execution. */
+public class SpeculativeExecutionVertex extends ExecutionVertex {
Review Comment:
can be package-private
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedSpeculativeExecutionVertex.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+/**
+ * {@code ArchivedSpeculativeExecutionVertex} is a readonly representation of
{@link
Review Comment:
`@code` -> `@link` ?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionJobVertex.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.VertexParallelismInformation;
+
+/** The ExecutionJobVertex which supports speculative execution. */
+public class SpeculativeExecutionJobVertex extends ExecutionJobVertex {
+
+ public SpeculativeExecutionJobVertex(
+ InternalExecutionGraphAccessor graph,
+ JobVertex jobVertex,
+ VertexParallelismInformation parallelismInfo)
+ throws JobException {
+ super(graph, jobVertex, parallelismInfo);
+ }
+
+ @Override
+ protected ExecutionVertex createExecutionVertex(
+ ExecutionJobVertex jobVertex,
+ int subTaskIndex,
+ IntermediateResult[] producedDataSets,
+ Time timeout,
+ long createTimestamp,
+ int maxPriorExecutionHistoryLength,
Review Comment:
`maxPriorExecutionHistoryLength` -> `executionHistorySizeLimit`, maybe you
can search it globallly.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]