You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2019/07/01 11:33:07 UTC

[flink] 02/02: [FLINK-12997][coordination] Release partitions on vertex reset

This is an automated email from the ASF dual-hosted git repository.

chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 6f6cfde0cea12f8d6966c48ba3124c53db15c9bc
Author: Chesnay Schepler <ch...@apache.org>
AuthorDate: Wed Jun 26 13:41:51 2019 +0200

    [FLINK-12997][coordination] Release partitions on vertex reset
---
 .../flink/runtime/executiongraph/Execution.java    |  2 +-
 .../runtime/executiongraph/ExecutionVertex.java    |  4 +
 .../executiongraph/ExecutionVertexTest.java        | 86 ++++++++++++++++++++++
 3 files changed, 91 insertions(+), 1 deletion(-)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index c033304..4939e61 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -1335,7 +1335,7 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 		}
 	}
 
-	private void stopTrackingAndReleasePartitions() {
+	void stopTrackingAndReleasePartitions() {
 		LOG.info("Discarding the results produced by task execution {}.", attemptId);
 		if (producedPartitions != null && producedPartitions.size() > 0) {
 			final PartitionTracker partitionTracker = getVertex().getExecutionGraph().getPartitionTracker();
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index 0d01e25..a12d198 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -591,6 +591,10 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
 			final ExecutionState oldState = oldExecution.getState();
 
 			if (oldState.isTerminal()) {
+				if (oldState == FINISHED) {
+					oldExecution.stopTrackingAndReleasePartitions();
+				}
+
 				priorExecutions.add(oldExecution.archive());
 
 				final Execution newExecution = new Execution(
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java
new file mode 100644
index 0000000..c9f3ef0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for the {@link ExecutionVertex}.
+ */
+public class ExecutionVertexTest extends TestLogger {
+
+	@Test
+	public void testResetForNewExecutionReleasesPartitions() throws Exception {
+		final JobVertex producerJobVertex = ExecutionGraphTestUtils.createNoOpVertex(1);
+		final JobVertex consumerJobVertex = ExecutionGraphTestUtils.createNoOpVertex(1);
+
+		consumerJobVertex.connectNewDataSetAsInput(producerJobVertex, DistributionPattern.POINTWISE, ResultPartitionType.BLOCKING);
+
+		final CompletableFuture<Collection<ResultPartitionID>> releasePartitionsFuture = new CompletableFuture<>();
+		final TestingPartitionTracker partitionTracker = new TestingPartitionTracker();
+		partitionTracker.setStopTrackingAndReleasePartitionsConsumer(releasePartitionsFuture::complete);
+
+		final ExecutionGraph executionGraph = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(producerJobVertex, consumerJobVertex)
+			.setPartitionTracker(partitionTracker)
+			.build();
+
+		executionGraph.scheduleForExecution();
+
+		final ExecutionJobVertex producerExecutionJobVertex = executionGraph.getJobVertex(producerJobVertex.getID());
+
+		Execution execution = producerExecutionJobVertex
+			.getTaskVertices()[0]
+			.getCurrentExecutionAttempt();
+
+		assertFalse(releasePartitionsFuture.isDone());
+
+		execution.markFinished();
+
+		assertFalse(releasePartitionsFuture.isDone());
+
+		producerExecutionJobVertex.resetForNewExecution(1L, 1L);
+
+		final IntermediateResultPartitionID intermediateResultPartitionID = producerExecutionJobVertex
+			.getProducedDataSets()[0]
+			.getPartitions()[0]
+			.getPartitionId();
+		final ResultPartitionID resultPartitionID = execution
+			.getResultPartitionDeploymentDescriptor(intermediateResultPartitionID)
+			.get()
+			.getShuffleDescriptor()
+			.getResultPartitionID();
+
+		assertThat(releasePartitionsFuture.get(), contains(resultPartitionID));
+	}
+}