You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/29 09:36:10 UTC

[GitHub] [flink] 6591812 commented on a diff in pull request #20082: [FLINK-28134][runtime] Introduce SpeculativeExecutionVertex

6591812 commented on code in PR #20082:
URL: https://github.com/apache/flink/pull/20082#discussion_r909414471


##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertexTest.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.JobStatus;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.TestingInternalFailuresListener;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link SpeculativeExecutionVertex}. */
+class SpeculativeExecutionVertexTest {
+
+    @RegisterExtension
+    private static final TestExecutorExtension<ScheduledExecutorService> EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    private TestingInternalFailuresListener internalFailuresListener;
+
+    @BeforeEach
+    void setUp() {
+        internalFailuresListener = new TestingInternalFailuresListener();
+    }
+
+    @Test
+    void testCreateSpeculativeExecution() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+
+        ev.createNewSpeculativeExecution(System.currentTimeMillis());
+        assertThat(ev.getCurrentExecutions()).hasSize(2);
+    }
+
+    @Test
+    void testResetExecutionVertex() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        e1.transitionState(ExecutionState.RUNNING);
+        e1.markFinished();
+        e2.cancel();
+        ev.resetForNewExecution();
+
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(0).get().getAttemptId())
+                .isEqualTo(e1.getAttemptId());
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(1).get().getAttemptId())
+                .isEqualTo(e2.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.getCurrentExecutionAttempt().getAttemptNumber()).isEqualTo(2);
+    }
+
+    @Test
+    void testCancel() throws Exception {

Review Comment:
   testCancelSpeculativeExecutionVertex()



##########
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:
   containsSourceVertexs



##########
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() {

Review Comment:
   containsSinkVertexes



##########
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);
+

Review Comment:
   this.currentExecutions = new LinkedHashMap<>(2);
   or
   this.currentExecutions = new LinkedHashMap<>("max execution from conf");



##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertexTest.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.JobStatus;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.TestingInternalFailuresListener;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link SpeculativeExecutionVertex}. */
+class SpeculativeExecutionVertexTest {
+
+    @RegisterExtension
+    private static final TestExecutorExtension<ScheduledExecutorService> EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    private TestingInternalFailuresListener internalFailuresListener;
+
+    @BeforeEach
+    void setUp() {
+        internalFailuresListener = new TestingInternalFailuresListener();
+    }
+
+    @Test
+    void testCreateSpeculativeExecution() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+
+        ev.createNewSpeculativeExecution(System.currentTimeMillis());
+        assertThat(ev.getCurrentExecutions()).hasSize(2);
+    }
+
+    @Test
+    void testResetExecutionVertex() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        e1.transitionState(ExecutionState.RUNNING);
+        e1.markFinished();
+        e2.cancel();
+        ev.resetForNewExecution();
+
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(0).get().getAttemptId())
+                .isEqualTo(e1.getAttemptId());
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(1).get().getAttemptId())
+                .isEqualTo(e2.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.getCurrentExecutionAttempt().getAttemptNumber()).isEqualTo(2);
+    }
+
+    @Test
+    void testCancel() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.cancel();
+        assertThat(e1.getState()).isSameAs(ExecutionState.CANCELED);
+        assertThat(e2.getState()).isSameAs(ExecutionState.CANCELED);
+    }
+
+    @Test
+    void testSuspend() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.suspend();
+        assertThat(e1.getState()).isSameAs(ExecutionState.CANCELED);
+        assertThat(e2.getState()).isSameAs(ExecutionState.CANCELED);
+    }
+
+    @Test
+    void testFail() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.fail(new Exception("Forced test failure."));
+        assertThat(internalFailuresListener.getFailedTasks())
+                .containsExactly(e1.getAttemptId(), e2.getAttemptId());
+    }
+
+    @Test
+    void testMarkFailed() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.markFailed(new Exception("Forced test failure."));
+        assertThat(internalFailuresListener.getFailedTasks())
+                .containsExactly(e1.getAttemptId(), e2.getAttemptId());
+    }
+
+    @Test
+    void testVertexTerminationAndJobTermination() throws Exception {
+        final JobVertex jobVertex = ExecutionGraphTestUtils.createNoOpVertex(1);
+        final JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(jobVertex);
+        final ExecutionGraph eg = createExecutionGraph(jobGraph);
+        eg.transitionToRunning();
+
+        final SpeculativeExecutionVertex ev =
+                (SpeculativeExecutionVertex)
+                        eg.getJobVertex(jobVertex.getID()).getTaskVertices()[0];
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+        final CompletableFuture<?> terminationFuture = ev.getTerminationFuture();
+
+        e1.transitionState(ExecutionState.RUNNING);
+        e1.markFinished();
+        assertThat(terminationFuture.isDone()).isFalse();
+        assertThat(eg.getState()).isSameAs(JobStatus.RUNNING);
+
+        e2.cancel();
+        assertThat(terminationFuture.isDone()).isTrue();
+        assertThat(eg.getState()).isSameAs(JobStatus.FINISHED);
+    }
+
+    @Test
+    void testArchiveFailedExecutions() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        e1.transitionState(ExecutionState.RUNNING);
+
+        final Execution e2 = ev.createNewSpeculativeExecution(0);
+        e2.transitionState(ExecutionState.FAILED);
+
+        ev.archiveFailedExecution(e2.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.currentExecution).isSameAs(e1);
+
+        final Execution e3 = ev.createNewSpeculativeExecution(0);
+        e3.transitionState(ExecutionState.RUNNING);
+        e1.transitionState(ExecutionState.FAILED);
+
+        ev.archiveFailedExecution(e1.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.currentExecution).isSameAs(e3);
+    }
+
+    @Test
+    void testArchiveTheOnlyCurrentExecution() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        e1.transitionState(ExecutionState.FAILED);
+
+        ev.archiveFailedExecution(e1.getAttemptId());
+
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.currentExecution).isSameAs(e1);
+    }
+
+    @Test
+    void testArchiveNonFailedExecutionWithArchiveFailedExecutionMethod() {
+        Assertions.assertThrows(
+                IllegalStateException.class,
+                () -> {
+                    final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+
+                    final Execution e1 = ev.getCurrentExecutionAttempt();
+                    e1.transitionState(ExecutionState.FAILED);
+
+                    final Execution e2 = ev.createNewSpeculativeExecution(0);

Review Comment:
   e2 --> speculativeExecution



##########
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:
   containsSourceVertexes



##########
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");

Review Comment:
   It's not allowed to get the partition producer of an un-finished SpeculativeExecutionVertex.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java:
##########
@@ -336,57 +357,56 @@ public void resetForNewExecution() {
     }
 
     private void resetForNewExecutionInternal(final long timestamp) {
-        final Execution oldExecution = currentExecution;
-        final ExecutionState oldState = oldExecution.getState();
-
-        if (oldState.isTerminal()) {
-            if (oldState == FINISHED) {
-                // pipelined partitions are released in Execution#cancel(), covering both job
-                // failures and vertex resets
-                // do not release pipelined partitions here to save RPC calls
-                oldExecution.handlePartitionCleanup(false, true);
-                getExecutionGraphAccessor()
-                        .getPartitionGroupReleaseStrategy()
-                        .vertexUnfinished(executionVertexId);
-            }
+        final boolean isFinished = (getExecutionState() == FINISHED);
 
-            executionHistory.add(oldExecution.archive());
+        resetExecutionsInternal();
 
-            final Execution newExecution =
-                    new Execution(
-                            getExecutionGraphAccessor().getFutureExecutor(),
-                            this,
-                            oldExecution.getAttemptNumber() + 1,
-                            timestamp,
-                            timeout);
+        InputSplitAssigner assigner = jobVertex.getSplitAssigner();
+        if (assigner != null) {
+            assigner.returnInputSplit(inputSplits, getParallelSubtaskIndex());
+            inputSplits.clear();
+        }
 
-            currentExecution = newExecution;
+        // if the execution was 'FINISHED' before, tell the ExecutionGraph that
+        // we take one step back on the road to reaching global FINISHED
+        if (isFinished) {
+            getJobVertex().executionVertexUnFinished();
+        }
 
-            synchronized (inputSplits) {
-                InputSplitAssigner assigner = jobVertex.getSplitAssigner();
-                if (assigner != null) {
-                    assigner.returnInputSplit(inputSplits, getParallelSubtaskIndex());
-                    inputSplits.clear();
-                }
-            }
+        // reset the intermediate results
+        for (IntermediateResultPartition resultPartition : resultPartitions.values()) {
+            resultPartition.resetForNewExecution();
+        }
 
-            // register this execution at the execution graph, to receive call backs
-            getExecutionGraphAccessor().registerExecution(newExecution);
+        final Execution newExecution = createNewExecution(timestamp);
+        currentExecution = newExecution;
 
-            // if the execution was 'FINISHED' before, tell the ExecutionGraph that
-            // we take one step back on the road to reaching global FINISHED
-            if (oldState == FINISHED) {
-                getJobVertex().executionVertexUnFinished();
-            }
+        // register this execution at the execution graph, to receive call backs

Review Comment:
   register this execution to the execution graph



##########
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() {

Review Comment:
   containsSinkVertexs



##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java:
##########
@@ -336,57 +357,56 @@ public void resetForNewExecution() {
     }
 
     private void resetForNewExecutionInternal(final long timestamp) {
-        final Execution oldExecution = currentExecution;
-        final ExecutionState oldState = oldExecution.getState();
-
-        if (oldState.isTerminal()) {
-            if (oldState == FINISHED) {
-                // pipelined partitions are released in Execution#cancel(), covering both job
-                // failures and vertex resets
-                // do not release pipelined partitions here to save RPC calls
-                oldExecution.handlePartitionCleanup(false, true);
-                getExecutionGraphAccessor()
-                        .getPartitionGroupReleaseStrategy()
-                        .vertexUnfinished(executionVertexId);
-            }
+        final boolean isFinished = (getExecutionState() == FINISHED);
 
-            executionHistory.add(oldExecution.archive());
+        resetExecutionsInternal();
 

Review Comment:
   Need to add synchronized (inputSplits) ?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertexTest.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.JobStatus;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.TestingInternalFailuresListener;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link SpeculativeExecutionVertex}. */
+class SpeculativeExecutionVertexTest {
+
+    @RegisterExtension
+    private static final TestExecutorExtension<ScheduledExecutorService> EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    private TestingInternalFailuresListener internalFailuresListener;
+
+    @BeforeEach
+    void setUp() {
+        internalFailuresListener = new TestingInternalFailuresListener();
+    }
+
+    @Test
+    void testCreateSpeculativeExecution() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+
+        ev.createNewSpeculativeExecution(System.currentTimeMillis());
+        assertThat(ev.getCurrentExecutions()).hasSize(2);
+    }
+
+    @Test
+    void testResetExecutionVertex() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        e1.transitionState(ExecutionState.RUNNING);
+        e1.markFinished();
+        e2.cancel();
+        ev.resetForNewExecution();
+
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(0).get().getAttemptId())
+                .isEqualTo(e1.getAttemptId());
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(1).get().getAttemptId())
+                .isEqualTo(e2.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.getCurrentExecutionAttempt().getAttemptNumber()).isEqualTo(2);
+    }
+
+    @Test
+    void testCancel() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.cancel();
+        assertThat(e1.getState()).isSameAs(ExecutionState.CANCELED);
+        assertThat(e2.getState()).isSameAs(ExecutionState.CANCELED);
+    }
+
+    @Test
+    void testSuspend() throws Exception {

Review Comment:
   testSuspendSpeculativeExecutionVertex()



##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertexTest.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.JobStatus;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.TestingInternalFailuresListener;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link SpeculativeExecutionVertex}. */
+class SpeculativeExecutionVertexTest {
+
+    @RegisterExtension
+    private static final TestExecutorExtension<ScheduledExecutorService> EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    private TestingInternalFailuresListener internalFailuresListener;
+
+    @BeforeEach
+    void setUp() {
+        internalFailuresListener = new TestingInternalFailuresListener();
+    }
+
+    @Test
+    void testCreateSpeculativeExecution() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+
+        ev.createNewSpeculativeExecution(System.currentTimeMillis());
+        assertThat(ev.getCurrentExecutions()).hasSize(2);
+    }
+
+    @Test
+    void testResetExecutionVertex() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        e1.transitionState(ExecutionState.RUNNING);
+        e1.markFinished();
+        e2.cancel();
+        ev.resetForNewExecution();
+
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(0).get().getAttemptId())
+                .isEqualTo(e1.getAttemptId());
+        assertThat(ev.getExecutionHistory().getHistoricalExecution(1).get().getAttemptId())
+                .isEqualTo(e2.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.getCurrentExecutionAttempt().getAttemptNumber()).isEqualTo(2);
+    }
+
+    @Test
+    void testCancel() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.cancel();
+        assertThat(e1.getState()).isSameAs(ExecutionState.CANCELED);
+        assertThat(e2.getState()).isSameAs(ExecutionState.CANCELED);
+    }
+
+    @Test
+    void testSuspend() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.suspend();
+        assertThat(e1.getState()).isSameAs(ExecutionState.CANCELED);
+        assertThat(e2.getState()).isSameAs(ExecutionState.CANCELED);
+    }
+
+    @Test
+    void testFail() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.fail(new Exception("Forced test failure."));
+        assertThat(internalFailuresListener.getFailedTasks())
+                .containsExactly(e1.getAttemptId(), e2.getAttemptId());
+    }
+
+    @Test
+    void testMarkFailed() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+
+        ev.markFailed(new Exception("Forced test failure."));
+        assertThat(internalFailuresListener.getFailedTasks())
+                .containsExactly(e1.getAttemptId(), e2.getAttemptId());
+    }
+
+    @Test
+    void testVertexTerminationAndJobTermination() throws Exception {
+        final JobVertex jobVertex = ExecutionGraphTestUtils.createNoOpVertex(1);
+        final JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(jobVertex);
+        final ExecutionGraph eg = createExecutionGraph(jobGraph);
+        eg.transitionToRunning();
+
+        final SpeculativeExecutionVertex ev =
+                (SpeculativeExecutionVertex)
+                        eg.getJobVertex(jobVertex.getID()).getTaskVertices()[0];
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        final Execution e2 = ev.createNewSpeculativeExecution(System.currentTimeMillis());
+        final CompletableFuture<?> terminationFuture = ev.getTerminationFuture();
+
+        e1.transitionState(ExecutionState.RUNNING);
+        e1.markFinished();
+        assertThat(terminationFuture.isDone()).isFalse();
+        assertThat(eg.getState()).isSameAs(JobStatus.RUNNING);
+
+        e2.cancel();
+        assertThat(terminationFuture.isDone()).isTrue();
+        assertThat(eg.getState()).isSameAs(JobStatus.FINISHED);
+    }
+
+    @Test
+    void testArchiveFailedExecutions() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        e1.transitionState(ExecutionState.RUNNING);
+
+        final Execution e2 = ev.createNewSpeculativeExecution(0);
+        e2.transitionState(ExecutionState.FAILED);
+
+        ev.archiveFailedExecution(e2.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.currentExecution).isSameAs(e1);
+
+        final Execution e3 = ev.createNewSpeculativeExecution(0);
+        e3.transitionState(ExecutionState.RUNNING);
+        e1.transitionState(ExecutionState.FAILED);
+
+        ev.archiveFailedExecution(e1.getAttemptId());
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.currentExecution).isSameAs(e3);
+    }
+
+    @Test
+    void testArchiveTheOnlyCurrentExecution() throws Exception {
+        final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+
+        final Execution e1 = ev.getCurrentExecutionAttempt();
+        e1.transitionState(ExecutionState.FAILED);
+
+        ev.archiveFailedExecution(e1.getAttemptId());
+
+        assertThat(ev.getCurrentExecutions()).hasSize(1);
+        assertThat(ev.currentExecution).isSameAs(e1);
+    }
+
+    @Test
+    void testArchiveNonFailedExecutionWithArchiveFailedExecutionMethod() {
+        Assertions.assertThrows(
+                IllegalStateException.class,
+                () -> {
+                    final SpeculativeExecutionVertex ev = createSpeculativeExecutionVertex();
+
+                    final Execution e1 = ev.getCurrentExecutionAttempt();

Review Comment:
   e1 -->currentExecution



##########
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:
   containsSourceVertexes



##########
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<>();

Review Comment:
   final List<CompletableFuture<?>> cancelResultFutures = new ArrayList<>(currentExecutions.size());



##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java:
##########
@@ -133,24 +137,33 @@ public ExecutionVertex(
 
         this.executionHistory = new ExecutionHistory(executionHistorySizeLimit);
 
-        this.currentExecution =
-                new Execution(
-                        getExecutionGraphAccessor().getFutureExecutor(),
-                        this,
-                        initialAttemptCount,
-                        createTimestamp,
-                        timeout);
-
-        getExecutionGraphAccessor().registerExecution(currentExecution);
+        this.nextAttemptNumber = initialAttemptCount;
 
         this.timeout = timeout;

Review Comment:
   add an empty line



##########
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:
   containsSourceVertexes



##########
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(

Review Comment:
   String.format() is heavy. Could cut it?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org